diff --git a/examples/common/env.sh b/examples/common/env.sh index 3a0a9cf43f2..e0c61a3ff92 100644 --- a/examples/common/env.sh +++ b/examples/common/env.sh @@ -26,7 +26,7 @@ fi # mysqld might be in /usr/sbin which will not be in the default PATH PATH="/usr/sbin:$PATH" -for binary in mysqld etcd etcdctl curl vtctlclient vttablet vtgate vtctld mysqlctl; do +for binary in mysqld etcd etcdctl curl vtctlclient vtctldclient vttablet vtgate vtctld mysqlctl; do command -v "$binary" > /dev/null || fail "${binary} is not installed in PATH. See https://vitess.io/docs/get-started/local/ for install instructions." done; diff --git a/examples/common/lib/utils.sh b/examples/common/lib/utils.sh index 66af7d31bd7..ed93f139a52 100644 --- a/examples/common/lib/utils.sh +++ b/examples/common/lib/utils.sh @@ -139,6 +139,35 @@ function wait_for_healthy_shard() { wait_for_shard_vreplication_engine "${keyspace}" "${shard}" } +# Wait for a workflow to reach the running state. Example: +# wait_for_workflow_running customer customer2customer +function wait_for_workflow_running() { + if [[ -z ${1} || -z ${2} ]]; then + fail "A keyspace and workflow must be specified when waiting for a workflow to reach the running state" + fi + + local keyspace=${1} + local workflow=${2} + local wait_secs=90 + local result="" + + echo "Waiting for the ${workflow} workflow in the ${keyspace} keyspace to finish the copy phase..." + + for _ in $(seq 1 ${wait_secs}); do + result=$(vtctldclient Workflow --keyspace="${keyspace}" show --workflow="${workflow}" 2>/dev/null | grep "Copy phase completed") + if [[ ${result} != "" ]]; then + break + fi + sleep 1 + done; + + if [[ ${result} == "" ]]; then + fail "Timed out after ${wait_secs} seconds waiting for the ${workflow} workflow in the ${keyspace} keyspace to reach the running state" + fi + + echo "The ${workflow} workflow in the ${keyspace} keyspace is now running. $(sed -rn 's/.*"(Copy phase.*)".*/\1/p' <<< "${result}")." +} + # Stop the specified binary name using the provided PID file. # Example: # stop_process "vtadmin-web" "$VTDATAROOT/tmp/vtadmin-web.pid" diff --git a/examples/local/202_move_tables.sh b/examples/local/202_move_tables.sh index f385acb12a3..a4a24150973 100755 --- a/examples/local/202_move_tables.sh +++ b/examples/local/202_move_tables.sh @@ -19,4 +19,7 @@ source ../common/env.sh -vtctlclient MoveTables -- --source commerce --tables 'customer,corder' Create customer.commerce2customer +vtctldclient MoveTables --workflow commerce2customer --target-keyspace customer create --source-keyspace commerce --tables "customer,corder" + +# Wait for the workflow to reach the running state. +wait_for_workflow_running customer commerce2customer diff --git a/examples/local/203_switch_reads.sh b/examples/local/203_switch_reads.sh index 4bca7e4e257..a307c583171 100755 --- a/examples/local/203_switch_reads.sh +++ b/examples/local/203_switch_reads.sh @@ -19,4 +19,4 @@ source ../common/env.sh -vtctlclient MoveTables -- --tablet_types=rdonly,replica SwitchTraffic customer.commerce2customer +vtctldclient MoveTables --workflow commerce2customer --target-keyspace customer switchtraffic --tablet-types "rdonly,replica" diff --git a/examples/local/204_switch_writes.sh b/examples/local/204_switch_writes.sh index 743ca1e2512..8305356a1cf 100755 --- a/examples/local/204_switch_writes.sh +++ b/examples/local/204_switch_writes.sh @@ -19,4 +19,4 @@ source ../common/env.sh -vtctlclient MoveTables -- --tablet_types=primary SwitchTraffic customer.commerce2customer +vtctldclient MoveTables --workflow commerce2customer --target-keyspace customer switchtraffic --tablet-types primary diff --git a/examples/local/205_clean_commerce.sh b/examples/local/205_clean_commerce.sh index 5d307a231d3..127437d1d1b 100755 --- a/examples/local/205_clean_commerce.sh +++ b/examples/local/205_clean_commerce.sh @@ -19,5 +19,4 @@ source ../common/env.sh -vtctlclient MoveTables Complete customer.commerce2customer - +vtctldclient MoveTables --workflow commerce2customer --target-keyspace customer complete diff --git a/go/cmd/internal/docgen/docgen.go b/go/cmd/internal/docgen/docgen.go index d7c1b45b154..3fdaf7a59bf 100644 --- a/go/cmd/internal/docgen/docgen.go +++ b/go/cmd/internal/docgen/docgen.go @@ -156,7 +156,7 @@ func restructure(rootDir string, dir string, name string, commands []*cobra.Comm } func newParentLinkSedCommand(parent string, file string) *exec.Cmd { - return exec.Command("sed", "-i ''", "-e", fmt.Sprintf("s:(./%s/):(../):", parent), file) + return exec.Command("sed", "-i", "", "-e", fmt.Sprintf("s:(./%s/):(../):i", parent), file) } func recursivelyDisableAutoGenTags(root *cobra.Command) { diff --git a/go/cmd/vtctldclient/cli/json.go b/go/cmd/vtctldclient/cli/json.go index 80af6d80d72..c76a505e670 100644 --- a/go/cmd/vtctldclient/cli/json.go +++ b/go/cmd/vtctldclient/cli/json.go @@ -54,3 +54,25 @@ func MarshalJSON(obj any) ([]byte, error) { return data, nil } } + +// MarshalJSONCompact works the same as MarshalJSON but elides zero value elements. +func MarshalJSONCompact(obj any) ([]byte, error) { + switch obj := obj.(type) { + case proto.Message: + m := protojson.MarshalOptions{ + Multiline: true, + Indent: " ", + UseEnumNumbers: true, + UseProtoNames: true, + EmitUnpopulated: false, // elide zero value elements + } + return m.Marshal(obj) + default: + data, err := json.MarshalIndent(obj, "", " ") + if err != nil { + return nil, fmt.Errorf("json.Marshal = %v", err) + } + + return data, nil + } +} diff --git a/go/cmd/vtctldclient/command/movetables.go b/go/cmd/vtctldclient/command/movetables.go new file mode 100644 index 00000000000..50caa99669b --- /dev/null +++ b/go/cmd/vtctldclient/command/movetables.go @@ -0,0 +1,545 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package command + +import ( + "bytes" + "fmt" + "sort" + "strings" + "time" + + "github.com/spf13/cobra" + + "vitess.io/vitess/go/cmd/vtctldclient/cli" + "vitess.io/vitess/go/protoutil" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vtctl/workflow" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" +) + +var ( + // The generic default for most commands. + tabletTypesDefault = []topodatapb.TabletType{ + topodatapb.TabletType_REPLICA, + topodatapb.TabletType_PRIMARY, + } + onDDLDefault = binlogdatapb.OnDDLAction_IGNORE.String() + maxReplicationLagDefault = 30 * time.Second + timeoutDefault = 30 * time.Second + + // MoveTables is the base command for all related actions. + MoveTables = &cobra.Command{ + Use: "MoveTables --workflow --target-keyspace [command] [command-flags]", + Short: "Perform commands related to moving tables from a source keyspace to a target keyspace.", + Long: `MoveTables commands: Create, Show, Status, SwitchTraffic, ReverseTraffic, Stop, Start, Cancel, and Delete. +See the --help output for each command for more details.`, + DisableFlagsInUseLine: true, + Aliases: []string{"movetables"}, + Args: cobra.ExactArgs(1), + } + + // MoveTablesCancel makes a MoveTablesCancel gRPC call to a vtctld. + MoveTablesCancel = &cobra.Command{ + Use: "cancel", + Short: "Cancel a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer cancel`, + DisableFlagsInUseLine: true, + Aliases: []string{"Cancel"}, + Args: cobra.NoArgs, + RunE: commandMoveTablesCancel, + } + + // MoveTablesComplete makes a MoveTablesComplete gRPC call to a vtctld. + MoveTablesComplete = &cobra.Command{ + Use: "complete", + Short: "Complete a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer complete`, + DisableFlagsInUseLine: true, + Aliases: []string{"Complete"}, + Args: cobra.NoArgs, + RunE: commandMoveTablesComplete, + } + + // MoveTablesCreate makes a MoveTablesCreate gRPC call to a vtctld. + MoveTablesCreate = &cobra.Command{ + Use: "create", + Short: "Create and optionally run a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer create --source-keyspace commerce --cells zone1 --cells zone2 --tablet-types replica`, + SilenceUsage: true, + DisableFlagsInUseLine: true, + Aliases: []string{"Create"}, + Args: cobra.NoArgs, + PreRunE: func(cmd *cobra.Command, args []string) error { + // Either specific tables or the all tables flags are required. + if !cmd.Flags().Lookup("tables").Changed && !cmd.Flags().Lookup("all-tables").Changed { + return fmt.Errorf("tables or all-tables are required to specify which tables to move") + } + if cmd.Flags().Lookup("cells").Changed { // Validate the provided value(s) + for i, cell := range moveTablesCreateOptions.Cells { // Which only means trimming whitespace + moveTablesCreateOptions.Cells[i] = strings.TrimSpace(cell) + } + } + if !cmd.Flags().Lookup("tablet-types").Changed { + moveTablesCreateOptions.TabletTypes = tabletTypesDefault + } + if _, ok := binlogdatapb.OnDDLAction_value[strings.ToUpper(moveTablesCreateOptions.OnDDL)]; !ok { + return fmt.Errorf("invalid on-ddl value: %s", moveTablesCreateOptions.OnDDL) + } + return nil + }, + RunE: commandMoveTablesCreate, + } + + // MoveTablesShow makes a GetWorkflows gRPC call to a vtctld. + MoveTablesShow = &cobra.Command{ + Use: "show", + Short: "Show the details for a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer show`, + DisableFlagsInUseLine: true, + Aliases: []string{"Show"}, + Args: cobra.NoArgs, + RunE: commandMoveTablesShow, + } + + // MoveTablesStart makes a WorfklowUpdate gRPC call to a vtctld. + MoveTablesStart = &cobra.Command{ + Use: "start", + Short: "Start the MoveTables workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer start`, + DisableFlagsInUseLine: true, + Aliases: []string{"Start"}, + Args: cobra.NoArgs, + PreRun: bridgeMoveTablesToWorkflow, + RunE: commandWorkflowUpdateState, + } + + // MoveTablesStatus makes a GetWorkflows gRPC call to a vtctld. + MoveTablesStatus = &cobra.Command{ + Use: "status", + Short: "Show the current status for a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 MoveTables --workflow commerce2customer --target-keyspace customer status`, + DisableFlagsInUseLine: true, + Aliases: []string{"Status", "progress", "Progress"}, + Args: cobra.NoArgs, + RunE: commandMoveTablesStatus, + } + + // MoveTablesStop makes a WorfklowUpdate gRPC call to a vtctld. + MoveTablesStop = &cobra.Command{ + Use: "stop", + Short: "Stop a MoveTables workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer stop`, + DisableFlagsInUseLine: true, + Aliases: []string{"Stop"}, + Args: cobra.NoArgs, + PreRun: bridgeMoveTablesToWorkflow, + RunE: commandWorkflowUpdateState, + } + + // MoveTablesReverseTraffic makes a WorkflowSwitchTraffic gRPC call to a vtctld. + MoveTablesReverseTraffic = &cobra.Command{ + Use: "reversetraffic", + Short: "Reverse traffic for a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer reversetraffic`, + DisableFlagsInUseLine: true, + Aliases: []string{"ReverseTraffic"}, + Args: cobra.NoArgs, + PreRunE: func(cmd *cobra.Command, args []string) error { + moveTablesSwitchTrafficOptions.Direction = workflow.DirectionBackward + if !cmd.Flags().Lookup("tablet-types").Changed { + // We switch traffic for all tablet types if none are provided. + moveTablesSwitchTrafficOptions.TabletTypes = []topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_REPLICA, + topodatapb.TabletType_RDONLY, + } + } + return nil + }, + RunE: commandMoveTablesSwitchTraffic, + } + + // MoveTablesSwitchTraffic makes a MoveTablesSwitchTraffic gRPC call to a vtctld. + MoveTablesSwitchTraffic = &cobra.Command{ + Use: "switchtraffic", + Short: "Switch traffic for a MoveTables VReplication workflow.", + Example: `vtctldclient --server localhost:15999 movetables --workflow commerce2customer --target-keyspace customer switchtraffic --tablet-types "replica,rdonly"`, + DisableFlagsInUseLine: true, + Aliases: []string{"SwitchTraffic"}, + Args: cobra.NoArgs, + PreRunE: func(cmd *cobra.Command, args []string) error { + moveTablesSwitchTrafficOptions.Direction = workflow.DirectionForward + if !cmd.Flags().Lookup("tablet-types").Changed { + // We switch traffic for all tablet types if none are provided. + moveTablesSwitchTrafficOptions.TabletTypes = []topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_REPLICA, + topodatapb.TabletType_RDONLY, + } + } + return nil + }, + RunE: commandMoveTablesSwitchTraffic, + } +) + +var ( + // Required options for all commands. + moveTablesOptions = struct { + Workflow string + TargetKeyspace string + Format string + }{} + moveTablesCancelOptions = struct { + KeepData bool + KeepRoutingRules bool + }{} + moveTablesCompleteOptions = struct { + KeepData bool + KeepRoutingRules bool + RenameTables bool + DryRun bool + }{} + moveTablesCreateOptions = struct { + Workflow string + SourceKeyspace string + Cells []string + TabletTypes []topodatapb.TabletType + TabletTypesInPreferenceOrder bool + SourceShards []string + ExternalClusterName string + AllTables bool + IncludeTables []string + ExcludeTables []string + SourceTimeZone string + OnDDL string + DeferSecondaryKeys bool + AutoStart bool + StopAfterCopy bool + }{} + moveTablesSwitchTrafficOptions = struct { + Cells []string + TabletTypes []topodatapb.TabletType + MaxReplicationLagAllowed time.Duration + EnableReverseReplication bool + Timeout time.Duration + DryRun bool + Direction workflow.TrafficSwitchDirection + }{} +) + +func bridgeMoveTablesToWorkflow(cmd *cobra.Command, args []string) { + workflowUpdateOptions.Workflow = moveTablesOptions.Workflow + workflowOptions.Keyspace = moveTablesOptions.TargetKeyspace +} + +func commandMoveTablesCreate(cmd *cobra.Command, args []string) error { + format := strings.ToLower(strings.TrimSpace(moveTablesOptions.Format)) + switch format { + case "text", "json": + default: + return fmt.Errorf("invalid output format, got %s", moveTablesOptions.Format) + } + + cli.FinishedParsing(cmd) + + tsp := tabletmanagerdatapb.TabletSelectionPreference_ANY + if moveTablesCreateOptions.TabletTypesInPreferenceOrder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + + req := &vtctldatapb.MoveTablesCreateRequest{ + Workflow: moveTablesOptions.Workflow, + TargetKeyspace: moveTablesOptions.TargetKeyspace, + SourceKeyspace: moveTablesCreateOptions.SourceKeyspace, + Cells: moveTablesCreateOptions.Cells, + TabletTypes: moveTablesCreateOptions.TabletTypes, + TabletSelectionPreference: tsp, + AllTables: moveTablesCreateOptions.AllTables, + IncludeTables: moveTablesCreateOptions.IncludeTables, + ExcludeTables: moveTablesCreateOptions.ExcludeTables, + OnDdl: moveTablesCreateOptions.OnDDL, + AutoStart: moveTablesCreateOptions.AutoStart, + StopAfterCopy: moveTablesCreateOptions.StopAfterCopy, + } + + resp, err := client.MoveTablesCreate(commandCtx, req) + if err != nil { + return err + } + + var output []byte + if format == "json" { + output, err = cli.MarshalJSON(resp) + if err != nil { + return err + } + } else { + tout := bytes.Buffer{} + tout.WriteString(fmt.Sprintf("The following vreplication streams exist for workflow %s.%s:\n\n", + moveTablesOptions.TargetKeyspace, moveTablesOptions.Workflow)) + for _, shardstreams := range resp.ShardStreams { + for _, shardstream := range shardstreams.Streams { + tablet := fmt.Sprintf("%s-%d", shardstream.Tablet.Cell, shardstream.Tablet.Uid) + tout.WriteString(fmt.Sprintf("id=%d on %s/%s: Status: %s. %s.\n", + shardstream.Id, moveTablesOptions.TargetKeyspace, tablet, shardstream.Status, shardstream.Info)) + } + } + output = tout.Bytes() + } + fmt.Printf("%s\n", output) + + return nil +} + +func commandMoveTablesCancel(cmd *cobra.Command, args []string) error { + format := strings.ToLower(strings.TrimSpace(moveTablesOptions.Format)) + switch format { + case "text", "json": + default: + return fmt.Errorf("invalid output format, got %s", moveTablesOptions.Format) + } + + cli.FinishedParsing(cmd) + + req := &vtctldatapb.WorkflowDeleteRequest{ + Keyspace: moveTablesOptions.TargetKeyspace, + Workflow: moveTablesOptions.Workflow, + KeepData: moveTablesCancelOptions.KeepData, + KeepRoutingRules: moveTablesCancelOptions.KeepRoutingRules, + } + resp, err := client.WorkflowDelete(commandCtx, req) + if err != nil { + return err + } + + var output []byte + if format == "json" { + // Sort the inner TabletInfo slice for deterministic output. + sort.Slice(resp.Details, func(i, j int) bool { + return resp.Details[i].Tablet.String() < resp.Details[j].Tablet.String() + }) + output, err = cli.MarshalJSONCompact(resp) + if err != nil { + return err + } + } else { + output = []byte(resp.Summary + "\n") + } + fmt.Printf("%s\n", output) + + return nil +} + +func commandMoveTablesComplete(cmd *cobra.Command, args []string) error { + format := strings.ToLower(strings.TrimSpace(moveTablesOptions.Format)) + switch format { + case "text", "json": + default: + return fmt.Errorf("invalid output format, got %s", moveTablesOptions.Format) + } + + cli.FinishedParsing(cmd) + + req := &vtctldatapb.MoveTablesCompleteRequest{ + Workflow: moveTablesOptions.Workflow, + TargetKeyspace: moveTablesOptions.TargetKeyspace, + KeepData: moveTablesCompleteOptions.KeepData, + KeepRoutingRules: moveTablesCompleteOptions.KeepRoutingRules, + RenameTables: moveTablesCompleteOptions.RenameTables, + DryRun: moveTablesCompleteOptions.DryRun, + } + resp, err := client.MoveTablesComplete(commandCtx, req) + if err != nil { + return err + } + + var output []byte + if format == "json" { + output, err = cli.MarshalJSONCompact(resp) + if err != nil { + return err + } + } else { + tout := bytes.Buffer{} + tout.WriteString(resp.Summary + "\n") + if len(resp.DryRunResults) > 0 { + tout.WriteString("\n") + for _, r := range resp.DryRunResults { + tout.WriteString(r + "\n") + } + } + output = tout.Bytes() + } + fmt.Printf("%s\n", output) + + return nil +} + +func commandMoveTablesStatus(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + req := &vtctldatapb.WorkflowStatusRequest{ + Keyspace: moveTablesOptions.TargetKeyspace, + Workflow: moveTablesOptions.Workflow, + } + resp, err := client.WorkflowStatus(commandCtx, req) + if err != nil { + return err + } + + data, err := cli.MarshalJSON(resp) + if err != nil { + return err + } + + fmt.Printf("%s\n", data) + + return nil +} + +func commandMoveTablesShow(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + req := &vtctldatapb.GetWorkflowsRequest{ + Keyspace: moveTablesOptions.TargetKeyspace, + Workflow: moveTablesOptions.Workflow, + } + resp, err := client.GetWorkflows(commandCtx, req) + if err != nil { + return err + } + + data, err := cli.MarshalJSON(resp) + if err != nil { + return err + } + + fmt.Printf("%s\n", data) + + return nil +} + +func commandMoveTablesSwitchTraffic(cmd *cobra.Command, args []string) error { + format := strings.ToLower(strings.TrimSpace(moveTablesOptions.Format)) + switch format { + case "text", "json": + default: + return fmt.Errorf("invalid output format, got %s", moveTablesOptions.Format) + } + + cli.FinishedParsing(cmd) + + req := &vtctldatapb.WorkflowSwitchTrafficRequest{ + Keyspace: moveTablesOptions.TargetKeyspace, + Workflow: moveTablesOptions.Workflow, + TabletTypes: moveTablesSwitchTrafficOptions.TabletTypes, + MaxReplicationLagAllowed: protoutil.DurationToProto(moveTablesSwitchTrafficOptions.MaxReplicationLagAllowed), + Timeout: protoutil.DurationToProto(moveTablesSwitchTrafficOptions.Timeout), + DryRun: moveTablesSwitchTrafficOptions.DryRun, + EnableReverseReplication: moveTablesSwitchTrafficOptions.EnableReverseReplication, + Direction: int32(moveTablesSwitchTrafficOptions.Direction), + } + resp, err := client.WorkflowSwitchTraffic(commandCtx, req) + if err != nil { + return err + } + + var output []byte + if format == "json" { + output, err = cli.MarshalJSONCompact(resp) + if err != nil { + return err + } + } else { + tout := bytes.Buffer{} + tout.WriteString(resp.Summary + "\n\n") + if req.DryRun { + for _, line := range resp.DryRunResults { + tout.WriteString(line + "\n") + } + } else { + tout.WriteString(fmt.Sprintf("Start State: %s\n", resp.StartState)) + tout.WriteString(fmt.Sprintf("Current State: %s\n", resp.CurrentState)) + } + output = tout.Bytes() + } + fmt.Printf("%s\n", output) + + return nil +} + +func init() { + MoveTables.PersistentFlags().StringVar(&moveTablesOptions.TargetKeyspace, "target-keyspace", "", "Keyspace where the tables are being moved to and where the workflow exists (required)") + MoveTables.MarkPersistentFlagRequired("target-keyspace") + MoveTables.Flags().StringVarP(&moveTablesOptions.Workflow, "workflow", "w", "", "The workflow you want to perform the command on (required)") + MoveTables.MarkPersistentFlagRequired("workflow") + MoveTables.Flags().StringVar(&moveTablesOptions.Format, "format", "text", "The format of the output; supported formats are: text,json") + Root.AddCommand(MoveTables) + + MoveTablesCancel.Flags().BoolVar(&moveTablesCancelOptions.KeepData, "keep-data", false, "Keep the partially copied table data from the MoveTables workflow in the target keyspace") + MoveTablesCancel.Flags().BoolVar(&moveTablesCancelOptions.KeepRoutingRules, "keep-routing-rules", false, "Keep the routing rules created for the MoveTables workflow") + MoveTables.AddCommand(MoveTablesCancel) + + MoveTablesComplete.Flags().BoolVar(&moveTablesCompleteOptions.KeepData, "keep-data", false, "Keep the original source table data that was copied by the MoveTables workflow") + MoveTablesComplete.Flags().BoolVar(&moveTablesCompleteOptions.KeepRoutingRules, "keep-routing-rules", false, "Keep the routing rules in place that direct table traffic from the source keyspace to the target keyspace of the MoveTables workflow") + MoveTablesComplete.Flags().BoolVar(&moveTablesCompleteOptions.RenameTables, "rename-tables", false, "Keep the original source table data that was copied by the MoveTables workflow, but rename each table to '__old'") + MoveTablesComplete.Flags().BoolVar(&moveTablesCompleteOptions.DryRun, "dry-run", false, "Print the actions that would be taken and report any known errors that would have occurred") + MoveTables.AddCommand(MoveTablesComplete) + + MoveTablesCreate.PersistentFlags().StringVar(&moveTablesCreateOptions.SourceKeyspace, "source-keyspace", "", "Keyspace where the tables are being moved from (required)") + MoveTablesCreate.MarkPersistentFlagRequired("source-keyspace") + MoveTablesCreate.Flags().StringSliceVarP(&moveTablesCreateOptions.Cells, "cells", "c", nil, "Cells and/or CellAliases to copy table data from") + MoveTablesCreate.Flags().StringSliceVar(&moveTablesCreateOptions.SourceShards, "source-shards", nil, "Source shards to copy data from when performing a partial MoveTables (experimental)") + MoveTablesCreate.Flags().Var((*topoproto.TabletTypeListFlag)(&moveTablesCreateOptions.TabletTypes), "tablet-types", "Source tablet types to replicate table data from (e.g. PRIMARY,REPLICA,RDONLY)") + MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.TabletTypesInPreferenceOrder, "tablet-types-in-preference-order", true, "When performing source tablet selection, look for candidates in the type order as they are listed in the tablet-types flag") + MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.AllTables, "all-tables", false, "Copy all tables from the source") + MoveTablesCreate.Flags().StringSliceVar(&moveTablesCreateOptions.IncludeTables, "tables", nil, "Source tables to copy") + MoveTablesCreate.Flags().StringSliceVar(&moveTablesCreateOptions.ExcludeTables, "exclude-tables", nil, "Source tables to exclude from copying") + MoveTablesCreate.Flags().StringVar(&moveTablesCreateOptions.OnDDL, "on-ddl", onDDLDefault, "What to do when DDL is encountered in the VReplication stream. Possible values are IGNORE, STOP, EXEC, and EXEC_IGNORE") + MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.AutoStart, "auto-start", true, "Start the MoveTables workflow after creating it") + MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.StopAfterCopy, "stop-after-copy", false, "Stop the MoveTables workflow after it's finished copying the existing rows and before it starts replicating changes") + MoveTables.AddCommand(MoveTablesCreate) + + MoveTables.AddCommand(MoveTablesShow) + + MoveTables.AddCommand(MoveTablesStart) + + MoveTables.AddCommand(MoveTablesStatus) + + MoveTables.AddCommand(MoveTablesStop) + + MoveTablesSwitchTraffic.Flags().StringSliceVarP(&moveTablesSwitchTrafficOptions.Cells, "cells", "c", nil, "Cells and/or CellAliases to switch traffic in") + MoveTablesSwitchTraffic.Flags().Var((*topoproto.TabletTypeListFlag)(&moveTablesSwitchTrafficOptions.TabletTypes), "tablet-types", "Tablet types to switch traffic for") + MoveTablesSwitchTraffic.Flags().DurationVar(&moveTablesSwitchTrafficOptions.Timeout, "timeout", timeoutDefault, "Specifies the maximum time to wait, in seconds, for VReplication to catch up on primary tablets. The traffic switch will be cancelled on timeout.") + MoveTablesSwitchTraffic.Flags().DurationVar(&moveTablesSwitchTrafficOptions.MaxReplicationLagAllowed, "max-replication-lag-allowed", maxReplicationLagDefault, "Allow traffic to be switched only if VReplication lag is below this") + MoveTablesSwitchTraffic.Flags().BoolVar(&moveTablesSwitchTrafficOptions.DryRun, "dry-run", false, "Print the actions that would be taken and report any known errors that would have occurred") + MoveTables.AddCommand(MoveTablesSwitchTraffic) + + MoveTablesReverseTraffic.Flags().StringSliceVarP(&moveTablesSwitchTrafficOptions.Cells, "cells", "c", nil, "Cells and/or CellAliases to switch traffic in") + MoveTablesReverseTraffic.Flags().BoolVar(&moveTablesSwitchTrafficOptions.DryRun, "dry-run", false, "Print the actions that would be taken and report any known errors that would have occurred") + MoveTablesReverseTraffic.Flags().DurationVar(&moveTablesSwitchTrafficOptions.MaxReplicationLagAllowed, "max-replication-lag-allowed", maxReplicationLagDefault, "Allow traffic to be switched only if VReplication lag is below this") + MoveTablesReverseTraffic.Flags().BoolVar(&moveTablesSwitchTrafficOptions.EnableReverseReplication, "enable-reverse-replication", true, "Setup replication going back to the original source keyspace to support rolling back the traffic cutover") + MoveTablesReverseTraffic.Flags().Var((*topoproto.TabletTypeListFlag)(&moveTablesSwitchTrafficOptions.TabletTypes), "tablet-types", "Tablet types to switch traffic for") + MoveTablesReverseTraffic.Flags().DurationVar(&moveTablesSwitchTrafficOptions.Timeout, "timeout", timeoutDefault, "Specifies the maximum time to wait, in seconds, for VReplication to catch up on primary tablets. The traffic switch will be cancelled on timeout.") + MoveTables.AddCommand(MoveTablesReverseTraffic) +} diff --git a/go/cmd/vtctldclient/command/workflows.go b/go/cmd/vtctldclient/command/workflows.go index f783ce9c307..a67bd1e43dc 100644 --- a/go/cmd/vtctldclient/command/workflows.go +++ b/go/cmd/vtctldclient/command/workflows.go @@ -29,6 +29,7 @@ import ( binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) @@ -44,19 +45,76 @@ var ( // Workflow is a parent command for Workflow* sub commands. Workflow = &cobra.Command{ - Use: "workflow", - Short: "Administer VReplication workflows (Reshard, MoveTables, etc) in the given keyspace", + Use: "Workflow --keyspace [command] [command-flags]", + Short: "Administer VReplication workflows (Reshard, MoveTables, etc) in the given keyspace.", + Long: `Workflow commands: List, Show, Start, Stop, Update, and Delete. +See the --help output for each command for more details.`, DisableFlagsInUseLine: true, - Aliases: []string{"Workflow"}, + Aliases: []string{"workflow"}, Args: cobra.ExactArgs(1), RunE: commandGetWorkflows, } + // WorkflowDelete makes a WorkflowDelete gRPC call to a vtctld. + WorkflowDelete = &cobra.Command{ + Use: "delete", + Short: "Delete a VReplication workflow.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer delete --workflow commerce2customer`, + DisableFlagsInUseLine: true, + Aliases: []string{"Delete"}, + Args: cobra.NoArgs, + RunE: commandWorkflowDelete, + } + + // WorkflowList makes a GetWorkflows gRPC call to a vtctld. + WorkflowList = &cobra.Command{ + Use: "list", + Short: "List the VReplication workflows in the given keyspace.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer list`, + DisableFlagsInUseLine: true, + Aliases: []string{"List"}, + Args: cobra.NoArgs, + RunE: commandWorkflowShow, + } + + // WorkflowShow makes a GetWorkflows gRPC call to a vtctld. + WorkflowShow = &cobra.Command{ + Use: "show", + Short: "Show the details for a VReplication workflow.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer show --workflow commerce2customer`, + DisableFlagsInUseLine: true, + Aliases: []string{"Show"}, + Args: cobra.NoArgs, + RunE: commandWorkflowShow, + } + + // WorkflowStart makes a WorfklowUpdate gRPC call to a vtctld. + WorkflowStart = &cobra.Command{ + Use: "start", + Short: "Start a VReplication workflow.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer start --workflow commerce2customer`, + DisableFlagsInUseLine: true, + Aliases: []string{"Start"}, + Args: cobra.NoArgs, + RunE: commandWorkflowUpdateState, + } + + // WorkflowStop makes a WorfklowUpdate gRPC call to a vtctld. + WorkflowStop = &cobra.Command{ + Use: "stop", + Short: "Stop a VReplication workflow.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer stop --workflow commerce2customer`, + DisableFlagsInUseLine: true, + Aliases: []string{"Stop"}, + Args: cobra.NoArgs, + RunE: commandWorkflowUpdateState, + } + // WorkflowUpdate makes a WorkflowUpdate gRPC call to a vtctld. WorkflowUpdate = &cobra.Command{ Use: "update", - Short: "Update the configuration parameters for a VReplication workflow", - Example: `vtctldclient --server=localhost:15999 workflow --keyspace=customer update --workflow=commerce2customer --cells "zone1" --cells "zone2" -c "zone3,zone4" -c "zone5"`, + Short: "Update the configuration parameters for a VReplication workflow.", + Example: `vtctldclient --server localhost:15999 workflow --keyspace customer update --workflow commerce2customer --cells zone1 --cells zone2 -c "zone3,zone4" -c zone5`, DisableFlagsInUseLine: true, Aliases: []string{"Update"}, Args: cobra.NoArgs, @@ -70,16 +128,10 @@ var ( } else { workflowUpdateOptions.Cells = textutil.SimulatedNullStringSlice } - if cmd.Flags().Lookup("tablet-types").Changed { // Validate the provided value(s) + if cmd.Flags().Lookup("tablet-types").Changed { changes = true - for i, tabletType := range workflowUpdateOptions.TabletTypes { - workflowUpdateOptions.TabletTypes[i] = strings.ToUpper(strings.TrimSpace(tabletType)) - if _, err := topoproto.ParseTabletType(workflowUpdateOptions.TabletTypes[i]); err != nil { - return err - } - } } else { - workflowUpdateOptions.TabletTypes = textutil.SimulatedNullStringSlice + workflowUpdateOptions.TabletTypes = []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)} } if cmd.Flags().Lookup("on-ddl").Changed { // Validate the provided value changes = true @@ -128,14 +180,80 @@ var ( workflowOptions = struct { Keyspace string }{} + workflowDeleteOptions = struct { + Workflow string + KeepData bool + KeepRoutingRules bool + }{} workflowUpdateOptions = struct { - Workflow string - Cells []string - TabletTypes []string - OnDDL string + Workflow string + Cells []string + TabletTypes []topodatapb.TabletType + TabletTypesInPreferenceOrder bool + OnDDL string }{} ) +func commandWorkflowDelete(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + req := &vtctldatapb.WorkflowDeleteRequest{ + Keyspace: workflowOptions.Keyspace, + Workflow: workflowDeleteOptions.Workflow, + KeepData: workflowDeleteOptions.KeepData, + KeepRoutingRules: workflowDeleteOptions.KeepRoutingRules, + } + resp, err := client.WorkflowDelete(commandCtx, req) + if err != nil { + return err + } + + // Sort the inner TabletInfo slice for deterministic output. + sort.Slice(resp.Details, func(i, j int) bool { + return resp.Details[i].Tablet.String() < resp.Details[j].Tablet.String() + }) + + data, err := cli.MarshalJSON(resp) + if err != nil { + return err + } + + fmt.Printf("%s\n", data) + + return nil +} + +func commandWorkflowShow(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + req := &vtctldatapb.GetWorkflowsRequest{ + Keyspace: workflowOptions.Keyspace, + Workflow: workflowDeleteOptions.Workflow, + } + resp, err := client.GetWorkflows(commandCtx, req) + if err != nil { + return err + } + + var data []byte + if strings.ToLower(cmd.Name()) == "list" { + // We only want the names + Names := make([]string, len(resp.Workflows)) + for i, wf := range resp.Workflows { + Names[i] = wf.Name + } + data, err = cli.MarshalJSON(Names) + } else { + data, err = cli.MarshalJSON(resp) + } + if err != nil { + return err + } + fmt.Printf("%s\n", data) + + return nil +} + func commandWorkflowUpdate(cmd *cobra.Command, args []string) error { cli.FinishedParsing(cmd) @@ -147,13 +265,69 @@ func commandWorkflowUpdate(cmd *cobra.Command, args []string) error { onddl = val } + // Simulated NULL when no value is provided. + tsp := tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN + if cmd.Flags().Lookup("tablet-types-in-order").Changed { + if workflowUpdateOptions.TabletTypesInPreferenceOrder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } else { + tsp = tabletmanagerdatapb.TabletSelectionPreference_ANY + } + } + + req := &vtctldatapb.WorkflowUpdateRequest{ + Keyspace: workflowOptions.Keyspace, + TabletRequest: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ + Workflow: workflowUpdateOptions.Workflow, + Cells: workflowUpdateOptions.Cells, + TabletTypes: workflowUpdateOptions.TabletTypes, + TabletSelectionPreference: tsp, + OnDdl: binlogdatapb.OnDDLAction(onddl), + }, + } + + resp, err := client.WorkflowUpdate(commandCtx, req) + if err != nil { + return err + } + + // Sort the inner TabletInfo slice for deterministic output. + sort.Slice(resp.Details, func(i, j int) bool { + return resp.Details[i].Tablet.String() < resp.Details[j].Tablet.String() + }) + + data, err := cli.MarshalJSON(resp) + if err != nil { + return err + } + + fmt.Printf("%s\n", data) + + return nil +} + +func commandWorkflowUpdateState(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + var state binlogdatapb.VReplicationWorkflowState + switch strings.ToLower(cmd.Name()) { + case "start": + state = binlogdatapb.VReplicationWorkflowState_Running + case "stop": + state = binlogdatapb.VReplicationWorkflowState_Stopped + default: + return fmt.Errorf("invalid workstate: %s", args[0]) + } + + // The only thing we're updating is the state. req := &vtctldatapb.WorkflowUpdateRequest{ Keyspace: workflowOptions.Keyspace, - TabletRequest: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + TabletRequest: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflowUpdateOptions.Workflow, - Cells: workflowUpdateOptions.Cells, - TabletTypes: workflowUpdateOptions.TabletTypes, - OnDdl: binlogdatapb.OnDDLAction(onddl), + Cells: textutil.SimulatedNullStringSlice, + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, + OnDdl: binlogdatapb.OnDDLAction(textutil.SimulatedNullInt), + State: state, }, } @@ -164,7 +338,7 @@ func commandWorkflowUpdate(cmd *cobra.Command, args []string) error { // Sort the inner TabletInfo slice for deterministic output. sort.Slice(resp.Details, func(i, j int) bool { - return resp.Details[i].Tablet < resp.Details[j].Tablet + return resp.Details[i].Tablet.String() < resp.Details[j].Tablet.String() }) data, err := cli.MarshalJSON(resp) @@ -184,10 +358,32 @@ func init() { Workflow.PersistentFlags().StringVarP(&workflowOptions.Keyspace, "keyspace", "k", "", "Keyspace context for the workflow (required)") Workflow.MarkPersistentFlagRequired("keyspace") Root.AddCommand(Workflow) + + WorkflowDelete.Flags().StringVarP(&workflowDeleteOptions.Workflow, "workflow", "w", "", "The workflow you want to delete (required)") + WorkflowDelete.MarkFlagRequired("workflow") + WorkflowDelete.Flags().BoolVar(&workflowDeleteOptions.KeepData, "keep-data", false, "Keep the partially copied table data from the workflow in the target keyspace") + WorkflowDelete.Flags().BoolVar(&workflowDeleteOptions.KeepRoutingRules, "keep-routing-rules", false, "Keep the routing rules created for the workflow") + Workflow.AddCommand(WorkflowDelete) + + Workflow.AddCommand(WorkflowList) + + WorkflowShow.Flags().StringVarP(&workflowDeleteOptions.Workflow, "workflow", "w", "", "The workflow you want the details for (required)") + WorkflowShow.MarkFlagRequired("workflow") + Workflow.AddCommand(WorkflowShow) + + WorkflowStart.Flags().StringVarP(&workflowUpdateOptions.Workflow, "workflow", "w", "", "The workflow you want to start (required)") + WorkflowStart.MarkFlagRequired("workflow") + Workflow.AddCommand(WorkflowStart) + + WorkflowStop.Flags().StringVarP(&workflowUpdateOptions.Workflow, "workflow", "w", "", "The workflow you want to stop (required)") + WorkflowStop.MarkFlagRequired("workflow") + Workflow.AddCommand(WorkflowStop) + WorkflowUpdate.Flags().StringVarP(&workflowUpdateOptions.Workflow, "workflow", "w", "", "The workflow you want to update (required)") WorkflowUpdate.MarkFlagRequired("workflow") WorkflowUpdate.Flags().StringSliceVarP(&workflowUpdateOptions.Cells, "cells", "c", nil, "New Cell(s) or CellAlias(es) (comma-separated) to replicate from") - WorkflowUpdate.Flags().StringSliceVarP(&workflowUpdateOptions.TabletTypes, "tablet-types", "t", nil, "New source tablet types to replicate from (e.g. PRIMARY,REPLICA,RDONLY)") + WorkflowUpdate.Flags().VarP((*topoproto.TabletTypeListFlag)(&workflowUpdateOptions.TabletTypes), "tablet-types", "t", "New source tablet types to replicate from (e.g. PRIMARY,REPLICA,RDONLY)") + WorkflowUpdate.Flags().BoolVar(&workflowUpdateOptions.TabletTypesInPreferenceOrder, "tablet-types-in-order", true, "When performing source tablet selection, look for candidates in the type order as they are listed in the tablet-types flag") WorkflowUpdate.Flags().StringVar(&workflowUpdateOptions.OnDDL, "on-ddl", "", "New instruction on what to do when DDL is encountered in the VReplication stream. Possible values are IGNORE, STOP, EXEC, and EXEC_IGNORE") Workflow.AddCommand(WorkflowUpdate) } diff --git a/go/flags/endtoend/vtctldclient.txt b/go/flags/endtoend/vtctldclient.txt index 4c059503434..e8187211f31 100644 --- a/go/flags/endtoend/vtctldclient.txt +++ b/go/flags/endtoend/vtctldclient.txt @@ -51,6 +51,7 @@ Available Commands: GetVSchema Prints a JSON representation of a keyspace's topo record. GetWorkflows Gets all vreplication workflows (Reshard, MoveTables, etc) in the given keyspace. LegacyVtctlCommand Invoke a legacy vtctlclient command. Flag parsing is best effort. + MoveTables Perform commands related to moving tables from a source keyspace to a target keyspace. PingTablet Checks that the specified tablet is awake and responding to RPCs. This command can be blocked by other in-flight operations. PlannedReparentShard Reparents the shard to a new primary, or away from an old primary. Both the old and new primaries must be up and running. RebuildKeyspaceGraph Rebuilds the serving data for the keyspace(s). This command may trigger an update to all connected clients. @@ -87,9 +88,9 @@ Available Commands: ValidateShard Validates that all nodes reachable from the specified shard are consistent. ValidateVersionKeyspace Validates that the version on the primary tablet of shard 0 matches all of the other tablets in the keyspace. ValidateVersionShard Validates that the version on the primary matches all of the replicas. + Workflow Administer VReplication workflows (Reshard, MoveTables, etc) in the given keyspace. completion Generate the autocompletion script for the specified shell help Help about any command - workflow Administer VReplication workflows (Reshard, MoveTables, etc) in the given keyspace Flags: --action_timeout duration timeout for the total command (default 1h0m0s) diff --git a/go/test/endtoend/onlineddl/vrepl/onlineddl_vrepl_test.go b/go/test/endtoend/onlineddl/vrepl/onlineddl_vrepl_test.go index 87aa57c59df..79cb24d22c3 100644 --- a/go/test/endtoend/onlineddl/vrepl/onlineddl_vrepl_test.go +++ b/go/test/endtoend/onlineddl/vrepl/onlineddl_vrepl_test.go @@ -26,6 +26,9 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/test/endtoend/cluster" @@ -35,8 +38,7 @@ import ( throttlebase "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/base" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) var ( @@ -521,12 +523,12 @@ func TestSchemaChange(t *testing.T) { onlineddl.CheckMigrationStatus(t, &vtParams, shards, uuid, schema.OnlineDDLStatusRunning) }) t.Run("wait for vreplication to run on shard -80", func(t *testing.T) { - vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, currentPrimaryTablet, uuid, normalMigrationWait, "Copying", "Running") - require.Contains(t, []string{"Copying", "Running"}, vreplStatus) + vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, currentPrimaryTablet, uuid, normalMigrationWait, binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()) + require.Contains(t, []string{binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()}, vreplStatus) }) t.Run("wait for vreplication to run on shard 80-", func(t *testing.T) { - vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, shards[1].Vttablets[0], uuid, normalMigrationWait, "Copying", "Running") - require.Contains(t, []string{"Copying", "Running"}, vreplStatus) + vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, shards[1].Vttablets[0], uuid, normalMigrationWait, binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()) + require.Contains(t, []string{binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()}, vreplStatus) }) t.Run("check status again", func(t *testing.T) { // again see that we're still 'running' @@ -616,12 +618,12 @@ func TestSchemaChange(t *testing.T) { onlineddl.CheckMigrationStatus(t, &vtParams, shards, uuid, schema.OnlineDDLStatusRunning) }) t.Run("wait for vreplication to run on shard -80", func(t *testing.T) { - vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, currentPrimaryTablet, uuid, normalMigrationWait, "Copying", "Running") - require.Contains(t, []string{"Copying", "Running"}, vreplStatus) + vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, currentPrimaryTablet, uuid, normalMigrationWait, binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()) + require.Contains(t, []string{binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()}, vreplStatus) }) t.Run("wait for vreplication to run on shard 80-", func(t *testing.T) { - vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, shards[1].Vttablets[0], uuid, normalMigrationWait, "Copying", "Running") - require.Contains(t, []string{"Copying", "Running"}, vreplStatus) + vreplStatus := onlineddl.WaitForVReplicationStatus(t, &vtParams, shards[1].Vttablets[0], uuid, normalMigrationWait, binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()) + require.Contains(t, []string{binlogdatapb.VReplicationWorkflowState_Copying.String(), binlogdatapb.VReplicationWorkflowState_Running.String()}, vreplStatus) }) t.Run("check status again", func(t *testing.T) { // again see that we're still 'running' diff --git a/go/test/endtoend/vreplication/helper_test.go b/go/test/endtoend/vreplication/helper_test.go index 49f35d49b06..79052307c7a 100644 --- a/go/test/endtoend/vreplication/helper_test.go +++ b/go/test/endtoend/vreplication/helper_test.go @@ -46,16 +46,14 @@ import ( "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) const ( defaultTick = 1 * time.Second defaultTimeout = 30 * time.Second workflowStateTimeout = 90 * time.Second - workflowStateCopying = "Copying" // nolint - workflowStateRunning = "Running" // nolint - workflowStateStopped = "Stopped" // nolint - workflowStateError = "Error" // nolint ) func execMultipleQueries(t *testing.T, conn *mysql.Conn, database string, lines string) { @@ -563,7 +561,7 @@ func confirmWorkflowHasCopiedNoData(t *testing.T, targetKS, workflow string) { state := attributeValue.Get("State").String() pos := attributeValue.Get("Pos").String() // If we've actually copied anything then we'll have a position in the stream - if (state == workflowStateRunning || state == workflowStateCopying) && pos != "" { + if (state == binlogdatapb.VReplicationWorkflowState_Running.String() || state == binlogdatapb.VReplicationWorkflowState_Copying.String()) && pos != "" { require.FailNowf(t, "Unexpected data copied in workflow", "The MoveTables workflow %q copied data in less than %s when it should have been waiting. Show output: %s", ksWorkflow, defaultTimeout, output) diff --git a/go/test/endtoend/vreplication/migrate_test.go b/go/test/endtoend/vreplication/migrate_test.go index 0c83658cee8..6155e6ec2e3 100644 --- a/go/test/endtoend/vreplication/migrate_test.go +++ b/go/test/endtoend/vreplication/migrate_test.go @@ -24,6 +24,8 @@ import ( "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/test/endtoend/cluster" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) func insertInitialDataIntoExternalCluster(t *testing.T, conn *mysql.Conn) { @@ -109,7 +111,7 @@ func TestMigrate(t *testing.T) { "--source=ext1.rating", "create", ksWorkflow); err != nil { t.Fatalf("Migrate command failed with %+v : %s\n", err, output) } - waitForWorkflowState(t, vc, ksWorkflow, workflowStateRunning) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) expectNumberOfStreams(t, vtgateConn, "migrate", "e1", "product:0", 1) waitForRowCount(t, vtgateConn, "product:0", "rating", 2) waitForRowCount(t, vtgateConn, "product:0", "review", 3) diff --git a/go/test/endtoend/vreplication/partial_movetables_seq_test.go b/go/test/endtoend/vreplication/partial_movetables_seq_test.go index 3ff921bbafe..b35b878b252 100644 --- a/go/test/endtoend/vreplication/partial_movetables_seq_test.go +++ b/go/test/endtoend/vreplication/partial_movetables_seq_test.go @@ -29,6 +29,8 @@ import ( "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/wrangler" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) /* @@ -253,7 +255,7 @@ func (wf *workflow) create() { panic(fmt.Sprintf("unknown workflow type: %s", wf.typ)) } require.NoError(t, err) - waitForWorkflowState(t, wf.tc.vc, fmt.Sprintf("%s.%s", wf.toKeyspace, wf.name), workflowStateRunning) + waitForWorkflowState(t, wf.tc.vc, fmt.Sprintf("%s.%s", wf.toKeyspace, wf.name), binlogdatapb.VReplicationWorkflowState_Running.String()) ks2 := wf.tc.vc.Cells[cell].Keyspaces[wf.toKeyspace] var i int64 for _, shardName := range wf.tc.keyspaces[wf.toKeyspace].shards { diff --git a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go index e12dbfa1cb1..a55ad3047e1 100644 --- a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go +++ b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go @@ -30,6 +30,8 @@ import ( "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/wrangler" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) const ( @@ -63,7 +65,7 @@ func createReshardWorkflow(t *testing.T, sourceShards, targetShards string) erro err := tstWorkflowExec(t, defaultCellName, workflowName, targetKs, targetKs, "", workflowActionCreate, "", sourceShards, targetShards) require.NoError(t, err) - waitForWorkflowState(t, vc, ksWorkflow, workflowStateRunning) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) confirmTablesHaveSecondaryKeys(t, []*cluster.VttabletProcess{targetTab1}, targetKs, "") catchup(t, targetTab1, workflowName, "Reshard") catchup(t, targetTab2, workflowName, "Reshard") @@ -78,7 +80,7 @@ func createMoveTablesWorkflow(t *testing.T, tables string) { err := tstWorkflowExec(t, defaultCellName, workflowName, sourceKs, targetKs, tables, workflowActionCreate, "", "", "") require.NoError(t, err) - waitForWorkflowState(t, vc, ksWorkflow, workflowStateRunning) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) confirmTablesHaveSecondaryKeys(t, []*cluster.VttabletProcess{targetTab1}, targetKs, tables) catchup(t, targetTab1, workflowName, "MoveTables") catchup(t, targetTab2, workflowName, "MoveTables") @@ -242,7 +244,7 @@ func revert(t *testing.T, workflowType string) { validateReadsRouteToSource(t, "replica") // cancel the workflow to cleanup - _, err := vc.VtctlClient.ExecuteCommandWithOutput(workflowType, "--", "Cancel", ksWorkflow) + _, err := vc.VtctldClient.ExecuteCommandWithOutput(workflowType, "--target-keyspace", targetKs, "--workflow", workflowName, "cancel") require.NoError(t, err, fmt.Sprintf("%s Cancel error: %v", workflowType, err)) } @@ -308,7 +310,7 @@ func testVSchemaForSequenceAfterMoveTables(t *testing.T) { "customer2", workflowActionCreate, "", "", "") require.NoError(t, err) - waitForWorkflowState(t, vc, "customer.wf2", workflowStateRunning) + waitForWorkflowState(t, vc, "customer.wf2", binlogdatapb.VReplicationWorkflowState_Running.String()) waitForLowLag(t, "customer", "wf2") err = tstWorkflowExec(t, defaultCellName, "wf2", sourceKs, targetKs, @@ -342,7 +344,7 @@ func testVSchemaForSequenceAfterMoveTables(t *testing.T) { err = tstWorkflowExec(t, defaultCellName, "wf3", targetKs, sourceKs, "customer2", workflowActionCreate, "", "", "") require.NoError(t, err) - waitForWorkflowState(t, vc, "product.wf3", workflowStateRunning) + waitForWorkflowState(t, vc, "product.wf3", binlogdatapb.VReplicationWorkflowState_Running.String()) waitForLowLag(t, "product", "wf3") err = tstWorkflowExec(t, defaultCellName, "wf3", targetKs, sourceKs, diff --git a/go/test/endtoend/vreplication/vreplication_test.go b/go/test/endtoend/vreplication/vreplication_test.go index 38a1747103a..35f7062d27d 100644 --- a/go/test/endtoend/vreplication/vreplication_test.go +++ b/go/test/endtoend/vreplication/vreplication_test.go @@ -170,7 +170,7 @@ func TestVReplicationDDLHandling(t *testing.T) { _, err = vtgateConn.ExecuteFetch(addColDDL, 1, false) require.NoError(t, err, "error executing %q: %v", addColDDL, err) // Confirm workflow is still running fine - waitForWorkflowState(t, vc, ksWorkflow, "Running") + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) // Confirm new col does not exist on target waitForQueryResult(t, vtgateConn, targetKs, checkColQueryTarget, "[[INT64(0)]]") // Confirm new col does exist on source @@ -200,7 +200,7 @@ func TestVReplicationDDLHandling(t *testing.T) { _, err = vtgateConn.ExecuteFetch(addColDDL, 1, false) require.NoError(t, err, "error executing %q: %v", addColDDL, err) // Confirm that the worfklow stopped because of the DDL - waitForWorkflowState(t, vc, ksWorkflow, "Stopped", fmt.Sprintf("Message==Stopped at DDL %s", addColDDL)) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Stopped.String(), fmt.Sprintf("Message==Stopped at DDL %s", addColDDL)) // Confirm that the target does not have new col waitForQueryResult(t, vtgateConn, targetKs, checkColQueryTarget, "[[INT64(0)]]") moveTablesAction(t, "Cancel", defaultCellName, workflow, sourceKs, targetKs, table) @@ -215,7 +215,7 @@ func TestVReplicationDDLHandling(t *testing.T) { _, err = vtgateConn.ExecuteFetch(dropColDDL, 1, false) require.NoError(t, err, "error executing %q: %v", dropColDDL, err) // Confirm workflow is still running fine - waitForWorkflowState(t, vc, ksWorkflow, "Running") + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) // Confirm new col was dropped on target waitForQueryResult(t, vtgateConn, targetKs, checkColQueryTarget, "[[INT64(0)]]") moveTablesAction(t, "Cancel", defaultCellName, workflow, sourceKs, targetKs, table) @@ -269,9 +269,9 @@ func TestVreplicationCopyThrottling(t *testing.T) { // We need to force primary tablet types as the history list has been increased on the source primary // We use a small timeout and ignore errors as we don't expect the MoveTables to start here // because of the InnoDB History List length. - moveTablesActionWithTabletTypes(t, "Create", defaultCell.Name, workflow, sourceKs, targetKs, table, "primary", 5*time.Second, true) + moveTablesActionWithTabletTypes(t, "Create", defaultCell.Name, workflow, sourceKs, targetKs, table, "primary", true) // Wait for the copy phase to start - waitForWorkflowState(t, vc, fmt.Sprintf("%s.%s", targetKs, workflow), workflowStateCopying) + waitForWorkflowState(t, vc, fmt.Sprintf("%s.%s", targetKs, workflow), binlogdatapb.VReplicationWorkflowState_Copying.String()) // The initial copy phase should be blocking on the history list confirmWorkflowHasCopiedNoData(t, targetKs, workflow) releaseInnoDBRowHistory(t, trxConn) @@ -1354,21 +1354,19 @@ func catchup(t *testing.T, vttablet *cluster.VttabletProcess, workflow, info str func moveTablesAction(t *testing.T, action, cell, workflow, sourceKs, targetKs, tables string, extraFlags ...string) { var err error - if len(extraFlags) > 0 { - err = vc.VtctlClient.ExecuteCommand("MoveTables", "--", "--source="+sourceKs, "--tables="+tables, - "--cells="+cell, "--tablet_types=primary,replica,rdonly", strings.Join(extraFlags, " "), - action, fmt.Sprintf("%s.%s", targetKs, workflow)) - } else { - err = vc.VtctlClient.ExecuteCommand("MoveTables", "--", "--source="+sourceKs, "--tables="+tables, "--cells="+cell, - "--tablet_types=primary,replica,rdonly", action, fmt.Sprintf("%s.%s", targetKs, workflow)) + args := []string{"MoveTables", "--workflow=" + workflow, "--target-keyspace=" + targetKs, action} + if strings.EqualFold(action, strings.ToLower(workflowActionCreate)) { + extraFlags = append(extraFlags, "--source-keyspace="+sourceKs, "--tables="+tables, "--cells="+cell, "--tablet-types=primary,replica,rdonly") } + args = append(args, extraFlags...) + err = vc.VtctldClient.ExecuteCommand(args...) if err != nil { t.Fatalf("MoveTables %s command failed with %+v\n", action, err) } } -func moveTablesActionWithTabletTypes(t *testing.T, action, cell, workflow, sourceKs, targetKs, tables string, tabletTypes string, timeout time.Duration, ignoreErrors bool) { - if err := vc.VtctlClient.ExecuteCommand("MoveTables", "--", "--source="+sourceKs, "--tables="+tables, "--cells="+cell, - "--tablet_types="+tabletTypes, "--timeout="+timeout.String(), action, fmt.Sprintf("%s.%s", targetKs, workflow)); err != nil { +func moveTablesActionWithTabletTypes(t *testing.T, action, cell, workflow, sourceKs, targetKs, tables string, tabletTypes string, ignoreErrors bool) { + if err := vc.VtctldClient.ExecuteCommand("MoveTables", "--workflow="+workflow, "--target-keyspace="+targetKs, action, + "--source-keyspace="+sourceKs, "--tables="+tables, "--cells="+cell, "--tablet-types="+tabletTypes); err != nil { if !ignoreErrors { t.Fatalf("MoveTables %s command failed with %+v\n", action, err) } diff --git a/go/textutil/strings.go b/go/textutil/strings.go index bd5dd6ff7f4..212ea742893 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -23,6 +23,8 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/proto/binlogdata" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) var ( @@ -99,6 +101,8 @@ func ValueIsSimulatedNull(val any) bool { return int32(cval) == int32(SimulatedNullInt) case int64: return int64(cval) == int64(SimulatedNullInt) + case []topodatapb.TabletType: + return len(cval) == 1 && cval[0] == topodatapb.TabletType(SimulatedNullInt) default: return false } diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index c1db2b67c75..45d7338d705 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -42,10 +42,11 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/log" - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/throttler" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) var ( @@ -58,17 +59,6 @@ var ( BlplQuery = "Query" // BlplTransaction is the key for the stats map. BlplTransaction = "Transaction" - - // VReplicationInit is for the Init state. - VReplicationInit = "Init" - // VReplicationCopying is for the Copying state. - VReplicationCopying = "Copying" - // BlpRunning is for the Running state. - BlpRunning = "Running" - // BlpStopped is for the Stopped state. - BlpStopped = "Stopped" - // BlpError is for the Error state. - BlpError = "Error" ) // Stats is the internal stats of a player. It is a different @@ -231,12 +221,12 @@ func NewBinlogPlayerTables(dbClient DBClient, tablet *topodatapb.Tablet, tables // If an error is encountered, it updates the vreplication state to "Error". // If a stop position was specified, and reached, the state is updated to "Stopped". func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { - if err := blp.setVReplicationState(BlpRunning, ""); err != nil { + if err := blp.setVReplicationState(binlogdatapb.VReplicationWorkflowState_Running, ""); err != nil { log.Errorf("Error writing Running state: %v", err) } if err := blp.applyEvents(ctx); err != nil { - if err := blp.setVReplicationState(BlpError, err.Error()); err != nil { + if err := blp.setVReplicationState(binlogdatapb.VReplicationWorkflowState_Error, err.Error()); err != nil { log.Errorf("Error writing stop state: %v", err) } return err @@ -291,14 +281,14 @@ func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { case blp.position.Equal(blp.stopPosition): msg := fmt.Sprintf("not starting BinlogPlayer, we're already at the desired position %v", blp.stopPosition) log.Info(msg) - if err := blp.setVReplicationState(BlpStopped, msg); err != nil { + if err := blp.setVReplicationState(binlogdatapb.VReplicationWorkflowState_Stopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } return nil case blp.position.AtLeast(blp.stopPosition): msg := fmt.Sprintf("starting point %v greater than stopping point %v", blp.position, blp.stopPosition) log.Error(msg) - if err := blp.setVReplicationState(BlpStopped, msg); err != nil { + if err := blp.setVReplicationState(binlogdatapb.VReplicationWorkflowState_Stopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } // Don't return an error. Otherwise, it will keep retrying. @@ -398,7 +388,7 @@ func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { if blp.position.AtLeast(blp.stopPosition) { msg := "Reached stopping position, done playing logs" log.Info(msg) - if err := blp.setVReplicationState(BlpStopped, msg); err != nil { + if err := blp.setVReplicationState(binlogdatapb.VReplicationWorkflowState_Stopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } return nil @@ -513,15 +503,15 @@ func (blp *BinlogPlayer) writeRecoveryPosition(tx *binlogdatapb.BinlogTransactio return nil } -func (blp *BinlogPlayer) setVReplicationState(state, message string) error { +func (blp *BinlogPlayer) setVReplicationState(state binlogdatapb.VReplicationWorkflowState, message string) error { if message != "" { blp.blplStats.History.Add(&StatsHistoryRecord{ Time: time.Now(), Message: message, }) } - blp.blplStats.State.Store(state) - query := fmt.Sprintf("update _vt.vreplication set state='%v', message=%v where id=%v", state, encodeString(MessageTruncate(message)), blp.uid) + blp.blplStats.State.Store(state.String()) + query := fmt.Sprintf("update _vt.vreplication set state='%v', message=%v where id=%v", state.String(), encodeString(MessageTruncate(message)), blp.uid) if _, err := blp.dbClient.ExecuteFetch(query, 1); err != nil { return fmt.Errorf("could not set state: %v: %v", query, err) } @@ -534,7 +524,7 @@ type VRSettings struct { StopPos mysql.Position MaxTPS int64 MaxReplicationLag int64 - State string + State binlogdatapb.VReplicationWorkflowState WorkflowType binlogdatapb.VReplicationWorkflowType WorkflowSubType binlogdatapb.VReplicationWorkflowSubType WorkflowName string @@ -557,7 +547,7 @@ func ReadVRSettings(dbClient DBClient, uid int32) (VRSettings, error) { maxTPS, err := vrRow.ToInt64("max_tps") if err != nil { - return VRSettings{}, fmt.Errorf("failed to parse max_tps column2: %v", err) + return VRSettings{}, fmt.Errorf("failed to parse max_tps column: %v", err) } maxReplicationLag, err := vrRow.ToInt64("max_replication_lag") if err != nil { @@ -588,7 +578,7 @@ func ReadVRSettings(dbClient DBClient, uid int32) (VRSettings, error) { StopPos: stopPos, MaxTPS: maxTPS, MaxReplicationLag: maxReplicationLag, - State: vrRow.AsString("state", ""), + State: binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[vrRow.AsString("state", "")]), WorkflowType: binlogdatapb.VReplicationWorkflowType(workflowType), WorkflowName: vrRow.AsString("workflow", ""), WorkflowSubType: binlogdatapb.VReplicationWorkflowSubType(workflowSubType), @@ -604,17 +594,17 @@ func CreateVReplication(workflow string, source *binlogdatapb.BinlogSource, posi "(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)", encodeString(workflow), encodeString(source.String()), encodeString(position), maxTPS, maxReplicationLag, - timeUpdated, BlpRunning, encodeString(dbName), workflowType, workflowSubType, deferSecondaryKeys) + timeUpdated, binlogdatapb.VReplicationWorkflowState_Running.String(), encodeString(dbName), workflowType, workflowSubType, deferSecondaryKeys) } // CreateVReplicationState returns a statement to create a stopped vreplication. -func CreateVReplicationState(workflow string, source *binlogdatapb.BinlogSource, position, state string, dbName string, +func CreateVReplicationState(workflow string, source *binlogdatapb.BinlogSource, position string, state binlogdatapb.VReplicationWorkflowState, dbName string, workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType) string { 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)", encodeString(workflow), encodeString(source.String()), encodeString(position), throttler.MaxRateModuleDisabled, - throttler.ReplicationLagModuleDisabled, time.Now().Unix(), state, encodeString(dbName), + throttler.ReplicationLagModuleDisabled, time.Now().Unix(), state.String(), encodeString(dbName), workflowType, workflowSubType) } @@ -658,21 +648,21 @@ func GenerateUpdateTimeThrottled(uid int32, timeThrottledUnix int64, componentTh func StartVReplication(uid int32) string { return fmt.Sprintf( "update _vt.vreplication set state='%v', stop_pos=NULL where id=%v", - BlpRunning, uid) + binlogdatapb.VReplicationWorkflowState_Running.String(), uid) } // StartVReplicationUntil returns a statement to start the replication with a stop position. func StartVReplicationUntil(uid int32, pos string) string { return fmt.Sprintf( "update _vt.vreplication set state='%v', stop_pos=%v where id=%v", - BlpRunning, encodeString(pos), uid) + binlogdatapb.VReplicationWorkflowState_Running.String(), encodeString(pos), uid) } // StopVReplication returns a statement to stop the replication. func StopVReplication(uid int32, message string) string { return fmt.Sprintf( "update _vt.vreplication set state='%v', message=%v where id=%v", - BlpStopped, encodeString(MessageTruncate(message)), uid) + binlogdatapb.VReplicationWorkflowState_Stopped.String(), encodeString(MessageTruncate(message)), uid) } // DeleteVReplication returns a statement to delete the replication. diff --git a/go/vt/binlog/binlogplayer/binlog_player_test.go b/go/vt/binlog/binlogplayer/binlog_player_test.go index 20f75430644..e8dd93ef46e 100644 --- a/go/vt/binlog/binlogplayer/binlog_player_test.go +++ b/go/vt/binlog/binlogplayer/binlog_player_test.go @@ -54,11 +54,11 @@ var ( sqltypes.NULL, // stop_pos sqltypes.NewInt64(9223372036854775807), // max_tps sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(0), // workflow_sub_type - sqltypes.NewInt64(0), // defer_secondary_keys + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(0), // workflow_sub_type + sqltypes.NewInt64(0), // defer_secondary_keys }, }, } @@ -171,15 +171,15 @@ func TestStopPosEqual(t *testing.T) { InsertID: 0, Rows: [][]sqltypes.Value{ { - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // stop_pos - sqltypes.NewInt64(9223372036854775807), // max_tps - sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(1), // workflow_sub_type - sqltypes.NewInt64(1), // defer_secondary_keys + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // stop_pos + sqltypes.NewInt64(9223372036854775807), // max_tps + sqltypes.NewInt64(9223372036854775807), // max_replication_lag + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(1), // workflow_sub_type + sqltypes.NewInt64(1), // defer_secondary_keys }, }, } @@ -208,15 +208,15 @@ func TestStopPosLess(t *testing.T) { InsertID: 0, Rows: [][]sqltypes.Value{ { - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos - sqltypes.NewVarBinary("MariaDB/0-1-1082"), // stop_pos - sqltypes.NewInt64(9223372036854775807), // max_tps - sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(1), // workflow_sub_type - sqltypes.NewInt64(1), // defer_secondary_keys + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos + sqltypes.NewVarBinary("MariaDB/0-1-1082"), // stop_pos + sqltypes.NewInt64(9223372036854775807), // max_tps + sqltypes.NewInt64(9223372036854775807), // max_replication_lag + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(1), // workflow_sub_type + sqltypes.NewInt64(1), // defer_secondary_keys }, }, } @@ -245,15 +245,15 @@ func TestStopPosGreater(t *testing.T) { InsertID: 0, Rows: [][]sqltypes.Value{ { - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos - sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos - sqltypes.NewInt64(9223372036854775807), // max_tps - sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(1), // workflow_sub_type - sqltypes.NewInt64(1), // defer_secondary_keys + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos + sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos + sqltypes.NewInt64(9223372036854775807), // max_tps + sqltypes.NewInt64(9223372036854775807), // max_replication_lag + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(1), // workflow_sub_type + sqltypes.NewInt64(1), // defer_secondary_keys }, }, } @@ -286,15 +286,15 @@ func TestContextCancel(t *testing.T) { InsertID: 0, Rows: [][]sqltypes.Value{ { - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos - sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos - sqltypes.NewInt64(9223372036854775807), // max_tps - sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(1), // workflow_sub_type - sqltypes.NewInt64(1), // defer_secondary_keys + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos + sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos + sqltypes.NewInt64(9223372036854775807), // max_tps + sqltypes.NewInt64(9223372036854775807), // max_replication_lag + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(1), // workflow_sub_type + sqltypes.NewInt64(1), // defer_secondary_keys }, }, } diff --git a/go/vt/discovery/tablet_picker.go b/go/vt/discovery/tablet_picker.go index 20d4126831a..3ebcfa9c06f 100644 --- a/go/vt/discovery/tablet_picker.go +++ b/go/vt/discovery/tablet_picker.go @@ -58,13 +58,13 @@ const ( TabletPickerTabletOrder_Any TabletPickerTabletOrder = iota // Provided tablet types are expected to be prioritized in the given order. TabletPickerTabletOrder_InOrder + InOrderHint = "in_order:" ) var ( tabletPickerRetryDelay = 30 * time.Second muTabletPickerRetryDelay sync.Mutex globalTPStats *tabletPickerStats - inOrderHint = "in_order:" tabletPickerCellPreferenceMap = map[string]TabletPickerCellPreference{ "preferlocalwithalias": TabletPickerCellPreference_PreferLocalWithAlias, @@ -457,7 +457,7 @@ func (tp *TabletPicker) incNoTabletFoundStat() { globalTPStats.mu.Lock() defer globalTPStats.mu.Unlock() cells := strings.Join(tp.cells, "_") - tabletTypes := strings.Join(topoproto.MakeStringTypeList(tp.tabletTypes), "_") + tabletTypes := strings.ReplaceAll(topoproto.MakeStringTypeCSV(tp.tabletTypes), ",", "_") labels := []string{cells, tp.keyspace, tp.shard, tabletTypes} globalTPStats.noTabletFoundError.Add(labels, 1) } diff --git a/go/vt/discovery/utils.go b/go/vt/discovery/utils.go index 02f3b7132af..3a601830d35 100644 --- a/go/vt/discovery/utils.go +++ b/go/vt/discovery/utils.go @@ -50,9 +50,9 @@ func RemoveUnhealthyTablets(tabletStatsList []TabletHealth) []TabletHealth { func ParseTabletTypesAndOrder(tabletTypesStr string) ([]topodatapb.TabletType, bool, error) { inOrder := false - if strings.HasPrefix(tabletTypesStr, inOrderHint) { + if strings.HasPrefix(tabletTypesStr, InOrderHint) { inOrder = true - tabletTypesStr = tabletTypesStr[len(inOrderHint):] + tabletTypesStr = tabletTypesStr[len(InOrderHint):] } tabletTypes, err := topoproto.ParseTabletTypes(tabletTypesStr) diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index eb29ff97c05..66a9b2c6d92 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -201,6 +201,68 @@ func (VReplicationWorkflowSubType) EnumDescriptor() ([]byte, []int) { return file_binlogdata_proto_rawDescGZIP(), []int{2} } +// VReplicationWorklfowState defines the valid states that a workflow can be in. +type VReplicationWorkflowState int32 + +const ( + VReplicationWorkflowState_Unknown VReplicationWorkflowState = 0 + VReplicationWorkflowState_Init VReplicationWorkflowState = 1 + VReplicationWorkflowState_Stopped VReplicationWorkflowState = 2 + VReplicationWorkflowState_Copying VReplicationWorkflowState = 3 + VReplicationWorkflowState_Running VReplicationWorkflowState = 4 + VReplicationWorkflowState_Error VReplicationWorkflowState = 5 + VReplicationWorkflowState_Lagging VReplicationWorkflowState = 6 +) + +// Enum value maps for VReplicationWorkflowState. +var ( + VReplicationWorkflowState_name = map[int32]string{ + 0: "Unknown", + 1: "Init", + 2: "Stopped", + 3: "Copying", + 4: "Running", + 5: "Error", + 6: "Lagging", + } + VReplicationWorkflowState_value = map[string]int32{ + "Unknown": 0, + "Init": 1, + "Stopped": 2, + "Copying": 3, + "Running": 4, + "Error": 5, + "Lagging": 6, + } +) + +func (x VReplicationWorkflowState) Enum() *VReplicationWorkflowState { + p := new(VReplicationWorkflowState) + *p = x + return p +} + +func (x VReplicationWorkflowState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (VReplicationWorkflowState) Descriptor() protoreflect.EnumDescriptor { + return file_binlogdata_proto_enumTypes[3].Descriptor() +} + +func (VReplicationWorkflowState) Type() protoreflect.EnumType { + return &file_binlogdata_proto_enumTypes[3] +} + +func (x VReplicationWorkflowState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use VReplicationWorkflowState.Descriptor instead. +func (VReplicationWorkflowState) EnumDescriptor() ([]byte, []int) { + return file_binlogdata_proto_rawDescGZIP(), []int{3} +} + // VEventType enumerates the event types. Many of these types // will not be encountered in RBR mode. type VEventType int32 @@ -301,11 +363,11 @@ func (x VEventType) String() string { } func (VEventType) Descriptor() protoreflect.EnumDescriptor { - return file_binlogdata_proto_enumTypes[3].Descriptor() + return file_binlogdata_proto_enumTypes[4].Descriptor() } func (VEventType) Type() protoreflect.EnumType { - return &file_binlogdata_proto_enumTypes[3] + return &file_binlogdata_proto_enumTypes[4] } func (x VEventType) Number() protoreflect.EnumNumber { @@ -314,7 +376,7 @@ func (x VEventType) Number() protoreflect.EnumNumber { // Deprecated: Use VEventType.Descriptor instead. func (VEventType) EnumDescriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{3} + return file_binlogdata_proto_rawDescGZIP(), []int{4} } // MigrationType specifies the type of migration for the Journal. @@ -348,11 +410,11 @@ func (x MigrationType) String() string { } func (MigrationType) Descriptor() protoreflect.EnumDescriptor { - return file_binlogdata_proto_enumTypes[4].Descriptor() + return file_binlogdata_proto_enumTypes[5].Descriptor() } func (MigrationType) Type() protoreflect.EnumType { - return &file_binlogdata_proto_enumTypes[4] + return &file_binlogdata_proto_enumTypes[5] } func (x MigrationType) Number() protoreflect.EnumNumber { @@ -361,7 +423,7 @@ func (x MigrationType) Number() protoreflect.EnumNumber { // Deprecated: Use MigrationType.Descriptor instead. func (MigrationType) EnumDescriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{4} + return file_binlogdata_proto_rawDescGZIP(), []int{5} } type BinlogTransaction_Statement_Category int32 @@ -419,11 +481,11 @@ func (x BinlogTransaction_Statement_Category) String() string { } func (BinlogTransaction_Statement_Category) Descriptor() protoreflect.EnumDescriptor { - return file_binlogdata_proto_enumTypes[5].Descriptor() + return file_binlogdata_proto_enumTypes[6].Descriptor() } func (BinlogTransaction_Statement_Category) Type() protoreflect.EnumType { - return &file_binlogdata_proto_enumTypes[5] + return &file_binlogdata_proto_enumTypes[6] } func (x BinlogTransaction_Statement_Category) Number() protoreflect.EnumNumber { @@ -465,11 +527,11 @@ func (x Filter_FieldEventMode) String() string { } func (Filter_FieldEventMode) Descriptor() protoreflect.EnumDescriptor { - return file_binlogdata_proto_enumTypes[6].Descriptor() + return file_binlogdata_proto_enumTypes[7].Descriptor() } func (Filter_FieldEventMode) Type() protoreflect.EnumType { - return &file_binlogdata_proto_enumTypes[6] + return &file_binlogdata_proto_enumTypes[7] } func (x Filter_FieldEventMode) Number() protoreflect.EnumNumber { @@ -3129,30 +3191,37 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x44, 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x34, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, - 0x0a, 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x2a, 0x8d, 0x02, 0x0a, 0x0a, - 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, - 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, 0x49, 0x44, 0x10, - 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, - 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f, 0x4c, 0x4c, - 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, 0x10, 0x05, 0x12, - 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x52, - 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, 0x44, 0x41, - 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x09, - 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x4f, 0x54, 0x48, - 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, 0x12, 0x09, 0x0a, - 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x45, 0x41, 0x52, - 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, 0x54, 0x49, 0x44, - 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, 0x10, 0x10, 0x12, - 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, 0x0a, 0x0a, 0x06, - 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x41, 0x56, 0x45, - 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x50, 0x59, 0x5f, - 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, 0x0a, 0x0d, 0x4d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, - 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x48, 0x41, 0x52, - 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, - 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x0a, 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x2a, 0x71, 0x0a, 0x19, 0x56, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, + 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, + 0x0b, 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, + 0x43, 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, + 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, + 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x2a, 0x8d, + 0x02, 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, + 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, + 0x49, 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, + 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, + 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, + 0x10, 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, + 0x0a, 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, + 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, + 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, + 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, + 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, + 0x45, 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, + 0x54, 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, + 0x10, 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, + 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, + 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, + 0x50, 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, + 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x0a, 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, + 0x48, 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, + 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, + 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, + 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3167,118 +3236,119 @@ func file_binlogdata_proto_rawDescGZIP() []byte { return file_binlogdata_proto_rawDescData } -var file_binlogdata_proto_enumTypes = make([]protoimpl.EnumInfo, 7) +var file_binlogdata_proto_enumTypes = make([]protoimpl.EnumInfo, 8) var file_binlogdata_proto_msgTypes = make([]protoimpl.MessageInfo, 33) var file_binlogdata_proto_goTypes = []interface{}{ (OnDDLAction)(0), // 0: binlogdata.OnDDLAction (VReplicationWorkflowType)(0), // 1: binlogdata.VReplicationWorkflowType (VReplicationWorkflowSubType)(0), // 2: binlogdata.VReplicationWorkflowSubType - (VEventType)(0), // 3: binlogdata.VEventType - (MigrationType)(0), // 4: binlogdata.MigrationType - (BinlogTransaction_Statement_Category)(0), // 5: binlogdata.BinlogTransaction.Statement.Category - (Filter_FieldEventMode)(0), // 6: binlogdata.Filter.FieldEventMode - (*Charset)(nil), // 7: binlogdata.Charset - (*BinlogTransaction)(nil), // 8: binlogdata.BinlogTransaction - (*StreamKeyRangeRequest)(nil), // 9: binlogdata.StreamKeyRangeRequest - (*StreamKeyRangeResponse)(nil), // 10: binlogdata.StreamKeyRangeResponse - (*StreamTablesRequest)(nil), // 11: binlogdata.StreamTablesRequest - (*StreamTablesResponse)(nil), // 12: binlogdata.StreamTablesResponse - (*CharsetConversion)(nil), // 13: binlogdata.CharsetConversion - (*Rule)(nil), // 14: binlogdata.Rule - (*Filter)(nil), // 15: binlogdata.Filter - (*BinlogSource)(nil), // 16: binlogdata.BinlogSource - (*RowChange)(nil), // 17: binlogdata.RowChange - (*RowEvent)(nil), // 18: binlogdata.RowEvent - (*FieldEvent)(nil), // 19: binlogdata.FieldEvent - (*ShardGtid)(nil), // 20: binlogdata.ShardGtid - (*VGtid)(nil), // 21: binlogdata.VGtid - (*KeyspaceShard)(nil), // 22: binlogdata.KeyspaceShard - (*Journal)(nil), // 23: binlogdata.Journal - (*VEvent)(nil), // 24: binlogdata.VEvent - (*MinimalTable)(nil), // 25: binlogdata.MinimalTable - (*MinimalSchema)(nil), // 26: binlogdata.MinimalSchema - (*VStreamRequest)(nil), // 27: binlogdata.VStreamRequest - (*VStreamResponse)(nil), // 28: binlogdata.VStreamResponse - (*VStreamRowsRequest)(nil), // 29: binlogdata.VStreamRowsRequest - (*VStreamRowsResponse)(nil), // 30: binlogdata.VStreamRowsResponse - (*LastPKEvent)(nil), // 31: binlogdata.LastPKEvent - (*TableLastPK)(nil), // 32: binlogdata.TableLastPK - (*VStreamResultsRequest)(nil), // 33: binlogdata.VStreamResultsRequest - (*VStreamResultsResponse)(nil), // 34: binlogdata.VStreamResultsResponse - (*BinlogTransaction_Statement)(nil), // 35: binlogdata.BinlogTransaction.Statement - nil, // 36: binlogdata.Rule.ConvertEnumToTextEntry - nil, // 37: binlogdata.Rule.ConvertCharsetEntry - nil, // 38: binlogdata.Rule.ConvertIntToEnumEntry - (*RowChange_Bitmap)(nil), // 39: binlogdata.RowChange.Bitmap - (*query.EventToken)(nil), // 40: query.EventToken - (*topodata.KeyRange)(nil), // 41: topodata.KeyRange - (topodata.TabletType)(0), // 42: topodata.TabletType - (*query.Row)(nil), // 43: query.Row - (*query.Field)(nil), // 44: query.Field - (*vtrpc.CallerID)(nil), // 45: vtrpc.CallerID - (*query.VTGateCallerID)(nil), // 46: query.VTGateCallerID - (*query.Target)(nil), // 47: query.Target - (*query.QueryResult)(nil), // 48: query.QueryResult + (VReplicationWorkflowState)(0), // 3: binlogdata.VReplicationWorkflowState + (VEventType)(0), // 4: binlogdata.VEventType + (MigrationType)(0), // 5: binlogdata.MigrationType + (BinlogTransaction_Statement_Category)(0), // 6: binlogdata.BinlogTransaction.Statement.Category + (Filter_FieldEventMode)(0), // 7: binlogdata.Filter.FieldEventMode + (*Charset)(nil), // 8: binlogdata.Charset + (*BinlogTransaction)(nil), // 9: binlogdata.BinlogTransaction + (*StreamKeyRangeRequest)(nil), // 10: binlogdata.StreamKeyRangeRequest + (*StreamKeyRangeResponse)(nil), // 11: binlogdata.StreamKeyRangeResponse + (*StreamTablesRequest)(nil), // 12: binlogdata.StreamTablesRequest + (*StreamTablesResponse)(nil), // 13: binlogdata.StreamTablesResponse + (*CharsetConversion)(nil), // 14: binlogdata.CharsetConversion + (*Rule)(nil), // 15: binlogdata.Rule + (*Filter)(nil), // 16: binlogdata.Filter + (*BinlogSource)(nil), // 17: binlogdata.BinlogSource + (*RowChange)(nil), // 18: binlogdata.RowChange + (*RowEvent)(nil), // 19: binlogdata.RowEvent + (*FieldEvent)(nil), // 20: binlogdata.FieldEvent + (*ShardGtid)(nil), // 21: binlogdata.ShardGtid + (*VGtid)(nil), // 22: binlogdata.VGtid + (*KeyspaceShard)(nil), // 23: binlogdata.KeyspaceShard + (*Journal)(nil), // 24: binlogdata.Journal + (*VEvent)(nil), // 25: binlogdata.VEvent + (*MinimalTable)(nil), // 26: binlogdata.MinimalTable + (*MinimalSchema)(nil), // 27: binlogdata.MinimalSchema + (*VStreamRequest)(nil), // 28: binlogdata.VStreamRequest + (*VStreamResponse)(nil), // 29: binlogdata.VStreamResponse + (*VStreamRowsRequest)(nil), // 30: binlogdata.VStreamRowsRequest + (*VStreamRowsResponse)(nil), // 31: binlogdata.VStreamRowsResponse + (*LastPKEvent)(nil), // 32: binlogdata.LastPKEvent + (*TableLastPK)(nil), // 33: binlogdata.TableLastPK + (*VStreamResultsRequest)(nil), // 34: binlogdata.VStreamResultsRequest + (*VStreamResultsResponse)(nil), // 35: binlogdata.VStreamResultsResponse + (*BinlogTransaction_Statement)(nil), // 36: binlogdata.BinlogTransaction.Statement + nil, // 37: binlogdata.Rule.ConvertEnumToTextEntry + nil, // 38: binlogdata.Rule.ConvertCharsetEntry + nil, // 39: binlogdata.Rule.ConvertIntToEnumEntry + (*RowChange_Bitmap)(nil), // 40: binlogdata.RowChange.Bitmap + (*query.EventToken)(nil), // 41: query.EventToken + (*topodata.KeyRange)(nil), // 42: topodata.KeyRange + (topodata.TabletType)(0), // 43: topodata.TabletType + (*query.Row)(nil), // 44: query.Row + (*query.Field)(nil), // 45: query.Field + (*vtrpc.CallerID)(nil), // 46: vtrpc.CallerID + (*query.VTGateCallerID)(nil), // 47: query.VTGateCallerID + (*query.Target)(nil), // 48: query.Target + (*query.QueryResult)(nil), // 49: query.QueryResult } var file_binlogdata_proto_depIdxs = []int32{ - 35, // 0: binlogdata.BinlogTransaction.statements:type_name -> binlogdata.BinlogTransaction.Statement - 40, // 1: binlogdata.BinlogTransaction.event_token:type_name -> query.EventToken - 41, // 2: binlogdata.StreamKeyRangeRequest.key_range:type_name -> topodata.KeyRange - 7, // 3: binlogdata.StreamKeyRangeRequest.charset:type_name -> binlogdata.Charset - 8, // 4: binlogdata.StreamKeyRangeResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction - 7, // 5: binlogdata.StreamTablesRequest.charset:type_name -> binlogdata.Charset - 8, // 6: binlogdata.StreamTablesResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction - 36, // 7: binlogdata.Rule.convert_enum_to_text:type_name -> binlogdata.Rule.ConvertEnumToTextEntry - 37, // 8: binlogdata.Rule.convert_charset:type_name -> binlogdata.Rule.ConvertCharsetEntry - 38, // 9: binlogdata.Rule.convert_int_to_enum:type_name -> binlogdata.Rule.ConvertIntToEnumEntry - 14, // 10: binlogdata.Filter.rules:type_name -> binlogdata.Rule - 6, // 11: binlogdata.Filter.field_event_mode:type_name -> binlogdata.Filter.FieldEventMode - 42, // 12: binlogdata.BinlogSource.tablet_type:type_name -> topodata.TabletType - 41, // 13: binlogdata.BinlogSource.key_range:type_name -> topodata.KeyRange - 15, // 14: binlogdata.BinlogSource.filter:type_name -> binlogdata.Filter + 36, // 0: binlogdata.BinlogTransaction.statements:type_name -> binlogdata.BinlogTransaction.Statement + 41, // 1: binlogdata.BinlogTransaction.event_token:type_name -> query.EventToken + 42, // 2: binlogdata.StreamKeyRangeRequest.key_range:type_name -> topodata.KeyRange + 8, // 3: binlogdata.StreamKeyRangeRequest.charset:type_name -> binlogdata.Charset + 9, // 4: binlogdata.StreamKeyRangeResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction + 8, // 5: binlogdata.StreamTablesRequest.charset:type_name -> binlogdata.Charset + 9, // 6: binlogdata.StreamTablesResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction + 37, // 7: binlogdata.Rule.convert_enum_to_text:type_name -> binlogdata.Rule.ConvertEnumToTextEntry + 38, // 8: binlogdata.Rule.convert_charset:type_name -> binlogdata.Rule.ConvertCharsetEntry + 39, // 9: binlogdata.Rule.convert_int_to_enum:type_name -> binlogdata.Rule.ConvertIntToEnumEntry + 15, // 10: binlogdata.Filter.rules:type_name -> binlogdata.Rule + 7, // 11: binlogdata.Filter.field_event_mode:type_name -> binlogdata.Filter.FieldEventMode + 43, // 12: binlogdata.BinlogSource.tablet_type:type_name -> topodata.TabletType + 42, // 13: binlogdata.BinlogSource.key_range:type_name -> topodata.KeyRange + 16, // 14: binlogdata.BinlogSource.filter:type_name -> binlogdata.Filter 0, // 15: binlogdata.BinlogSource.on_ddl:type_name -> binlogdata.OnDDLAction - 43, // 16: binlogdata.RowChange.before:type_name -> query.Row - 43, // 17: binlogdata.RowChange.after:type_name -> query.Row - 39, // 18: binlogdata.RowChange.data_columns:type_name -> binlogdata.RowChange.Bitmap - 17, // 19: binlogdata.RowEvent.row_changes:type_name -> binlogdata.RowChange - 44, // 20: binlogdata.FieldEvent.fields:type_name -> query.Field - 32, // 21: binlogdata.ShardGtid.table_p_ks:type_name -> binlogdata.TableLastPK - 20, // 22: binlogdata.VGtid.shard_gtids:type_name -> binlogdata.ShardGtid - 4, // 23: binlogdata.Journal.migration_type:type_name -> binlogdata.MigrationType - 20, // 24: binlogdata.Journal.shard_gtids:type_name -> binlogdata.ShardGtid - 22, // 25: binlogdata.Journal.participants:type_name -> binlogdata.KeyspaceShard - 3, // 26: binlogdata.VEvent.type:type_name -> binlogdata.VEventType - 18, // 27: binlogdata.VEvent.row_event:type_name -> binlogdata.RowEvent - 19, // 28: binlogdata.VEvent.field_event:type_name -> binlogdata.FieldEvent - 21, // 29: binlogdata.VEvent.vgtid:type_name -> binlogdata.VGtid - 23, // 30: binlogdata.VEvent.journal:type_name -> binlogdata.Journal - 31, // 31: binlogdata.VEvent.last_p_k_event:type_name -> binlogdata.LastPKEvent - 44, // 32: binlogdata.MinimalTable.fields:type_name -> query.Field - 25, // 33: binlogdata.MinimalSchema.tables:type_name -> binlogdata.MinimalTable - 45, // 34: binlogdata.VStreamRequest.effective_caller_id:type_name -> vtrpc.CallerID - 46, // 35: binlogdata.VStreamRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 47, // 36: binlogdata.VStreamRequest.target:type_name -> query.Target - 15, // 37: binlogdata.VStreamRequest.filter:type_name -> binlogdata.Filter - 32, // 38: binlogdata.VStreamRequest.table_last_p_ks:type_name -> binlogdata.TableLastPK - 24, // 39: binlogdata.VStreamResponse.events:type_name -> binlogdata.VEvent - 45, // 40: binlogdata.VStreamRowsRequest.effective_caller_id:type_name -> vtrpc.CallerID - 46, // 41: binlogdata.VStreamRowsRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 47, // 42: binlogdata.VStreamRowsRequest.target:type_name -> query.Target - 48, // 43: binlogdata.VStreamRowsRequest.lastpk:type_name -> query.QueryResult - 44, // 44: binlogdata.VStreamRowsResponse.fields:type_name -> query.Field - 44, // 45: binlogdata.VStreamRowsResponse.pkfields:type_name -> query.Field - 43, // 46: binlogdata.VStreamRowsResponse.rows:type_name -> query.Row - 43, // 47: binlogdata.VStreamRowsResponse.lastpk:type_name -> query.Row - 32, // 48: binlogdata.LastPKEvent.table_last_p_k:type_name -> binlogdata.TableLastPK - 48, // 49: binlogdata.TableLastPK.lastpk:type_name -> query.QueryResult - 45, // 50: binlogdata.VStreamResultsRequest.effective_caller_id:type_name -> vtrpc.CallerID - 46, // 51: binlogdata.VStreamResultsRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 47, // 52: binlogdata.VStreamResultsRequest.target:type_name -> query.Target - 44, // 53: binlogdata.VStreamResultsResponse.fields:type_name -> query.Field - 43, // 54: binlogdata.VStreamResultsResponse.rows:type_name -> query.Row - 5, // 55: binlogdata.BinlogTransaction.Statement.category:type_name -> binlogdata.BinlogTransaction.Statement.Category - 7, // 56: binlogdata.BinlogTransaction.Statement.charset:type_name -> binlogdata.Charset - 13, // 57: binlogdata.Rule.ConvertCharsetEntry.value:type_name -> binlogdata.CharsetConversion + 44, // 16: binlogdata.RowChange.before:type_name -> query.Row + 44, // 17: binlogdata.RowChange.after:type_name -> query.Row + 40, // 18: binlogdata.RowChange.data_columns:type_name -> binlogdata.RowChange.Bitmap + 18, // 19: binlogdata.RowEvent.row_changes:type_name -> binlogdata.RowChange + 45, // 20: binlogdata.FieldEvent.fields:type_name -> query.Field + 33, // 21: binlogdata.ShardGtid.table_p_ks:type_name -> binlogdata.TableLastPK + 21, // 22: binlogdata.VGtid.shard_gtids:type_name -> binlogdata.ShardGtid + 5, // 23: binlogdata.Journal.migration_type:type_name -> binlogdata.MigrationType + 21, // 24: binlogdata.Journal.shard_gtids:type_name -> binlogdata.ShardGtid + 23, // 25: binlogdata.Journal.participants:type_name -> binlogdata.KeyspaceShard + 4, // 26: binlogdata.VEvent.type:type_name -> binlogdata.VEventType + 19, // 27: binlogdata.VEvent.row_event:type_name -> binlogdata.RowEvent + 20, // 28: binlogdata.VEvent.field_event:type_name -> binlogdata.FieldEvent + 22, // 29: binlogdata.VEvent.vgtid:type_name -> binlogdata.VGtid + 24, // 30: binlogdata.VEvent.journal:type_name -> binlogdata.Journal + 32, // 31: binlogdata.VEvent.last_p_k_event:type_name -> binlogdata.LastPKEvent + 45, // 32: binlogdata.MinimalTable.fields:type_name -> query.Field + 26, // 33: binlogdata.MinimalSchema.tables:type_name -> binlogdata.MinimalTable + 46, // 34: binlogdata.VStreamRequest.effective_caller_id:type_name -> vtrpc.CallerID + 47, // 35: binlogdata.VStreamRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 48, // 36: binlogdata.VStreamRequest.target:type_name -> query.Target + 16, // 37: binlogdata.VStreamRequest.filter:type_name -> binlogdata.Filter + 33, // 38: binlogdata.VStreamRequest.table_last_p_ks:type_name -> binlogdata.TableLastPK + 25, // 39: binlogdata.VStreamResponse.events:type_name -> binlogdata.VEvent + 46, // 40: binlogdata.VStreamRowsRequest.effective_caller_id:type_name -> vtrpc.CallerID + 47, // 41: binlogdata.VStreamRowsRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 48, // 42: binlogdata.VStreamRowsRequest.target:type_name -> query.Target + 49, // 43: binlogdata.VStreamRowsRequest.lastpk:type_name -> query.QueryResult + 45, // 44: binlogdata.VStreamRowsResponse.fields:type_name -> query.Field + 45, // 45: binlogdata.VStreamRowsResponse.pkfields:type_name -> query.Field + 44, // 46: binlogdata.VStreamRowsResponse.rows:type_name -> query.Row + 44, // 47: binlogdata.VStreamRowsResponse.lastpk:type_name -> query.Row + 33, // 48: binlogdata.LastPKEvent.table_last_p_k:type_name -> binlogdata.TableLastPK + 49, // 49: binlogdata.TableLastPK.lastpk:type_name -> query.QueryResult + 46, // 50: binlogdata.VStreamResultsRequest.effective_caller_id:type_name -> vtrpc.CallerID + 47, // 51: binlogdata.VStreamResultsRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 48, // 52: binlogdata.VStreamResultsRequest.target:type_name -> query.Target + 45, // 53: binlogdata.VStreamResultsResponse.fields:type_name -> query.Field + 44, // 54: binlogdata.VStreamResultsResponse.rows:type_name -> query.Row + 6, // 55: binlogdata.BinlogTransaction.Statement.category:type_name -> binlogdata.BinlogTransaction.Statement.Category + 8, // 56: binlogdata.BinlogTransaction.Statement.charset:type_name -> binlogdata.Charset + 14, // 57: binlogdata.Rule.ConvertCharsetEntry.value:type_name -> binlogdata.CharsetConversion 58, // [58:58] is the sub-list for method output_type 58, // [58:58] is the sub-list for method input_type 58, // [58:58] is the sub-list for extension type_name @@ -3658,7 +3728,7 @@ func file_binlogdata_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_binlogdata_proto_rawDesc, - NumEnums: 7, + NumEnums: 8, NumMessages: 33, NumExtensions: 0, NumServices: 0, diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 977a40d22f8..1199ca6cb46 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -45,6 +45,57 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This structure allows us to manage tablet selection preferences +// which are eventually passed to a TabletPicker. +type TabletSelectionPreference int32 + +const ( + TabletSelectionPreference_ANY TabletSelectionPreference = 0 + TabletSelectionPreference_INORDER TabletSelectionPreference = 1 + TabletSelectionPreference_UNKNOWN TabletSelectionPreference = 3 // Don't change any existing value +) + +// Enum value maps for TabletSelectionPreference. +var ( + TabletSelectionPreference_name = map[int32]string{ + 0: "ANY", + 1: "INORDER", + 3: "UNKNOWN", + } + TabletSelectionPreference_value = map[string]int32{ + "ANY": 0, + "INORDER": 1, + "UNKNOWN": 3, + } +) + +func (x TabletSelectionPreference) Enum() *TabletSelectionPreference { + p := new(TabletSelectionPreference) + *p = x + return p +} + +func (x TabletSelectionPreference) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TabletSelectionPreference) Descriptor() protoreflect.EnumDescriptor { + return file_tabletmanagerdata_proto_enumTypes[0].Descriptor() +} + +func (TabletSelectionPreference) Type() protoreflect.EnumType { + return &file_tabletmanagerdata_proto_enumTypes[0] +} + +func (x TabletSelectionPreference) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TabletSelectionPreference.Descriptor instead. +func (TabletSelectionPreference) EnumDescriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{0} +} + type TableDefinition struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4727,21 +4778,31 @@ func (x *RestoreFromBackupResponse) GetEvent() *logutil.Event { return nil } -type VDiffRequest struct { +type CreateVReplicationWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` - Action string `protobuf:"bytes,3,opt,name=action,proto3" json:"action,omitempty"` - ActionArg string `protobuf:"bytes,4,opt,name=action_arg,json=actionArg,proto3" json:"action_arg,omitempty"` - VdiffUuid string `protobuf:"bytes,5,opt,name=vdiff_uuid,json=vdiffUuid,proto3" json:"vdiff_uuid,omitempty"` - Options *VDiffOptions `protobuf:"bytes,6,opt,name=options,proto3" json:"options,omitempty"` -} - -func (x *VDiffRequest) Reset() { - *x = VDiffRequest{} + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` + BinlogSource []*binlogdata.BinlogSource `protobuf:"bytes,2,rep,name=binlog_source,json=binlogSource,proto3" json:"binlog_source,omitempty"` + // Optional parameters. + Cells []string `protobuf:"bytes,3,rep,name=cells,proto3" json:"cells,omitempty"` + // TabletTypes is the list of tablet types to use when selecting source tablets. + TabletTypes []topodata.TabletType `protobuf:"varint,4,rep,packed,name=tablet_types,json=tabletTypes,proto3,enum=topodata.TabletType" json:"tablet_types,omitempty"` + TabletSelectionPreference TabletSelectionPreference `protobuf:"varint,5,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` + WorkflowType binlogdata.VReplicationWorkflowType `protobuf:"varint,6,opt,name=workflow_type,json=workflowType,proto3,enum=binlogdata.VReplicationWorkflowType" json:"workflow_type,omitempty"` + WorkflowSubType binlogdata.VReplicationWorkflowSubType `protobuf:"varint,7,opt,name=workflow_sub_type,json=workflowSubType,proto3,enum=binlogdata.VReplicationWorkflowSubType" json:"workflow_sub_type,omitempty"` + // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table + // copy finishes. + DeferSecondaryKeys bool `protobuf:"varint,8,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` + // 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"` +} + +func (x *CreateVReplicationWorkflowRequest) Reset() { + *x = CreateVReplicationWorkflowRequest{} if protoimpl.UnsafeEnabled { mi := &file_tabletmanagerdata_proto_msgTypes[96] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4749,13 +4810,13 @@ func (x *VDiffRequest) Reset() { } } -func (x *VDiffRequest) String() string { +func (x *CreateVReplicationWorkflowRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VDiffRequest) ProtoMessage() {} +func (*CreateVReplicationWorkflowRequest) ProtoMessage() {} -func (x *VDiffRequest) ProtoReflect() protoreflect.Message { +func (x *CreateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { mi := &file_tabletmanagerdata_proto_msgTypes[96] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4767,65 +4828,91 @@ func (x *VDiffRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VDiffRequest.ProtoReflect.Descriptor instead. -func (*VDiffRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use CreateVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. +func (*CreateVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { return file_tabletmanagerdata_proto_rawDescGZIP(), []int{96} } -func (x *VDiffRequest) GetKeyspace() string { +func (x *CreateVReplicationWorkflowRequest) GetWorkflow() string { if x != nil { - return x.Keyspace + return x.Workflow } return "" } -func (x *VDiffRequest) GetWorkflow() string { +func (x *CreateVReplicationWorkflowRequest) GetBinlogSource() []*binlogdata.BinlogSource { if x != nil { - return x.Workflow + return x.BinlogSource } - return "" + return nil } -func (x *VDiffRequest) GetAction() string { +func (x *CreateVReplicationWorkflowRequest) GetCells() []string { if x != nil { - return x.Action + return x.Cells } - return "" + return nil } -func (x *VDiffRequest) GetActionArg() string { +func (x *CreateVReplicationWorkflowRequest) GetTabletTypes() []topodata.TabletType { if x != nil { - return x.ActionArg + return x.TabletTypes } - return "" + return nil } -func (x *VDiffRequest) GetVdiffUuid() string { +func (x *CreateVReplicationWorkflowRequest) GetTabletSelectionPreference() TabletSelectionPreference { if x != nil { - return x.VdiffUuid + return x.TabletSelectionPreference } - return "" + return TabletSelectionPreference_ANY } -func (x *VDiffRequest) GetOptions() *VDiffOptions { +func (x *CreateVReplicationWorkflowRequest) GetWorkflowType() binlogdata.VReplicationWorkflowType { if x != nil { - return x.Options + return x.WorkflowType } - return nil + return binlogdata.VReplicationWorkflowType(0) } -type VDiffResponse struct { +func (x *CreateVReplicationWorkflowRequest) GetWorkflowSubType() binlogdata.VReplicationWorkflowSubType { + if x != nil { + return x.WorkflowSubType + } + return binlogdata.VReplicationWorkflowSubType(0) +} + +func (x *CreateVReplicationWorkflowRequest) GetDeferSecondaryKeys() bool { + if x != nil { + return x.DeferSecondaryKeys + } + return false +} + +func (x *CreateVReplicationWorkflowRequest) GetAutoStart() bool { + if x != nil { + return x.AutoStart + } + return false +} + +func (x *CreateVReplicationWorkflowRequest) GetStopAfterCopy() bool { + if x != nil { + return x.StopAfterCopy + } + return false +} + +type CreateVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` - Output *query.QueryResult `protobuf:"bytes,2,opt,name=output,proto3" json:"output,omitempty"` - VdiffUuid string `protobuf:"bytes,3,opt,name=vdiff_uuid,json=vdiffUuid,proto3" json:"vdiff_uuid,omitempty"` + Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` } -func (x *VDiffResponse) Reset() { - *x = VDiffResponse{} +func (x *CreateVReplicationWorkflowResponse) Reset() { + *x = CreateVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { mi := &file_tabletmanagerdata_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4833,13 +4920,13 @@ func (x *VDiffResponse) Reset() { } } -func (x *VDiffResponse) String() string { +func (x *CreateVReplicationWorkflowResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VDiffResponse) ProtoMessage() {} +func (*CreateVReplicationWorkflowResponse) ProtoMessage() {} -func (x *VDiffResponse) ProtoReflect() protoreflect.Message { +func (x *CreateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { mi := &file_tabletmanagerdata_proto_msgTypes[97] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4851,45 +4938,28 @@ func (x *VDiffResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VDiffResponse.ProtoReflect.Descriptor instead. -func (*VDiffResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use CreateVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. +func (*CreateVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { return file_tabletmanagerdata_proto_rawDescGZIP(), []int{97} } -func (x *VDiffResponse) GetId() int64 { - if x != nil { - return x.Id - } - return 0 -} - -func (x *VDiffResponse) GetOutput() *query.QueryResult { +func (x *CreateVReplicationWorkflowResponse) GetResult() *query.QueryResult { if x != nil { - return x.Output + return x.Result } return nil } -func (x *VDiffResponse) GetVdiffUuid() string { - if x != nil { - return x.VdiffUuid - } - return "" -} - -// options that influence the tablet selected by the picker for streaming data from -type VDiffPickerOptions struct { +type DeleteVReplicationWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - TabletTypes string `protobuf:"bytes,1,opt,name=tablet_types,json=tabletTypes,proto3" json:"tablet_types,omitempty"` - SourceCell string `protobuf:"bytes,2,opt,name=source_cell,json=sourceCell,proto3" json:"source_cell,omitempty"` - TargetCell string `protobuf:"bytes,3,opt,name=target_cell,json=targetCell,proto3" json:"target_cell,omitempty"` + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` } -func (x *VDiffPickerOptions) Reset() { - *x = VDiffPickerOptions{} +func (x *DeleteVReplicationWorkflowRequest) Reset() { + *x = DeleteVReplicationWorkflowRequest{} if protoimpl.UnsafeEnabled { mi := &file_tabletmanagerdata_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4897,13 +4967,13 @@ func (x *VDiffPickerOptions) Reset() { } } -func (x *VDiffPickerOptions) String() string { +func (x *DeleteVReplicationWorkflowRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VDiffPickerOptions) ProtoMessage() {} +func (*DeleteVReplicationWorkflowRequest) ProtoMessage() {} -func (x *VDiffPickerOptions) ProtoReflect() protoreflect.Message { +func (x *DeleteVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { mi := &file_tabletmanagerdata_proto_msgTypes[98] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4915,45 +4985,28 @@ func (x *VDiffPickerOptions) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VDiffPickerOptions.ProtoReflect.Descriptor instead. -func (*VDiffPickerOptions) Descriptor() ([]byte, []int) { +// Deprecated: Use DeleteVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. +func (*DeleteVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { return file_tabletmanagerdata_proto_rawDescGZIP(), []int{98} } -func (x *VDiffPickerOptions) GetTabletTypes() string { - if x != nil { - return x.TabletTypes - } - return "" -} - -func (x *VDiffPickerOptions) GetSourceCell() string { - if x != nil { - return x.SourceCell - } - return "" -} - -func (x *VDiffPickerOptions) GetTargetCell() string { +func (x *DeleteVReplicationWorkflowRequest) GetWorkflow() string { if x != nil { - return x.TargetCell + return x.Workflow } return "" } -// options that only influence how vdiff differences are reported -type VDiffReportOptions struct { +type DeleteVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - OnlyPks bool `protobuf:"varint,1,opt,name=only_pks,json=onlyPks,proto3" json:"only_pks,omitempty"` - DebugQuery bool `protobuf:"varint,2,opt,name=debug_query,json=debugQuery,proto3" json:"debug_query,omitempty"` - Format string `protobuf:"bytes,3,opt,name=format,proto3" json:"format,omitempty"` + Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` } -func (x *VDiffReportOptions) Reset() { - *x = VDiffReportOptions{} +func (x *DeleteVReplicationWorkflowResponse) Reset() { + *x = DeleteVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { mi := &file_tabletmanagerdata_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4961,13 +5014,13 @@ func (x *VDiffReportOptions) Reset() { } } -func (x *VDiffReportOptions) String() string { +func (x *DeleteVReplicationWorkflowResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VDiffReportOptions) ProtoMessage() {} +func (*DeleteVReplicationWorkflowResponse) ProtoMessage() {} -func (x *VDiffReportOptions) ProtoReflect() protoreflect.Message { +func (x *DeleteVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { mi := &file_tabletmanagerdata_proto_msgTypes[99] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -4979,64 +5032,99 @@ func (x *VDiffReportOptions) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VDiffReportOptions.ProtoReflect.Descriptor instead. -func (*VDiffReportOptions) Descriptor() ([]byte, []int) { +// Deprecated: Use DeleteVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. +func (*DeleteVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { return file_tabletmanagerdata_proto_rawDescGZIP(), []int{99} } -func (x *VDiffReportOptions) GetOnlyPks() bool { +func (x *DeleteVReplicationWorkflowResponse) GetResult() *query.QueryResult { if x != nil { - return x.OnlyPks + return x.Result } - return false + return nil } -func (x *VDiffReportOptions) GetDebugQuery() bool { - if x != nil { - return x.DebugQuery +type ReadVReplicationWorkflowRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` +} + +func (x *ReadVReplicationWorkflowRequest) Reset() { + *x = ReadVReplicationWorkflowRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return false } -func (x *VDiffReportOptions) GetFormat() string { +func (x *ReadVReplicationWorkflowRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadVReplicationWorkflowRequest) ProtoMessage() {} + +func (x *ReadVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[100] + 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 ReadVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. +func (*ReadVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{100} +} + +func (x *ReadVReplicationWorkflowRequest) GetWorkflow() string { if x != nil { - return x.Format + return x.Workflow } return "" } -type VDiffCoreOptions struct { +type ReadVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Tables string `protobuf:"bytes,1,opt,name=tables,proto3" json:"tables,omitempty"` - AutoRetry bool `protobuf:"varint,2,opt,name=auto_retry,json=autoRetry,proto3" json:"auto_retry,omitempty"` - MaxRows int64 `protobuf:"varint,3,opt,name=max_rows,json=maxRows,proto3" json:"max_rows,omitempty"` - Checksum bool `protobuf:"varint,4,opt,name=checksum,proto3" json:"checksum,omitempty"` - SamplePct int64 `protobuf:"varint,5,opt,name=sample_pct,json=samplePct,proto3" json:"sample_pct,omitempty"` - TimeoutSeconds int64 `protobuf:"varint,6,opt,name=timeout_seconds,json=timeoutSeconds,proto3" json:"timeout_seconds,omitempty"` - MaxExtraRowsToCompare int64 `protobuf:"varint,7,opt,name=max_extra_rows_to_compare,json=maxExtraRowsToCompare,proto3" json:"max_extra_rows_to_compare,omitempty"` - UpdateTableStats bool `protobuf:"varint,8,opt,name=update_table_stats,json=updateTableStats,proto3" json:"update_table_stats,omitempty"` + Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` + Cells string `protobuf:"bytes,3,opt,name=cells,proto3" json:"cells,omitempty"` + TabletTypes []topodata.TabletType `protobuf:"varint,4,rep,packed,name=tablet_types,json=tabletTypes,proto3,enum=topodata.TabletType" json:"tablet_types,omitempty"` + TabletSelectionPreference TabletSelectionPreference `protobuf:"varint,5,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` + DbName string `protobuf:"bytes,6,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + Tags string `protobuf:"bytes,7,opt,name=tags,proto3" json:"tags,omitempty"` + WorkflowType binlogdata.VReplicationWorkflowType `protobuf:"varint,8,opt,name=workflow_type,json=workflowType,proto3,enum=binlogdata.VReplicationWorkflowType" json:"workflow_type,omitempty"` + 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"` } -func (x *VDiffCoreOptions) Reset() { - *x = VDiffCoreOptions{} +func (x *ReadVReplicationWorkflowResponse) Reset() { + *x = ReadVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[100] + mi := &file_tabletmanagerdata_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *VDiffCoreOptions) String() string { +func (x *ReadVReplicationWorkflowResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VDiffCoreOptions) ProtoMessage() {} +func (*ReadVReplicationWorkflowResponse) ProtoMessage() {} -func (x *VDiffCoreOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[100] +func (x *ReadVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[101] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5047,81 +5135,476 @@ func (x *VDiffCoreOptions) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VDiffCoreOptions.ProtoReflect.Descriptor instead. -func (*VDiffCoreOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{100} +// Deprecated: Use ReadVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. +func (*ReadVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101} } -func (x *VDiffCoreOptions) GetTables() string { +func (x *ReadVReplicationWorkflowResponse) GetWorkflow() string { if x != nil { - return x.Tables + return x.Workflow } return "" } -func (x *VDiffCoreOptions) GetAutoRetry() bool { +func (x *ReadVReplicationWorkflowResponse) GetCells() string { if x != nil { - return x.AutoRetry + return x.Cells } - return false + return "" } -func (x *VDiffCoreOptions) GetMaxRows() int64 { +func (x *ReadVReplicationWorkflowResponse) GetTabletTypes() []topodata.TabletType { if x != nil { - return x.MaxRows + return x.TabletTypes } - return 0 + return nil } -func (x *VDiffCoreOptions) GetChecksum() bool { +func (x *ReadVReplicationWorkflowResponse) GetTabletSelectionPreference() TabletSelectionPreference { if x != nil { - return x.Checksum + return x.TabletSelectionPreference } - return false + return TabletSelectionPreference_ANY } -func (x *VDiffCoreOptions) GetSamplePct() int64 { +func (x *ReadVReplicationWorkflowResponse) GetDbName() string { if x != nil { - return x.SamplePct + return x.DbName } - return 0 + return "" } -func (x *VDiffCoreOptions) GetTimeoutSeconds() int64 { +func (x *ReadVReplicationWorkflowResponse) GetTags() string { if x != nil { - return x.TimeoutSeconds + return x.Tags } - return 0 + return "" } -func (x *VDiffCoreOptions) GetMaxExtraRowsToCompare() int64 { +func (x *ReadVReplicationWorkflowResponse) GetWorkflowType() binlogdata.VReplicationWorkflowType { if x != nil { - return x.MaxExtraRowsToCompare + return x.WorkflowType } - return 0 + return binlogdata.VReplicationWorkflowType(0) } -func (x *VDiffCoreOptions) GetUpdateTableStats() bool { +func (x *ReadVReplicationWorkflowResponse) GetWorkflowSubType() binlogdata.VReplicationWorkflowSubType { if x != nil { - return x.UpdateTableStats + return x.WorkflowSubType + } + return binlogdata.VReplicationWorkflowSubType(0) +} + +func (x *ReadVReplicationWorkflowResponse) GetDeferSecondaryKeys() bool { + if x != nil { + return x.DeferSecondaryKeys } return false } -type VDiffOptions struct { +func (x *ReadVReplicationWorkflowResponse) GetStreams() []*ReadVReplicationWorkflowResponse_Stream { + if x != nil { + return x.Streams + } + return nil +} + +type VDiffRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - PickerOptions *VDiffPickerOptions `protobuf:"bytes,1,opt,name=picker_options,json=pickerOptions,proto3" json:"picker_options,omitempty"` - CoreOptions *VDiffCoreOptions `protobuf:"bytes,2,opt,name=core_options,json=coreOptions,proto3" json:"core_options,omitempty"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` + Action string `protobuf:"bytes,3,opt,name=action,proto3" json:"action,omitempty"` + ActionArg string `protobuf:"bytes,4,opt,name=action_arg,json=actionArg,proto3" json:"action_arg,omitempty"` + VdiffUuid string `protobuf:"bytes,5,opt,name=vdiff_uuid,json=vdiffUuid,proto3" json:"vdiff_uuid,omitempty"` + Options *VDiffOptions `protobuf:"bytes,6,opt,name=options,proto3" json:"options,omitempty"` +} + +func (x *VDiffRequest) Reset() { + *x = VDiffRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VDiffRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffRequest) ProtoMessage() {} + +func (x *VDiffRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[102] + 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 VDiffRequest.ProtoReflect.Descriptor instead. +func (*VDiffRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{102} +} + +func (x *VDiffRequest) GetKeyspace() string { + if x != nil { + return x.Keyspace + } + return "" +} + +func (x *VDiffRequest) GetWorkflow() string { + if x != nil { + return x.Workflow + } + return "" +} + +func (x *VDiffRequest) GetAction() string { + if x != nil { + return x.Action + } + return "" +} + +func (x *VDiffRequest) GetActionArg() string { + if x != nil { + return x.ActionArg + } + return "" +} + +func (x *VDiffRequest) GetVdiffUuid() string { + if x != nil { + return x.VdiffUuid + } + return "" +} + +func (x *VDiffRequest) GetOptions() *VDiffOptions { + if x != nil { + return x.Options + } + return nil +} + +type VDiffResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Output *query.QueryResult `protobuf:"bytes,2,opt,name=output,proto3" json:"output,omitempty"` + VdiffUuid string `protobuf:"bytes,3,opt,name=vdiff_uuid,json=vdiffUuid,proto3" json:"vdiff_uuid,omitempty"` +} + +func (x *VDiffResponse) Reset() { + *x = VDiffResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VDiffResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffResponse) ProtoMessage() {} + +func (x *VDiffResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[103] + 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 VDiffResponse.ProtoReflect.Descriptor instead. +func (*VDiffResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{103} +} + +func (x *VDiffResponse) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *VDiffResponse) GetOutput() *query.QueryResult { + if x != nil { + return x.Output + } + return nil +} + +func (x *VDiffResponse) GetVdiffUuid() string { + if x != nil { + return x.VdiffUuid + } + return "" +} + +// options that influence the tablet selected by the picker for streaming data from +type VDiffPickerOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TabletTypes string `protobuf:"bytes,1,opt,name=tablet_types,json=tabletTypes,proto3" json:"tablet_types,omitempty"` + SourceCell string `protobuf:"bytes,2,opt,name=source_cell,json=sourceCell,proto3" json:"source_cell,omitempty"` + TargetCell string `protobuf:"bytes,3,opt,name=target_cell,json=targetCell,proto3" json:"target_cell,omitempty"` +} + +func (x *VDiffPickerOptions) Reset() { + *x = VDiffPickerOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[104] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VDiffPickerOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffPickerOptions) ProtoMessage() {} + +func (x *VDiffPickerOptions) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[104] + 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 VDiffPickerOptions.ProtoReflect.Descriptor instead. +func (*VDiffPickerOptions) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{104} +} + +func (x *VDiffPickerOptions) GetTabletTypes() string { + if x != nil { + return x.TabletTypes + } + return "" +} + +func (x *VDiffPickerOptions) GetSourceCell() string { + if x != nil { + return x.SourceCell + } + return "" +} + +func (x *VDiffPickerOptions) GetTargetCell() string { + if x != nil { + return x.TargetCell + } + return "" +} + +// options that only influence how vdiff differences are reported +type VDiffReportOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + OnlyPks bool `protobuf:"varint,1,opt,name=only_pks,json=onlyPks,proto3" json:"only_pks,omitempty"` + DebugQuery bool `protobuf:"varint,2,opt,name=debug_query,json=debugQuery,proto3" json:"debug_query,omitempty"` + Format string `protobuf:"bytes,3,opt,name=format,proto3" json:"format,omitempty"` +} + +func (x *VDiffReportOptions) Reset() { + *x = VDiffReportOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[105] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VDiffReportOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffReportOptions) ProtoMessage() {} + +func (x *VDiffReportOptions) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[105] + 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 VDiffReportOptions.ProtoReflect.Descriptor instead. +func (*VDiffReportOptions) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{105} +} + +func (x *VDiffReportOptions) GetOnlyPks() bool { + if x != nil { + return x.OnlyPks + } + return false +} + +func (x *VDiffReportOptions) GetDebugQuery() bool { + if x != nil { + return x.DebugQuery + } + return false +} + +func (x *VDiffReportOptions) GetFormat() string { + if x != nil { + return x.Format + } + return "" +} + +type VDiffCoreOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Tables string `protobuf:"bytes,1,opt,name=tables,proto3" json:"tables,omitempty"` + AutoRetry bool `protobuf:"varint,2,opt,name=auto_retry,json=autoRetry,proto3" json:"auto_retry,omitempty"` + MaxRows int64 `protobuf:"varint,3,opt,name=max_rows,json=maxRows,proto3" json:"max_rows,omitempty"` + Checksum bool `protobuf:"varint,4,opt,name=checksum,proto3" json:"checksum,omitempty"` + SamplePct int64 `protobuf:"varint,5,opt,name=sample_pct,json=samplePct,proto3" json:"sample_pct,omitempty"` + TimeoutSeconds int64 `protobuf:"varint,6,opt,name=timeout_seconds,json=timeoutSeconds,proto3" json:"timeout_seconds,omitempty"` + MaxExtraRowsToCompare int64 `protobuf:"varint,7,opt,name=max_extra_rows_to_compare,json=maxExtraRowsToCompare,proto3" json:"max_extra_rows_to_compare,omitempty"` + UpdateTableStats bool `protobuf:"varint,8,opt,name=update_table_stats,json=updateTableStats,proto3" json:"update_table_stats,omitempty"` +} + +func (x *VDiffCoreOptions) Reset() { + *x = VDiffCoreOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[106] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VDiffCoreOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VDiffCoreOptions) ProtoMessage() {} + +func (x *VDiffCoreOptions) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[106] + 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 VDiffCoreOptions.ProtoReflect.Descriptor instead. +func (*VDiffCoreOptions) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{106} +} + +func (x *VDiffCoreOptions) GetTables() string { + if x != nil { + return x.Tables + } + return "" +} + +func (x *VDiffCoreOptions) GetAutoRetry() bool { + if x != nil { + return x.AutoRetry + } + return false +} + +func (x *VDiffCoreOptions) GetMaxRows() int64 { + if x != nil { + return x.MaxRows + } + return 0 +} + +func (x *VDiffCoreOptions) GetChecksum() bool { + if x != nil { + return x.Checksum + } + return false +} + +func (x *VDiffCoreOptions) GetSamplePct() int64 { + if x != nil { + return x.SamplePct + } + return 0 +} + +func (x *VDiffCoreOptions) GetTimeoutSeconds() int64 { + if x != nil { + return x.TimeoutSeconds + } + return 0 +} + +func (x *VDiffCoreOptions) GetMaxExtraRowsToCompare() int64 { + if x != nil { + return x.MaxExtraRowsToCompare + } + return 0 +} + +func (x *VDiffCoreOptions) GetUpdateTableStats() bool { + if x != nil { + return x.UpdateTableStats + } + return false +} + +type VDiffOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PickerOptions *VDiffPickerOptions `protobuf:"bytes,1,opt,name=picker_options,json=pickerOptions,proto3" json:"picker_options,omitempty"` + CoreOptions *VDiffCoreOptions `protobuf:"bytes,2,opt,name=core_options,json=coreOptions,proto3" json:"core_options,omitempty"` ReportOptions *VDiffReportOptions `protobuf:"bytes,3,opt,name=report_options,json=reportOptions,proto3" json:"report_options,omitempty"` } func (x *VDiffOptions) Reset() { *x = VDiffOptions{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[101] + mi := &file_tabletmanagerdata_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5134,7 +5617,7 @@ func (x *VDiffOptions) String() string { func (*VDiffOptions) ProtoMessage() {} func (x *VDiffOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[101] + mi := &file_tabletmanagerdata_proto_msgTypes[107] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5147,7 +5630,7 @@ func (x *VDiffOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffOptions.ProtoReflect.Descriptor instead. func (*VDiffOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{107} } func (x *VDiffOptions) GetPickerOptions() *VDiffPickerOptions { @@ -5171,34 +5654,36 @@ func (x *VDiffOptions) GetReportOptions() *VDiffReportOptions { return nil } -type UpdateVRWorkflowRequest struct { +type UpdateVReplicationWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` - Cells []string `protobuf:"bytes,2,rep,name=cells,proto3" json:"cells,omitempty"` - TabletTypes []string `protobuf:"bytes,3,rep,name=tablet_types,json=tabletTypes,proto3" json:"tablet_types,omitempty"` - OnDdl binlogdata.OnDDLAction `protobuf:"varint,4,opt,name=on_ddl,json=onDdl,proto3,enum=binlogdata.OnDDLAction" json:"on_ddl,omitempty"` + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` + Cells []string `protobuf:"bytes,2,rep,name=cells,proto3" json:"cells,omitempty"` + TabletTypes []topodata.TabletType `protobuf:"varint,3,rep,packed,name=tablet_types,json=tabletTypes,proto3,enum=topodata.TabletType" json:"tablet_types,omitempty"` + TabletSelectionPreference TabletSelectionPreference `protobuf:"varint,4,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` + OnDdl binlogdata.OnDDLAction `protobuf:"varint,5,opt,name=on_ddl,json=onDdl,proto3,enum=binlogdata.OnDDLAction" json:"on_ddl,omitempty"` + State binlogdata.VReplicationWorkflowState `protobuf:"varint,6,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` } -func (x *UpdateVRWorkflowRequest) Reset() { - *x = UpdateVRWorkflowRequest{} +func (x *UpdateVReplicationWorkflowRequest) Reset() { + *x = UpdateVReplicationWorkflowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[102] + mi := &file_tabletmanagerdata_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *UpdateVRWorkflowRequest) String() string { +func (x *UpdateVReplicationWorkflowRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*UpdateVRWorkflowRequest) ProtoMessage() {} +func (*UpdateVReplicationWorkflowRequest) ProtoMessage() {} -func (x *UpdateVRWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[102] +func (x *UpdateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[108] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5209,40 +5694,54 @@ func (x *UpdateVRWorkflowRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use UpdateVRWorkflowRequest.ProtoReflect.Descriptor instead. -func (*UpdateVRWorkflowRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{102} +// Deprecated: Use UpdateVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{108} } -func (x *UpdateVRWorkflowRequest) GetWorkflow() string { +func (x *UpdateVReplicationWorkflowRequest) GetWorkflow() string { if x != nil { return x.Workflow } return "" } -func (x *UpdateVRWorkflowRequest) GetCells() []string { +func (x *UpdateVReplicationWorkflowRequest) GetCells() []string { if x != nil { return x.Cells } return nil } -func (x *UpdateVRWorkflowRequest) GetTabletTypes() []string { +func (x *UpdateVReplicationWorkflowRequest) GetTabletTypes() []topodata.TabletType { if x != nil { return x.TabletTypes } return nil } -func (x *UpdateVRWorkflowRequest) GetOnDdl() binlogdata.OnDDLAction { +func (x *UpdateVReplicationWorkflowRequest) GetTabletSelectionPreference() TabletSelectionPreference { + if x != nil { + return x.TabletSelectionPreference + } + return TabletSelectionPreference_ANY +} + +func (x *UpdateVReplicationWorkflowRequest) GetOnDdl() binlogdata.OnDDLAction { if x != nil { return x.OnDdl } return binlogdata.OnDDLAction(0) } -type UpdateVRWorkflowResponse struct { +func (x *UpdateVReplicationWorkflowRequest) GetState() binlogdata.VReplicationWorkflowState { + if x != nil { + return x.State + } + return binlogdata.VReplicationWorkflowState(0) +} + +type UpdateVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -5250,23 +5749,23 @@ type UpdateVRWorkflowResponse struct { Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` } -func (x *UpdateVRWorkflowResponse) Reset() { - *x = UpdateVRWorkflowResponse{} +func (x *UpdateVReplicationWorkflowResponse) Reset() { + *x = UpdateVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[103] + mi := &file_tabletmanagerdata_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *UpdateVRWorkflowResponse) String() string { +func (x *UpdateVReplicationWorkflowResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*UpdateVRWorkflowResponse) ProtoMessage() {} +func (*UpdateVReplicationWorkflowResponse) ProtoMessage() {} -func (x *UpdateVRWorkflowResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[103] +func (x *UpdateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[109] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5277,12 +5776,12 @@ func (x *UpdateVRWorkflowResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use UpdateVRWorkflowResponse.ProtoReflect.Descriptor instead. -func (*UpdateVRWorkflowResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{103} +// Deprecated: Use UpdateVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{109} } -func (x *UpdateVRWorkflowResponse) GetResult() *query.QueryResult { +func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { if x != nil { return x.Result } @@ -5300,7 +5799,7 @@ type ResetSequencesRequest struct { func (x *ResetSequencesRequest) Reset() { *x = ResetSequencesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[104] + mi := &file_tabletmanagerdata_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5313,7 +5812,7 @@ func (x *ResetSequencesRequest) String() string { func (*ResetSequencesRequest) ProtoMessage() {} func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[104] + mi := &file_tabletmanagerdata_proto_msgTypes[110] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5326,7 +5825,7 @@ func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesRequest.ProtoReflect.Descriptor instead. func (*ResetSequencesRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{104} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{110} } func (x *ResetSequencesRequest) GetTables() []string { @@ -5345,7 +5844,7 @@ type ResetSequencesResponse struct { func (x *ResetSequencesResponse) Reset() { *x = ResetSequencesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[105] + mi := &file_tabletmanagerdata_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5358,7 +5857,7 @@ func (x *ResetSequencesResponse) String() string { func (*ResetSequencesResponse) ProtoMessage() {} func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[105] + mi := &file_tabletmanagerdata_proto_msgTypes[111] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5371,7 +5870,158 @@ func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesResponse.ProtoReflect.Descriptor instead. func (*ResetSequencesResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{105} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{111} +} + +type ReadVReplicationWorkflowResponse_Stream struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int32 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Bls *binlogdata.BinlogSource `protobuf:"bytes,2,opt,name=bls,proto3" json:"bls,omitempty"` + Pos string `protobuf:"bytes,3,opt,name=pos,proto3" json:"pos,omitempty"` + StopPos string `protobuf:"bytes,4,opt,name=stop_pos,json=stopPos,proto3" json:"stop_pos,omitempty"` + MaxTps int64 `protobuf:"varint,5,opt,name=max_tps,json=maxTps,proto3" json:"max_tps,omitempty"` + MaxReplicationLag int64 `protobuf:"varint,6,opt,name=max_replication_lag,json=maxReplicationLag,proto3" json:"max_replication_lag,omitempty"` + TimeUpdated *vttime.Time `protobuf:"bytes,7,opt,name=time_updated,json=timeUpdated,proto3" json:"time_updated,omitempty"` + TransactionTimestamp *vttime.Time `protobuf:"bytes,8,opt,name=transaction_timestamp,json=transactionTimestamp,proto3" json:"transaction_timestamp,omitempty"` + State binlogdata.VReplicationWorkflowState `protobuf:"varint,9,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` + Message string `protobuf:"bytes,10,opt,name=message,proto3" json:"message,omitempty"` + RowsCopied int64 `protobuf:"varint,11,opt,name=rows_copied,json=rowsCopied,proto3" json:"rows_copied,omitempty"` + TimeHeartbeat *vttime.Time `protobuf:"bytes,12,opt,name=time_heartbeat,json=timeHeartbeat,proto3" json:"time_heartbeat,omitempty"` + TimeThrottled *vttime.Time `protobuf:"bytes,13,opt,name=time_throttled,json=timeThrottled,proto3" json:"time_throttled,omitempty"` + ComponentThrottled string `protobuf:"bytes,14,opt,name=component_throttled,json=componentThrottled,proto3" json:"component_throttled,omitempty"` +} + +func (x *ReadVReplicationWorkflowResponse_Stream) Reset() { + *x = ReadVReplicationWorkflowResponse_Stream{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReadVReplicationWorkflowResponse_Stream) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadVReplicationWorkflowResponse_Stream) ProtoMessage() {} + +func (x *ReadVReplicationWorkflowResponse_Stream) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[115] + 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 ReadVReplicationWorkflowResponse_Stream.ProtoReflect.Descriptor instead. +func (*ReadVReplicationWorkflowResponse_Stream) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101, 0} +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetId() int32 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetBls() *binlogdata.BinlogSource { + if x != nil { + return x.Bls + } + return nil +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetPos() string { + if x != nil { + return x.Pos + } + return "" +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetStopPos() string { + if x != nil { + return x.StopPos + } + return "" +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetMaxTps() int64 { + if x != nil { + return x.MaxTps + } + return 0 +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetMaxReplicationLag() int64 { + if x != nil { + return x.MaxReplicationLag + } + return 0 +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetTimeUpdated() *vttime.Time { + if x != nil { + return x.TimeUpdated + } + return nil +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetTransactionTimestamp() *vttime.Time { + if x != nil { + return x.TransactionTimestamp + } + return nil +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetState() binlogdata.VReplicationWorkflowState { + if x != nil { + return x.State + } + return binlogdata.VReplicationWorkflowState(0) +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetRowsCopied() int64 { + if x != nil { + return x.RowsCopied + } + return 0 +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetTimeHeartbeat() *vttime.Time { + if x != nil { + return x.TimeHeartbeat + } + return nil +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetTimeThrottled() *vttime.Time { + if x != nil { + return x.TimeThrottled + } + return nil +} + +func (x *ReadVReplicationWorkflowResponse_Stream) GetComponentThrottled() string { + if x != nil { + return x.ComponentThrottled + } + return "" } var File_tabletmanagerdata_proto protoreflect.FileDescriptor @@ -5848,99 +6498,245 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 0x46, 0x72, 0x6f, 0x6d, 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, 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, 0x68, 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, 0x22, 0xb0, 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, 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, 0x9e, 0x01, 0x0a, 0x17, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 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, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x06, - 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x04, 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, 0x22, 0x46, 0x0a, 0x18, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xd4, 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, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x3d, 0x0a, 0x0d, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x03, 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, 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, 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, 0x49, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 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, 0x07, 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, 0x08, + 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, 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, 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, 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, + 0x73, 0x75, 0x6c, 0x74, 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, 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, 0x68, 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, 0x22, 0xb0, 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, 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, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 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, 0xe9, 0x02, + 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, 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, 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, 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 ( @@ -5955,181 +6751,215 @@ func file_tabletmanagerdata_proto_rawDescGZIP() []byte { return file_tabletmanagerdata_proto_rawDescData } -var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 109) +var file_tabletmanagerdata_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 116) var file_tabletmanagerdata_proto_goTypes = []interface{}{ - (*TableDefinition)(nil), // 0: tabletmanagerdata.TableDefinition - (*SchemaDefinition)(nil), // 1: tabletmanagerdata.SchemaDefinition - (*SchemaChangeResult)(nil), // 2: tabletmanagerdata.SchemaChangeResult - (*UserPermission)(nil), // 3: tabletmanagerdata.UserPermission - (*DbPermission)(nil), // 4: tabletmanagerdata.DbPermission - (*Permissions)(nil), // 5: tabletmanagerdata.Permissions - (*PingRequest)(nil), // 6: tabletmanagerdata.PingRequest - (*PingResponse)(nil), // 7: tabletmanagerdata.PingResponse - (*SleepRequest)(nil), // 8: tabletmanagerdata.SleepRequest - (*SleepResponse)(nil), // 9: tabletmanagerdata.SleepResponse - (*ExecuteHookRequest)(nil), // 10: tabletmanagerdata.ExecuteHookRequest - (*ExecuteHookResponse)(nil), // 11: tabletmanagerdata.ExecuteHookResponse - (*GetSchemaRequest)(nil), // 12: tabletmanagerdata.GetSchemaRequest - (*GetSchemaResponse)(nil), // 13: tabletmanagerdata.GetSchemaResponse - (*GetPermissionsRequest)(nil), // 14: tabletmanagerdata.GetPermissionsRequest - (*GetPermissionsResponse)(nil), // 15: tabletmanagerdata.GetPermissionsResponse - (*SetReadOnlyRequest)(nil), // 16: tabletmanagerdata.SetReadOnlyRequest - (*SetReadOnlyResponse)(nil), // 17: tabletmanagerdata.SetReadOnlyResponse - (*SetReadWriteRequest)(nil), // 18: tabletmanagerdata.SetReadWriteRequest - (*SetReadWriteResponse)(nil), // 19: tabletmanagerdata.SetReadWriteResponse - (*ChangeTypeRequest)(nil), // 20: tabletmanagerdata.ChangeTypeRequest - (*ChangeTypeResponse)(nil), // 21: tabletmanagerdata.ChangeTypeResponse - (*RefreshStateRequest)(nil), // 22: tabletmanagerdata.RefreshStateRequest - (*RefreshStateResponse)(nil), // 23: tabletmanagerdata.RefreshStateResponse - (*RunHealthCheckRequest)(nil), // 24: tabletmanagerdata.RunHealthCheckRequest - (*RunHealthCheckResponse)(nil), // 25: tabletmanagerdata.RunHealthCheckResponse - (*ReloadSchemaRequest)(nil), // 26: tabletmanagerdata.ReloadSchemaRequest - (*ReloadSchemaResponse)(nil), // 27: tabletmanagerdata.ReloadSchemaResponse - (*PreflightSchemaRequest)(nil), // 28: tabletmanagerdata.PreflightSchemaRequest - (*PreflightSchemaResponse)(nil), // 29: tabletmanagerdata.PreflightSchemaResponse - (*ApplySchemaRequest)(nil), // 30: tabletmanagerdata.ApplySchemaRequest - (*ApplySchemaResponse)(nil), // 31: tabletmanagerdata.ApplySchemaResponse - (*LockTablesRequest)(nil), // 32: tabletmanagerdata.LockTablesRequest - (*LockTablesResponse)(nil), // 33: tabletmanagerdata.LockTablesResponse - (*UnlockTablesRequest)(nil), // 34: tabletmanagerdata.UnlockTablesRequest - (*UnlockTablesResponse)(nil), // 35: tabletmanagerdata.UnlockTablesResponse - (*ExecuteQueryRequest)(nil), // 36: tabletmanagerdata.ExecuteQueryRequest - (*ExecuteQueryResponse)(nil), // 37: tabletmanagerdata.ExecuteQueryResponse - (*ExecuteFetchAsDbaRequest)(nil), // 38: tabletmanagerdata.ExecuteFetchAsDbaRequest - (*ExecuteFetchAsDbaResponse)(nil), // 39: tabletmanagerdata.ExecuteFetchAsDbaResponse - (*ExecuteFetchAsAllPrivsRequest)(nil), // 40: tabletmanagerdata.ExecuteFetchAsAllPrivsRequest - (*ExecuteFetchAsAllPrivsResponse)(nil), // 41: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse - (*ExecuteFetchAsAppRequest)(nil), // 42: tabletmanagerdata.ExecuteFetchAsAppRequest - (*ExecuteFetchAsAppResponse)(nil), // 43: tabletmanagerdata.ExecuteFetchAsAppResponse - (*ReplicationStatusRequest)(nil), // 44: tabletmanagerdata.ReplicationStatusRequest - (*ReplicationStatusResponse)(nil), // 45: tabletmanagerdata.ReplicationStatusResponse - (*PrimaryStatusRequest)(nil), // 46: tabletmanagerdata.PrimaryStatusRequest - (*PrimaryStatusResponse)(nil), // 47: tabletmanagerdata.PrimaryStatusResponse - (*PrimaryPositionRequest)(nil), // 48: tabletmanagerdata.PrimaryPositionRequest - (*PrimaryPositionResponse)(nil), // 49: tabletmanagerdata.PrimaryPositionResponse - (*WaitForPositionRequest)(nil), // 50: tabletmanagerdata.WaitForPositionRequest - (*WaitForPositionResponse)(nil), // 51: tabletmanagerdata.WaitForPositionResponse - (*StopReplicationRequest)(nil), // 52: tabletmanagerdata.StopReplicationRequest - (*StopReplicationResponse)(nil), // 53: tabletmanagerdata.StopReplicationResponse - (*StopReplicationMinimumRequest)(nil), // 54: tabletmanagerdata.StopReplicationMinimumRequest - (*StopReplicationMinimumResponse)(nil), // 55: tabletmanagerdata.StopReplicationMinimumResponse - (*StartReplicationRequest)(nil), // 56: tabletmanagerdata.StartReplicationRequest - (*StartReplicationResponse)(nil), // 57: tabletmanagerdata.StartReplicationResponse - (*StartReplicationUntilAfterRequest)(nil), // 58: tabletmanagerdata.StartReplicationUntilAfterRequest - (*StartReplicationUntilAfterResponse)(nil), // 59: tabletmanagerdata.StartReplicationUntilAfterResponse - (*GetReplicasRequest)(nil), // 60: tabletmanagerdata.GetReplicasRequest - (*GetReplicasResponse)(nil), // 61: tabletmanagerdata.GetReplicasResponse - (*ResetReplicationRequest)(nil), // 62: tabletmanagerdata.ResetReplicationRequest - (*ResetReplicationResponse)(nil), // 63: tabletmanagerdata.ResetReplicationResponse - (*VReplicationExecRequest)(nil), // 64: tabletmanagerdata.VReplicationExecRequest - (*VReplicationExecResponse)(nil), // 65: tabletmanagerdata.VReplicationExecResponse - (*VReplicationWaitForPosRequest)(nil), // 66: tabletmanagerdata.VReplicationWaitForPosRequest - (*VReplicationWaitForPosResponse)(nil), // 67: tabletmanagerdata.VReplicationWaitForPosResponse - (*InitPrimaryRequest)(nil), // 68: tabletmanagerdata.InitPrimaryRequest - (*InitPrimaryResponse)(nil), // 69: tabletmanagerdata.InitPrimaryResponse - (*PopulateReparentJournalRequest)(nil), // 70: tabletmanagerdata.PopulateReparentJournalRequest - (*PopulateReparentJournalResponse)(nil), // 71: tabletmanagerdata.PopulateReparentJournalResponse - (*InitReplicaRequest)(nil), // 72: tabletmanagerdata.InitReplicaRequest - (*InitReplicaResponse)(nil), // 73: tabletmanagerdata.InitReplicaResponse - (*DemotePrimaryRequest)(nil), // 74: tabletmanagerdata.DemotePrimaryRequest - (*DemotePrimaryResponse)(nil), // 75: tabletmanagerdata.DemotePrimaryResponse - (*UndoDemotePrimaryRequest)(nil), // 76: tabletmanagerdata.UndoDemotePrimaryRequest - (*UndoDemotePrimaryResponse)(nil), // 77: tabletmanagerdata.UndoDemotePrimaryResponse - (*ReplicaWasPromotedRequest)(nil), // 78: tabletmanagerdata.ReplicaWasPromotedRequest - (*ReplicaWasPromotedResponse)(nil), // 79: tabletmanagerdata.ReplicaWasPromotedResponse - (*ResetReplicationParametersRequest)(nil), // 80: tabletmanagerdata.ResetReplicationParametersRequest - (*ResetReplicationParametersResponse)(nil), // 81: tabletmanagerdata.ResetReplicationParametersResponse - (*FullStatusRequest)(nil), // 82: tabletmanagerdata.FullStatusRequest - (*FullStatusResponse)(nil), // 83: tabletmanagerdata.FullStatusResponse - (*SetReplicationSourceRequest)(nil), // 84: tabletmanagerdata.SetReplicationSourceRequest - (*SetReplicationSourceResponse)(nil), // 85: tabletmanagerdata.SetReplicationSourceResponse - (*ReplicaWasRestartedRequest)(nil), // 86: tabletmanagerdata.ReplicaWasRestartedRequest - (*ReplicaWasRestartedResponse)(nil), // 87: tabletmanagerdata.ReplicaWasRestartedResponse - (*StopReplicationAndGetStatusRequest)(nil), // 88: tabletmanagerdata.StopReplicationAndGetStatusRequest - (*StopReplicationAndGetStatusResponse)(nil), // 89: tabletmanagerdata.StopReplicationAndGetStatusResponse - (*PromoteReplicaRequest)(nil), // 90: tabletmanagerdata.PromoteReplicaRequest - (*PromoteReplicaResponse)(nil), // 91: tabletmanagerdata.PromoteReplicaResponse - (*BackupRequest)(nil), // 92: tabletmanagerdata.BackupRequest - (*BackupResponse)(nil), // 93: tabletmanagerdata.BackupResponse - (*RestoreFromBackupRequest)(nil), // 94: tabletmanagerdata.RestoreFromBackupRequest - (*RestoreFromBackupResponse)(nil), // 95: tabletmanagerdata.RestoreFromBackupResponse - (*VDiffRequest)(nil), // 96: tabletmanagerdata.VDiffRequest - (*VDiffResponse)(nil), // 97: tabletmanagerdata.VDiffResponse - (*VDiffPickerOptions)(nil), // 98: tabletmanagerdata.VDiffPickerOptions - (*VDiffReportOptions)(nil), // 99: tabletmanagerdata.VDiffReportOptions - (*VDiffCoreOptions)(nil), // 100: tabletmanagerdata.VDiffCoreOptions - (*VDiffOptions)(nil), // 101: tabletmanagerdata.VDiffOptions - (*UpdateVRWorkflowRequest)(nil), // 102: tabletmanagerdata.UpdateVRWorkflowRequest - (*UpdateVRWorkflowResponse)(nil), // 103: tabletmanagerdata.UpdateVRWorkflowResponse - (*ResetSequencesRequest)(nil), // 104: tabletmanagerdata.ResetSequencesRequest - (*ResetSequencesResponse)(nil), // 105: tabletmanagerdata.ResetSequencesResponse - nil, // 106: tabletmanagerdata.UserPermission.PrivilegesEntry - nil, // 107: tabletmanagerdata.DbPermission.PrivilegesEntry - nil, // 108: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry - (*query.Field)(nil), // 109: query.Field - (topodata.TabletType)(0), // 110: topodata.TabletType - (*vtrpc.CallerID)(nil), // 111: vtrpc.CallerID - (*query.QueryResult)(nil), // 112: query.QueryResult - (*replicationdata.Status)(nil), // 113: replicationdata.Status - (*replicationdata.PrimaryStatus)(nil), // 114: replicationdata.PrimaryStatus - (*topodata.TabletAlias)(nil), // 115: topodata.TabletAlias - (*replicationdata.FullStatus)(nil), // 116: replicationdata.FullStatus - (replicationdata.StopReplicationMode)(0), // 117: replicationdata.StopReplicationMode - (*replicationdata.StopReplicationStatus)(nil), // 118: replicationdata.StopReplicationStatus - (*logutil.Event)(nil), // 119: logutil.Event - (*vttime.Time)(nil), // 120: vttime.Time - (binlogdata.OnDDLAction)(0), // 121: binlogdata.OnDDLAction + (TabletSelectionPreference)(0), // 0: tabletmanagerdata.TabletSelectionPreference + (*TableDefinition)(nil), // 1: tabletmanagerdata.TableDefinition + (*SchemaDefinition)(nil), // 2: tabletmanagerdata.SchemaDefinition + (*SchemaChangeResult)(nil), // 3: tabletmanagerdata.SchemaChangeResult + (*UserPermission)(nil), // 4: tabletmanagerdata.UserPermission + (*DbPermission)(nil), // 5: tabletmanagerdata.DbPermission + (*Permissions)(nil), // 6: tabletmanagerdata.Permissions + (*PingRequest)(nil), // 7: tabletmanagerdata.PingRequest + (*PingResponse)(nil), // 8: tabletmanagerdata.PingResponse + (*SleepRequest)(nil), // 9: tabletmanagerdata.SleepRequest + (*SleepResponse)(nil), // 10: tabletmanagerdata.SleepResponse + (*ExecuteHookRequest)(nil), // 11: tabletmanagerdata.ExecuteHookRequest + (*ExecuteHookResponse)(nil), // 12: tabletmanagerdata.ExecuteHookResponse + (*GetSchemaRequest)(nil), // 13: tabletmanagerdata.GetSchemaRequest + (*GetSchemaResponse)(nil), // 14: tabletmanagerdata.GetSchemaResponse + (*GetPermissionsRequest)(nil), // 15: tabletmanagerdata.GetPermissionsRequest + (*GetPermissionsResponse)(nil), // 16: tabletmanagerdata.GetPermissionsResponse + (*SetReadOnlyRequest)(nil), // 17: tabletmanagerdata.SetReadOnlyRequest + (*SetReadOnlyResponse)(nil), // 18: tabletmanagerdata.SetReadOnlyResponse + (*SetReadWriteRequest)(nil), // 19: tabletmanagerdata.SetReadWriteRequest + (*SetReadWriteResponse)(nil), // 20: tabletmanagerdata.SetReadWriteResponse + (*ChangeTypeRequest)(nil), // 21: tabletmanagerdata.ChangeTypeRequest + (*ChangeTypeResponse)(nil), // 22: tabletmanagerdata.ChangeTypeResponse + (*RefreshStateRequest)(nil), // 23: tabletmanagerdata.RefreshStateRequest + (*RefreshStateResponse)(nil), // 24: tabletmanagerdata.RefreshStateResponse + (*RunHealthCheckRequest)(nil), // 25: tabletmanagerdata.RunHealthCheckRequest + (*RunHealthCheckResponse)(nil), // 26: tabletmanagerdata.RunHealthCheckResponse + (*ReloadSchemaRequest)(nil), // 27: tabletmanagerdata.ReloadSchemaRequest + (*ReloadSchemaResponse)(nil), // 28: tabletmanagerdata.ReloadSchemaResponse + (*PreflightSchemaRequest)(nil), // 29: tabletmanagerdata.PreflightSchemaRequest + (*PreflightSchemaResponse)(nil), // 30: tabletmanagerdata.PreflightSchemaResponse + (*ApplySchemaRequest)(nil), // 31: tabletmanagerdata.ApplySchemaRequest + (*ApplySchemaResponse)(nil), // 32: tabletmanagerdata.ApplySchemaResponse + (*LockTablesRequest)(nil), // 33: tabletmanagerdata.LockTablesRequest + (*LockTablesResponse)(nil), // 34: tabletmanagerdata.LockTablesResponse + (*UnlockTablesRequest)(nil), // 35: tabletmanagerdata.UnlockTablesRequest + (*UnlockTablesResponse)(nil), // 36: tabletmanagerdata.UnlockTablesResponse + (*ExecuteQueryRequest)(nil), // 37: tabletmanagerdata.ExecuteQueryRequest + (*ExecuteQueryResponse)(nil), // 38: tabletmanagerdata.ExecuteQueryResponse + (*ExecuteFetchAsDbaRequest)(nil), // 39: tabletmanagerdata.ExecuteFetchAsDbaRequest + (*ExecuteFetchAsDbaResponse)(nil), // 40: tabletmanagerdata.ExecuteFetchAsDbaResponse + (*ExecuteFetchAsAllPrivsRequest)(nil), // 41: tabletmanagerdata.ExecuteFetchAsAllPrivsRequest + (*ExecuteFetchAsAllPrivsResponse)(nil), // 42: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse + (*ExecuteFetchAsAppRequest)(nil), // 43: tabletmanagerdata.ExecuteFetchAsAppRequest + (*ExecuteFetchAsAppResponse)(nil), // 44: tabletmanagerdata.ExecuteFetchAsAppResponse + (*ReplicationStatusRequest)(nil), // 45: tabletmanagerdata.ReplicationStatusRequest + (*ReplicationStatusResponse)(nil), // 46: tabletmanagerdata.ReplicationStatusResponse + (*PrimaryStatusRequest)(nil), // 47: tabletmanagerdata.PrimaryStatusRequest + (*PrimaryStatusResponse)(nil), // 48: tabletmanagerdata.PrimaryStatusResponse + (*PrimaryPositionRequest)(nil), // 49: tabletmanagerdata.PrimaryPositionRequest + (*PrimaryPositionResponse)(nil), // 50: tabletmanagerdata.PrimaryPositionResponse + (*WaitForPositionRequest)(nil), // 51: tabletmanagerdata.WaitForPositionRequest + (*WaitForPositionResponse)(nil), // 52: tabletmanagerdata.WaitForPositionResponse + (*StopReplicationRequest)(nil), // 53: tabletmanagerdata.StopReplicationRequest + (*StopReplicationResponse)(nil), // 54: tabletmanagerdata.StopReplicationResponse + (*StopReplicationMinimumRequest)(nil), // 55: tabletmanagerdata.StopReplicationMinimumRequest + (*StopReplicationMinimumResponse)(nil), // 56: tabletmanagerdata.StopReplicationMinimumResponse + (*StartReplicationRequest)(nil), // 57: tabletmanagerdata.StartReplicationRequest + (*StartReplicationResponse)(nil), // 58: tabletmanagerdata.StartReplicationResponse + (*StartReplicationUntilAfterRequest)(nil), // 59: tabletmanagerdata.StartReplicationUntilAfterRequest + (*StartReplicationUntilAfterResponse)(nil), // 60: tabletmanagerdata.StartReplicationUntilAfterResponse + (*GetReplicasRequest)(nil), // 61: tabletmanagerdata.GetReplicasRequest + (*GetReplicasResponse)(nil), // 62: tabletmanagerdata.GetReplicasResponse + (*ResetReplicationRequest)(nil), // 63: tabletmanagerdata.ResetReplicationRequest + (*ResetReplicationResponse)(nil), // 64: tabletmanagerdata.ResetReplicationResponse + (*VReplicationExecRequest)(nil), // 65: tabletmanagerdata.VReplicationExecRequest + (*VReplicationExecResponse)(nil), // 66: tabletmanagerdata.VReplicationExecResponse + (*VReplicationWaitForPosRequest)(nil), // 67: tabletmanagerdata.VReplicationWaitForPosRequest + (*VReplicationWaitForPosResponse)(nil), // 68: tabletmanagerdata.VReplicationWaitForPosResponse + (*InitPrimaryRequest)(nil), // 69: tabletmanagerdata.InitPrimaryRequest + (*InitPrimaryResponse)(nil), // 70: tabletmanagerdata.InitPrimaryResponse + (*PopulateReparentJournalRequest)(nil), // 71: tabletmanagerdata.PopulateReparentJournalRequest + (*PopulateReparentJournalResponse)(nil), // 72: tabletmanagerdata.PopulateReparentJournalResponse + (*InitReplicaRequest)(nil), // 73: tabletmanagerdata.InitReplicaRequest + (*InitReplicaResponse)(nil), // 74: tabletmanagerdata.InitReplicaResponse + (*DemotePrimaryRequest)(nil), // 75: tabletmanagerdata.DemotePrimaryRequest + (*DemotePrimaryResponse)(nil), // 76: tabletmanagerdata.DemotePrimaryResponse + (*UndoDemotePrimaryRequest)(nil), // 77: tabletmanagerdata.UndoDemotePrimaryRequest + (*UndoDemotePrimaryResponse)(nil), // 78: tabletmanagerdata.UndoDemotePrimaryResponse + (*ReplicaWasPromotedRequest)(nil), // 79: tabletmanagerdata.ReplicaWasPromotedRequest + (*ReplicaWasPromotedResponse)(nil), // 80: tabletmanagerdata.ReplicaWasPromotedResponse + (*ResetReplicationParametersRequest)(nil), // 81: tabletmanagerdata.ResetReplicationParametersRequest + (*ResetReplicationParametersResponse)(nil), // 82: tabletmanagerdata.ResetReplicationParametersResponse + (*FullStatusRequest)(nil), // 83: tabletmanagerdata.FullStatusRequest + (*FullStatusResponse)(nil), // 84: tabletmanagerdata.FullStatusResponse + (*SetReplicationSourceRequest)(nil), // 85: tabletmanagerdata.SetReplicationSourceRequest + (*SetReplicationSourceResponse)(nil), // 86: tabletmanagerdata.SetReplicationSourceResponse + (*ReplicaWasRestartedRequest)(nil), // 87: tabletmanagerdata.ReplicaWasRestartedRequest + (*ReplicaWasRestartedResponse)(nil), // 88: tabletmanagerdata.ReplicaWasRestartedResponse + (*StopReplicationAndGetStatusRequest)(nil), // 89: tabletmanagerdata.StopReplicationAndGetStatusRequest + (*StopReplicationAndGetStatusResponse)(nil), // 90: tabletmanagerdata.StopReplicationAndGetStatusResponse + (*PromoteReplicaRequest)(nil), // 91: tabletmanagerdata.PromoteReplicaRequest + (*PromoteReplicaResponse)(nil), // 92: tabletmanagerdata.PromoteReplicaResponse + (*BackupRequest)(nil), // 93: tabletmanagerdata.BackupRequest + (*BackupResponse)(nil), // 94: tabletmanagerdata.BackupResponse + (*RestoreFromBackupRequest)(nil), // 95: tabletmanagerdata.RestoreFromBackupRequest + (*RestoreFromBackupResponse)(nil), // 96: tabletmanagerdata.RestoreFromBackupResponse + (*CreateVReplicationWorkflowRequest)(nil), // 97: tabletmanagerdata.CreateVReplicationWorkflowRequest + (*CreateVReplicationWorkflowResponse)(nil), // 98: tabletmanagerdata.CreateVReplicationWorkflowResponse + (*DeleteVReplicationWorkflowRequest)(nil), // 99: tabletmanagerdata.DeleteVReplicationWorkflowRequest + (*DeleteVReplicationWorkflowResponse)(nil), // 100: tabletmanagerdata.DeleteVReplicationWorkflowResponse + (*ReadVReplicationWorkflowRequest)(nil), // 101: tabletmanagerdata.ReadVReplicationWorkflowRequest + (*ReadVReplicationWorkflowResponse)(nil), // 102: tabletmanagerdata.ReadVReplicationWorkflowResponse + (*VDiffRequest)(nil), // 103: tabletmanagerdata.VDiffRequest + (*VDiffResponse)(nil), // 104: tabletmanagerdata.VDiffResponse + (*VDiffPickerOptions)(nil), // 105: tabletmanagerdata.VDiffPickerOptions + (*VDiffReportOptions)(nil), // 106: tabletmanagerdata.VDiffReportOptions + (*VDiffCoreOptions)(nil), // 107: tabletmanagerdata.VDiffCoreOptions + (*VDiffOptions)(nil), // 108: tabletmanagerdata.VDiffOptions + (*UpdateVReplicationWorkflowRequest)(nil), // 109: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*UpdateVReplicationWorkflowResponse)(nil), // 110: tabletmanagerdata.UpdateVReplicationWorkflowResponse + (*ResetSequencesRequest)(nil), // 111: tabletmanagerdata.ResetSequencesRequest + (*ResetSequencesResponse)(nil), // 112: tabletmanagerdata.ResetSequencesResponse + nil, // 113: tabletmanagerdata.UserPermission.PrivilegesEntry + nil, // 114: tabletmanagerdata.DbPermission.PrivilegesEntry + nil, // 115: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + (*ReadVReplicationWorkflowResponse_Stream)(nil), // 116: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + (*query.Field)(nil), // 117: query.Field + (topodata.TabletType)(0), // 118: topodata.TabletType + (*vtrpc.CallerID)(nil), // 119: vtrpc.CallerID + (*query.QueryResult)(nil), // 120: query.QueryResult + (*replicationdata.Status)(nil), // 121: replicationdata.Status + (*replicationdata.PrimaryStatus)(nil), // 122: replicationdata.PrimaryStatus + (*topodata.TabletAlias)(nil), // 123: topodata.TabletAlias + (*replicationdata.FullStatus)(nil), // 124: replicationdata.FullStatus + (replicationdata.StopReplicationMode)(0), // 125: replicationdata.StopReplicationMode + (*replicationdata.StopReplicationStatus)(nil), // 126: replicationdata.StopReplicationStatus + (*logutil.Event)(nil), // 127: logutil.Event + (*vttime.Time)(nil), // 128: vttime.Time + (*binlogdata.BinlogSource)(nil), // 129: binlogdata.BinlogSource + (binlogdata.VReplicationWorkflowType)(0), // 130: binlogdata.VReplicationWorkflowType + (binlogdata.VReplicationWorkflowSubType)(0), // 131: binlogdata.VReplicationWorkflowSubType + (binlogdata.OnDDLAction)(0), // 132: binlogdata.OnDDLAction + (binlogdata.VReplicationWorkflowState)(0), // 133: binlogdata.VReplicationWorkflowState } var file_tabletmanagerdata_proto_depIdxs = []int32{ - 109, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field - 0, // 1: tabletmanagerdata.SchemaDefinition.table_definitions:type_name -> tabletmanagerdata.TableDefinition - 1, // 2: tabletmanagerdata.SchemaChangeResult.before_schema:type_name -> tabletmanagerdata.SchemaDefinition - 1, // 3: tabletmanagerdata.SchemaChangeResult.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 106, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry - 107, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry - 3, // 6: tabletmanagerdata.Permissions.user_permissions:type_name -> tabletmanagerdata.UserPermission - 4, // 7: tabletmanagerdata.Permissions.db_permissions:type_name -> tabletmanagerdata.DbPermission - 108, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry - 1, // 9: tabletmanagerdata.GetSchemaResponse.schema_definition:type_name -> tabletmanagerdata.SchemaDefinition - 5, // 10: tabletmanagerdata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions - 110, // 11: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType - 2, // 12: tabletmanagerdata.PreflightSchemaResponse.change_results:type_name -> tabletmanagerdata.SchemaChangeResult - 1, // 13: tabletmanagerdata.ApplySchemaRequest.before_schema:type_name -> tabletmanagerdata.SchemaDefinition - 1, // 14: tabletmanagerdata.ApplySchemaRequest.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 1, // 15: tabletmanagerdata.ApplySchemaResponse.before_schema:type_name -> tabletmanagerdata.SchemaDefinition - 1, // 16: tabletmanagerdata.ApplySchemaResponse.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 111, // 17: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID - 112, // 18: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult - 112, // 19: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult - 112, // 20: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult - 112, // 21: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult - 113, // 22: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status - 114, // 23: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus - 112, // 24: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult - 115, // 25: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias - 115, // 26: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias - 114, // 27: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus - 116, // 28: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus - 115, // 29: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias - 115, // 30: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias - 117, // 31: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode - 118, // 32: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus - 119, // 33: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event - 120, // 34: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time - 120, // 35: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time - 119, // 36: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event - 101, // 37: tabletmanagerdata.VDiffRequest.options:type_name -> tabletmanagerdata.VDiffOptions - 112, // 38: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult - 98, // 39: tabletmanagerdata.VDiffOptions.picker_options:type_name -> tabletmanagerdata.VDiffPickerOptions - 100, // 40: tabletmanagerdata.VDiffOptions.core_options:type_name -> tabletmanagerdata.VDiffCoreOptions - 99, // 41: tabletmanagerdata.VDiffOptions.report_options:type_name -> tabletmanagerdata.VDiffReportOptions - 121, // 42: tabletmanagerdata.UpdateVRWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction - 112, // 43: tabletmanagerdata.UpdateVRWorkflowResponse.result:type_name -> query.QueryResult - 44, // [44:44] is the sub-list for method output_type - 44, // [44:44] is the sub-list for method input_type - 44, // [44:44] is the sub-list for extension type_name - 44, // [44:44] is the sub-list for extension extendee - 0, // [0:44] is the sub-list for field type_name + 117, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field + 1, // 1: tabletmanagerdata.SchemaDefinition.table_definitions:type_name -> tabletmanagerdata.TableDefinition + 2, // 2: tabletmanagerdata.SchemaChangeResult.before_schema:type_name -> tabletmanagerdata.SchemaDefinition + 2, // 3: tabletmanagerdata.SchemaChangeResult.after_schema:type_name -> tabletmanagerdata.SchemaDefinition + 113, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry + 114, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry + 4, // 6: tabletmanagerdata.Permissions.user_permissions:type_name -> tabletmanagerdata.UserPermission + 5, // 7: tabletmanagerdata.Permissions.db_permissions:type_name -> tabletmanagerdata.DbPermission + 115, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + 2, // 9: tabletmanagerdata.GetSchemaResponse.schema_definition:type_name -> tabletmanagerdata.SchemaDefinition + 6, // 10: tabletmanagerdata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions + 118, // 11: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType + 3, // 12: tabletmanagerdata.PreflightSchemaResponse.change_results:type_name -> tabletmanagerdata.SchemaChangeResult + 2, // 13: tabletmanagerdata.ApplySchemaRequest.before_schema:type_name -> tabletmanagerdata.SchemaDefinition + 2, // 14: tabletmanagerdata.ApplySchemaRequest.after_schema:type_name -> tabletmanagerdata.SchemaDefinition + 2, // 15: tabletmanagerdata.ApplySchemaResponse.before_schema:type_name -> tabletmanagerdata.SchemaDefinition + 2, // 16: tabletmanagerdata.ApplySchemaResponse.after_schema:type_name -> tabletmanagerdata.SchemaDefinition + 119, // 17: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID + 120, // 18: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult + 120, // 19: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult + 120, // 20: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult + 120, // 21: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult + 121, // 22: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status + 122, // 23: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus + 120, // 24: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult + 123, // 25: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias + 123, // 26: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias + 122, // 27: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus + 124, // 28: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus + 123, // 29: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias + 123, // 30: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias + 125, // 31: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode + 126, // 32: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus + 127, // 33: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event + 128, // 34: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time + 128, // 35: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time + 127, // 36: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event + 129, // 37: tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source:type_name -> binlogdata.BinlogSource + 118, // 38: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 0, // 39: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 130, // 40: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 131, // 41: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 120, // 42: tabletmanagerdata.CreateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 120, // 43: tabletmanagerdata.DeleteVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 118, // 44: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types:type_name -> topodata.TabletType + 0, // 45: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 130, // 46: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 131, // 47: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 116, // 48: tabletmanagerdata.ReadVReplicationWorkflowResponse.streams:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + 108, // 49: tabletmanagerdata.VDiffRequest.options:type_name -> tabletmanagerdata.VDiffOptions + 120, // 50: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult + 105, // 51: tabletmanagerdata.VDiffOptions.picker_options:type_name -> tabletmanagerdata.VDiffPickerOptions + 107, // 52: tabletmanagerdata.VDiffOptions.core_options:type_name -> tabletmanagerdata.VDiffCoreOptions + 106, // 53: tabletmanagerdata.VDiffOptions.report_options:type_name -> tabletmanagerdata.VDiffReportOptions + 118, // 54: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 0, // 55: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 132, // 56: tabletmanagerdata.UpdateVReplicationWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction + 133, // 57: tabletmanagerdata.UpdateVReplicationWorkflowRequest.state:type_name -> binlogdata.VReplicationWorkflowState + 120, // 58: tabletmanagerdata.UpdateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 129, // 59: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls:type_name -> binlogdata.BinlogSource + 128, // 60: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated:type_name -> vttime.Time + 128, // 61: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp:type_name -> vttime.Time + 133, // 62: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.state:type_name -> binlogdata.VReplicationWorkflowState + 128, // 63: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat:type_name -> vttime.Time + 128, // 64: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled:type_name -> vttime.Time + 65, // [65:65] is the sub-list for method output_type + 65, // [65:65] is the sub-list for method input_type + 65, // [65:65] is the sub-list for extension type_name + 65, // [65:65] is the sub-list for extension extendee + 0, // [0:65] is the sub-list for field type_name } func init() { file_tabletmanagerdata_proto_init() } @@ -7291,7 +8121,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffRequest); i { + switch v := v.(*CreateVReplicationWorkflowRequest); i { case 0: return &v.state case 1: @@ -7303,7 +8133,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffResponse); i { + switch v := v.(*CreateVReplicationWorkflowResponse); i { case 0: return &v.state case 1: @@ -7315,7 +8145,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffPickerOptions); i { + switch v := v.(*DeleteVReplicationWorkflowRequest); i { case 0: return &v.state case 1: @@ -7327,7 +8157,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffReportOptions); i { + switch v := v.(*DeleteVReplicationWorkflowResponse); i { case 0: return &v.state case 1: @@ -7339,7 +8169,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffCoreOptions); i { + switch v := v.(*ReadVReplicationWorkflowRequest); i { case 0: return &v.state case 1: @@ -7351,7 +8181,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffOptions); i { + switch v := v.(*ReadVReplicationWorkflowResponse); i { case 0: return &v.state case 1: @@ -7363,7 +8193,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateVRWorkflowRequest); i { + switch v := v.(*VDiffRequest); i { case 0: return &v.state case 1: @@ -7375,7 +8205,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateVRWorkflowResponse); i { + switch v := v.(*VDiffResponse); i { case 0: return &v.state case 1: @@ -7387,7 +8217,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResetSequencesRequest); i { + switch v := v.(*VDiffPickerOptions); i { case 0: return &v.state case 1: @@ -7399,6 +8229,78 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VDiffReportOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VDiffCoreOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VDiffOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateVReplicationWorkflowRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateVReplicationWorkflowResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResetSequencesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ResetSequencesResponse); i { case 0: return &v.state @@ -7410,19 +8312,32 @@ func file_tabletmanagerdata_proto_init() { return nil } } + file_tabletmanagerdata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReadVReplicationWorkflowResponse_Stream); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_tabletmanagerdata_proto_rawDesc, - NumEnums: 0, - NumMessages: 109, + NumEnums: 1, + NumMessages: 116, NumExtensions: 0, NumServices: 0, }, GoTypes: file_tabletmanagerdata_proto_goTypes, DependencyIndexes: file_tabletmanagerdata_proto_depIdxs, + EnumInfos: file_tabletmanagerdata_proto_enumTypes, MessageInfos: file_tabletmanagerdata_proto_msgTypes, }.Build() File_tabletmanagerdata_proto = out.File diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go index 4549f882127..ca3d9f75b24 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go @@ -4186,7 +4186,7 @@ func (m *RestoreFromBackupResponse) MarshalToSizedBufferVT(dAtA []byte) (int, er return len(dAtA) - i, nil } -func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { +func (m *CreateVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4199,12 +4199,12 @@ func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *CreateVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *CreateVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4216,55 +4216,104 @@ func (m *VDiffRequest) 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 + if m.StopAfterCopy { + i-- + if m.StopAfterCopy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x32 + dAtA[i] = 0x50 } - if len(m.VdiffUuid) > 0 { - i -= len(m.VdiffUuid) - copy(dAtA[i:], m.VdiffUuid) - i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + if m.AutoStart { i-- - dAtA[i] = 0x2a + if m.AutoStart { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x48 } - if len(m.ActionArg) > 0 { - i -= len(m.ActionArg) - copy(dAtA[i:], m.ActionArg) - i = encodeVarint(dAtA, i, uint64(len(m.ActionArg))) + if m.DeferSecondaryKeys { i-- - dAtA[i] = 0x22 + if m.DeferSecondaryKeys { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 } - if len(m.Action) > 0 { - i -= len(m.Action) - copy(dAtA[i:], m.Action) - i = encodeVarint(dAtA, i, uint64(len(m.Action))) + if m.WorkflowSubType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowSubType)) i-- - dAtA[i] = 0x1a + dAtA[i] = 0x38 + } + if m.WorkflowType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowType)) + i-- + dAtA[i] = 0x30 + } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) + i-- + dAtA[i] = 0x28 + } + if len(m.TabletTypes) > 0 { + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x22 + } + if len(m.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.BinlogSource) > 0 { + for iNdEx := len(m.BinlogSource) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.BinlogSource[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } } if len(m.Workflow) > 0 { i -= len(m.Workflow) copy(dAtA[i:], m.Workflow) i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- - dAtA[i] = 0x12 - } - if len(m.Keyspace) > 0 { - i -= len(m.Keyspace) - copy(dAtA[i:], m.Keyspace) - i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) - i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { +func (m *CreateVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4277,12 +4326,12 @@ func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *CreateVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *CreateVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4294,32 +4343,20 @@ func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.VdiffUuid) > 0 { - i -= len(m.VdiffUuid) - copy(dAtA[i:], m.VdiffUuid) - i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) - i-- - dAtA[i] = 0x1a - } - if m.Output != nil { - size, err := m.Output.MarshalToSizedBufferVT(dAtA[:i]) + if m.Result != nil { + size, err := m.Result.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } i -= size i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x12 - } - if m.Id != 0 { - i = encodeVarint(dAtA, i, uint64(m.Id)) - i-- - dAtA[i] = 0x8 + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { +func (m *DeleteVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4332,12 +4369,12 @@ func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffPickerOptions) MarshalToVT(dAtA []byte) (int, error) { +func (m *DeleteVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *DeleteVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4349,31 +4386,17 @@ func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.TargetCell) > 0 { - i -= len(m.TargetCell) - copy(dAtA[i:], m.TargetCell) - i = encodeVarint(dAtA, i, uint64(len(m.TargetCell))) - i-- - dAtA[i] = 0x1a - } - if len(m.SourceCell) > 0 { - i -= len(m.SourceCell) - copy(dAtA[i:], m.SourceCell) - i = encodeVarint(dAtA, i, uint64(len(m.SourceCell))) - i-- - dAtA[i] = 0x12 - } - if len(m.TabletTypes) > 0 { - i -= len(m.TabletTypes) - copy(dAtA[i:], m.TabletTypes) - i = encodeVarint(dAtA, i, uint64(len(m.TabletTypes))) + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffReportOptions) MarshalVT() (dAtA []byte, err error) { +func (m *DeleteVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4386,12 +4409,12 @@ func (m *VDiffReportOptions) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffReportOptions) MarshalToVT(dAtA []byte) (int, error) { +func (m *DeleteVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffReportOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *DeleteVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4403,37 +4426,20 @@ func (m *VDiffReportOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Format) > 0 { - i -= len(m.Format) - copy(dAtA[i:], m.Format) - i = encodeVarint(dAtA, i, uint64(len(m.Format))) - i-- - dAtA[i] = 0x1a - } - if m.DebugQuery { - i-- - if m.DebugQuery { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x10 - } - if m.OnlyPks { - i-- - if m.OnlyPks { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if m.Result != nil { + size, err := m.Result.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x8 + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffCoreOptions) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4446,12 +4452,12 @@ func (m *VDiffCoreOptions) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffCoreOptions) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffCoreOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4463,67 +4469,17 @@ func (m *VDiffCoreOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.UpdateTableStats { - i-- - if m.UpdateTableStats { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x40 - } - if m.MaxExtraRowsToCompare != 0 { - i = encodeVarint(dAtA, i, uint64(m.MaxExtraRowsToCompare)) - i-- - dAtA[i] = 0x38 - } - if m.TimeoutSeconds != 0 { - i = encodeVarint(dAtA, i, uint64(m.TimeoutSeconds)) - i-- - dAtA[i] = 0x30 - } - if m.SamplePct != 0 { - i = encodeVarint(dAtA, i, uint64(m.SamplePct)) - i-- - dAtA[i] = 0x28 - } - if m.Checksum { - i-- - if m.Checksum { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x20 - } - if m.MaxRows != 0 { - i = encodeVarint(dAtA, i, uint64(m.MaxRows)) - i-- - dAtA[i] = 0x18 - } - if m.AutoRetry { - i-- - if m.AutoRetry { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x10 - } - if len(m.Tables) > 0 { - i -= len(m.Tables) - copy(dAtA[i:], m.Tables) - i = encodeVarint(dAtA, i, uint64(len(m.Tables))) + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffOptions) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4536,12 +4492,12 @@ func (m *VDiffOptions) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffOptions) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4553,58 +4509,131 @@ func (m *VDiffOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.ReportOptions != nil { - size, err := m.ReportOptions.MarshalToSizedBufferVT(dAtA[:i]) + if len(m.ComponentThrottled) > 0 { + i -= len(m.ComponentThrottled) + copy(dAtA[i:], m.ComponentThrottled) + i = encodeVarint(dAtA, i, uint64(len(m.ComponentThrottled))) + i-- + dAtA[i] = 0x72 + } + if m.TimeThrottled != nil { + size, err := m.TimeThrottled.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } i -= size i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x1a + dAtA[i] = 0x6a } - if m.CoreOptions != nil { - size, err := m.CoreOptions.MarshalToSizedBufferVT(dAtA[:i]) + if m.TimeHeartbeat != nil { + size, err := m.TimeHeartbeat.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } i -= size i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x12 + dAtA[i] = 0x62 } - if m.PickerOptions != nil { - size, err := m.PickerOptions.MarshalToSizedBufferVT(dAtA[:i]) + if m.RowsCopied != 0 { + i = encodeVarint(dAtA, i, uint64(m.RowsCopied)) + i-- + dAtA[i] = 0x58 + } + if len(m.Message) > 0 { + i -= len(m.Message) + copy(dAtA[i:], m.Message) + i = encodeVarint(dAtA, i, uint64(len(m.Message))) + i-- + dAtA[i] = 0x52 + } + if m.State != 0 { + i = encodeVarint(dAtA, i, uint64(m.State)) + i-- + dAtA[i] = 0x48 + } + if m.TransactionTimestamp != nil { + size, err := m.TransactionTimestamp.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } i -= size i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0xa + dAtA[i] = 0x42 } - return len(dAtA) - i, nil -} - -func (m *UpdateVRWorkflowRequest) MarshalVT() (dAtA []byte, err error) { - if m == nil { - return nil, nil + if m.TimeUpdated != nil { + size, err := m.TimeUpdated.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a } - size := m.SizeVT() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBufferVT(dAtA[:size]) - if err != nil { - return nil, err + if m.MaxReplicationLag != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxReplicationLag)) + i-- + dAtA[i] = 0x30 } - return dAtA[:n], nil + if m.MaxTps != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxTps)) + i-- + dAtA[i] = 0x28 + } + if len(m.StopPos) > 0 { + i -= len(m.StopPos) + copy(dAtA[i:], m.StopPos) + i = encodeVarint(dAtA, i, uint64(len(m.StopPos))) + i-- + dAtA[i] = 0x22 + } + if len(m.Pos) > 0 { + i -= len(m.Pos) + copy(dAtA[i:], m.Pos) + i = encodeVarint(dAtA, i, uint64(len(m.Pos))) + i-- + dAtA[i] = 0x1a + } + if m.Bls != nil { + size, err := m.Bls.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Id != 0 { + i = encodeVarint(dAtA, i, uint64(m.Id)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ReadVReplicationWorkflowResponse) 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 *UpdateVRWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *UpdateVRWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4616,40 +4645,96 @@ func (m *UpdateVRWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, erro i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.OnDdl != 0 { - i = encodeVarint(dAtA, i, uint64(m.OnDdl)) + if len(m.Streams) > 0 { + for iNdEx := len(m.Streams) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Streams[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } + } + if m.DeferSecondaryKeys { i-- - dAtA[i] = 0x20 + if m.DeferSecondaryKeys { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + } + if m.WorkflowSubType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowSubType)) + i-- + dAtA[i] = 0x48 + } + if m.WorkflowType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowType)) + i-- + dAtA[i] = 0x40 + } + if len(m.Tags) > 0 { + i -= len(m.Tags) + copy(dAtA[i:], m.Tags) + i = encodeVarint(dAtA, i, uint64(len(m.Tags))) + i-- + dAtA[i] = 0x3a + } + if len(m.DbName) > 0 { + i -= len(m.DbName) + copy(dAtA[i:], m.DbName) + i = encodeVarint(dAtA, i, uint64(len(m.DbName))) + i-- + dAtA[i] = 0x32 + } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) + i-- + dAtA[i] = 0x28 } if len(m.TabletTypes) > 0 { - for iNdEx := len(m.TabletTypes) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.TabletTypes[iNdEx]) - copy(dAtA[i:], m.TabletTypes[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.TabletTypes[iNdEx]))) - i-- - dAtA[i] = 0x1a - } + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x22 } if len(m.Cells) > 0 { - for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Cells[iNdEx]) - copy(dAtA[i:], m.Cells[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) - i-- - dAtA[i] = 0x12 - } + i -= len(m.Cells) + copy(dAtA[i:], m.Cells) + i = encodeVarint(dAtA, i, uint64(len(m.Cells))) + i-- + dAtA[i] = 0x1a } if len(m.Workflow) > 0 { i -= len(m.Workflow) copy(dAtA[i:], m.Workflow) i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- - dAtA[i] = 0xa + dAtA[i] = 0x12 } return len(dAtA) - i, nil } -func (m *UpdateVRWorkflowResponse) MarshalVT() (dAtA []byte, err error) { +func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4662,12 +4747,12 @@ func (m *UpdateVRWorkflowResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *UpdateVRWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *VDiffRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *UpdateVRWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *VDiffRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4679,20 +4764,55 @@ func (m *UpdateVRWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, err i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.Result != nil { - size, err := m.Result.MarshalToSizedBufferVT(dAtA[:i]) + 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] = 0x32 + } + if len(m.VdiffUuid) > 0 { + i -= len(m.VdiffUuid) + copy(dAtA[i:], m.VdiffUuid) + i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + i-- + dAtA[i] = 0x2a + } + if len(m.ActionArg) > 0 { + i -= len(m.ActionArg) + copy(dAtA[i:], m.ActionArg) + i = encodeVarint(dAtA, i, uint64(len(m.ActionArg))) + i-- + dAtA[i] = 0x22 + } + if len(m.Action) > 0 { + i -= len(m.Action) + copy(dAtA[i:], m.Action) + i = encodeVarint(dAtA, i, uint64(len(m.Action))) + i-- + dAtA[i] = 0x1a + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { +func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4705,12 +4825,12 @@ func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ResetSequencesRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *VDiffResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4722,19 +4842,32 @@ func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Tables) > 0 { - for iNdEx := len(m.Tables) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Tables[iNdEx]) - copy(dAtA[i:], m.Tables[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.Tables[iNdEx]))) - i-- - dAtA[i] = 0xa + if len(m.VdiffUuid) > 0 { + i -= len(m.VdiffUuid) + copy(dAtA[i:], m.VdiffUuid) + i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + i-- + dAtA[i] = 0x1a + } + if m.Output != nil { + size, err := m.Output.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Id != 0 { + i = encodeVarint(dAtA, i, uint64(m.Id)) + i-- + dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func (m *ResetSequencesResponse) MarshalVT() (dAtA []byte, err error) { +func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -4747,12 +4880,12 @@ func (m *ResetSequencesResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ResetSequencesResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *VDiffPickerOptions) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ResetSequencesResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -4764,352 +4897,789 @@ func (m *ResetSequencesResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.TargetCell) > 0 { + i -= len(m.TargetCell) + copy(dAtA[i:], m.TargetCell) + i = encodeVarint(dAtA, i, uint64(len(m.TargetCell))) + i-- + dAtA[i] = 0x1a + } + if len(m.SourceCell) > 0 { + i -= len(m.SourceCell) + copy(dAtA[i:], m.SourceCell) + i = encodeVarint(dAtA, i, uint64(len(m.SourceCell))) + i-- + dAtA[i] = 0x12 + } + if len(m.TabletTypes) > 0 { + i -= len(m.TabletTypes) + copy(dAtA[i:], m.TabletTypes) + i = encodeVarint(dAtA, i, uint64(len(m.TabletTypes))) + i-- + dAtA[i] = 0xa + } return len(dAtA) - i, nil } -func encodeVarint(dAtA []byte, offset int, v uint64) int { - offset -= sov(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *VDiffReportOptions) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil } - dAtA[offset] = uint8(v) - return base + 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 *TableDefinition) SizeVT() (n int) { + +func (m *VDiffReportOptions) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffReportOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - l = len(m.Schema) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.Format) > 0 { + i -= len(m.Format) + copy(dAtA[i:], m.Format) + i = encodeVarint(dAtA, i, uint64(len(m.Format))) + i-- + dAtA[i] = 0x1a } - if len(m.Columns) > 0 { - for _, s := range m.Columns { - l = len(s) - n += 1 + l + sov(uint64(l)) + if m.DebugQuery { + i-- + if m.DebugQuery { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x10 } - if len(m.PrimaryKeyColumns) > 0 { - for _, s := range m.PrimaryKeyColumns { - l = len(s) - n += 1 + l + sov(uint64(l)) + if m.OnlyPks { + i-- + if m.OnlyPks { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - l = len(m.Type) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - if m.DataLength != 0 { - n += 1 + sov(uint64(m.DataLength)) - } - if m.RowCount != 0 { - n += 1 + sov(uint64(m.RowCount)) - } - if len(m.Fields) > 0 { - for _, e := range m.Fields { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) - } - } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *SchemaDefinition) SizeVT() (n int) { +func (m *VDiffCoreOptions) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 - } - var l int - _ = l - l = len(m.DatabaseSchema) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return nil, nil } - if len(m.TableDefinitions) > 0 { - for _, e := range m.TableDefinitions { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) - } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *SchemaChangeResult) SizeVT() (n int) { +func (m *VDiffCoreOptions) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffCoreOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if m.BeforeSchema != nil { - l = m.BeforeSchema.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if m.AfterSchema != nil { - l = m.AfterSchema.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.UpdateTableStats { + i-- + if m.UpdateTableStats { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 } - n += len(m.unknownFields) - return n -} - -func (m *UserPermission) SizeVT() (n int) { - if m == nil { - return 0 + if m.MaxExtraRowsToCompare != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxExtraRowsToCompare)) + i-- + dAtA[i] = 0x38 } - var l int - _ = l - l = len(m.Host) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.TimeoutSeconds != 0 { + i = encodeVarint(dAtA, i, uint64(m.TimeoutSeconds)) + i-- + dAtA[i] = 0x30 } - l = len(m.User) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.SamplePct != 0 { + i = encodeVarint(dAtA, i, uint64(m.SamplePct)) + i-- + dAtA[i] = 0x28 } - if m.PasswordChecksum != 0 { - n += 1 + sov(uint64(m.PasswordChecksum)) + if m.Checksum { + i-- + if m.Checksum { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 } - if len(m.Privileges) > 0 { - for k, v := range m.Privileges { - _ = k - _ = v - mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) - n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + if m.MaxRows != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxRows)) + i-- + dAtA[i] = 0x18 + } + if m.AutoRetry { + i-- + if m.AutoRetry { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x10 } - n += len(m.unknownFields) - return n + if len(m.Tables) > 0 { + i -= len(m.Tables) + copy(dAtA[i:], m.Tables) + i = encodeVarint(dAtA, i, uint64(len(m.Tables))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } -func (m *DbPermission) SizeVT() (n int) { +func (m *VDiffOptions) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Host) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - l = len(m.Db) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - l = len(m.User) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return nil, nil } - if len(m.Privileges) > 0 { - for k, v := range m.Privileges { - _ = k - _ = v - mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) - n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) - } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *Permissions) SizeVT() (n int) { +func (m *VDiffOptions) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if len(m.UserPermissions) > 0 { - for _, e := range m.UserPermissions { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.ReportOptions != nil { + size, err := m.ReportOptions.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a } - if len(m.DbPermissions) > 0 { - for _, e := range m.DbPermissions { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.CoreOptions != nil { + size, err := m.CoreOptions.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.PickerOptions != nil { + size, err := m.PickerOptions.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *PingRequest) SizeVT() (n int) { +func (m *UpdateVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 + return nil, nil } - var l int - _ = l - l = len(m.Payload) - if l > 0 { - n += 1 + l + sov(uint64(l)) + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *PingResponse) SizeVT() (n int) { +func (m *UpdateVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpdateVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Payload) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - n += len(m.unknownFields) - return n + if m.State != 0 { + i = encodeVarint(dAtA, i, uint64(m.State)) + i-- + dAtA[i] = 0x30 + } + if m.OnDdl != 0 { + i = encodeVarint(dAtA, i, uint64(m.OnDdl)) + i-- + dAtA[i] = 0x28 + } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) + i-- + dAtA[i] = 0x20 + } + if len(m.TabletTypes) > 0 { + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x1a + } + if len(m.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } -func (m *SleepRequest) SizeVT() (n int) { +func (m *UpdateVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 + return nil, nil } - var l int - _ = l - if m.Duration != 0 { - n += 1 + sov(uint64(m.Duration)) + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *SleepResponse) SizeVT() (n int) { +func (m *UpdateVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpdateVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - n += len(m.unknownFields) - return n + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Result != nil { + size, err := m.Result.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 *ExecuteHookRequest) SizeVT() (n int) { +func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 + 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 *ResetSequencesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if len(m.Parameters) > 0 { - for _, s := range m.Parameters { - l = len(s) - n += 1 + l + sov(uint64(l)) + if len(m.Tables) > 0 { + for iNdEx := len(m.Tables) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Tables[iNdEx]) + copy(dAtA[i:], m.Tables[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Tables[iNdEx]))) + i-- + dAtA[i] = 0xa } } - if len(m.ExtraEnv) > 0 { - for k, v := range m.ExtraEnv { - _ = k - _ = v - mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) - n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) - } + return len(dAtA) - i, nil +} + +func (m *ResetSequencesResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil } - n += len(m.unknownFields) - return n + 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 *ExecuteHookResponse) SizeVT() (n int) { +func (m *ResetSequencesResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ResetSequencesResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if m.ExitStatus != 0 { - n += 1 + sov(uint64(m.ExitStatus)) - } - l = len(m.Stdout) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - l = len(m.Stderr) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *GetSchemaRequest) SizeVT() (n int) { +func encodeVarint(dAtA []byte, offset int, v uint64) int { + offset -= sov(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *TableDefinition) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l - if len(m.Tables) > 0 { - for _, s := range m.Tables { + l = len(m.Name) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Schema) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Columns) > 0 { + for _, s := range m.Columns { l = len(s) n += 1 + l + sov(uint64(l)) } } - if m.IncludeViews { - n += 2 - } - if len(m.ExcludeTables) > 0 { - for _, s := range m.ExcludeTables { + if len(m.PrimaryKeyColumns) > 0 { + for _, s := range m.PrimaryKeyColumns { l = len(s) n += 1 + l + sov(uint64(l)) } } - if m.TableSchemaOnly { - n += 2 + l = len(m.Type) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.DataLength != 0 { + n += 1 + sov(uint64(m.DataLength)) + } + if m.RowCount != 0 { + n += 1 + sov(uint64(m.RowCount)) + } + if len(m.Fields) > 0 { + for _, e := range m.Fields { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } } n += len(m.unknownFields) return n } -func (m *GetSchemaResponse) SizeVT() (n int) { +func (m *SchemaDefinition) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l - if m.SchemaDefinition != nil { - l = m.SchemaDefinition.SizeVT() + l = len(m.DatabaseSchema) + if l > 0 { n += 1 + l + sov(uint64(l)) } + if len(m.TableDefinitions) > 0 { + for _, e := range m.TableDefinitions { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } n += len(m.unknownFields) return n } -func (m *GetPermissionsRequest) SizeVT() (n int) { +func (m *SchemaChangeResult) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l + if m.BeforeSchema != nil { + l = m.BeforeSchema.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.AfterSchema != nil { + l = m.AfterSchema.SizeVT() + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } -func (m *GetPermissionsResponse) SizeVT() (n int) { +func (m *UserPermission) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l - if m.Permissions != nil { - l = m.Permissions.SizeVT() + l = len(m.Host) + if l > 0 { n += 1 + l + sov(uint64(l)) } + l = len(m.User) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.PasswordChecksum != 0 { + n += 1 + sov(uint64(m.PasswordChecksum)) + } + if len(m.Privileges) > 0 { + for k, v := range m.Privileges { + _ = k + _ = v + mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) + n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + } + } n += len(m.unknownFields) return n } -func (m *SetReadOnlyRequest) SizeVT() (n int) { +func (m *DbPermission) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Host) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Db) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.User) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Privileges) > 0 { + for k, v := range m.Privileges { + _ = k + _ = v + mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) + n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *Permissions) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.UserPermissions) > 0 { + for _, e := range m.UserPermissions { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + if len(m.DbPermissions) > 0 { + for _, e := range m.DbPermissions { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *PingRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Payload) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *PingResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Payload) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SleepRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Duration != 0 { + n += 1 + sov(uint64(m.Duration)) + } + n += len(m.unknownFields) + return n +} + +func (m *SleepResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *ExecuteHookRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Parameters) > 0 { + for _, s := range m.Parameters { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.ExtraEnv) > 0 { + for k, v := range m.ExtraEnv { + _ = k + _ = v + mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + 1 + len(v) + sov(uint64(len(v))) + n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ExecuteHookResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExitStatus != 0 { + n += 1 + sov(uint64(m.ExitStatus)) + } + l = len(m.Stdout) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Stderr) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetSchemaRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Tables) > 0 { + for _, s := range m.Tables { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if m.IncludeViews { + n += 2 + } + if len(m.ExcludeTables) > 0 { + for _, s := range m.ExcludeTables { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if m.TableSchemaOnly { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *GetSchemaResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SchemaDefinition != nil { + l = m.SchemaDefinition.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *GetPermissionsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *GetPermissionsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Permissions != nil { + l = m.Permissions.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *SetReadOnlyRequest) SizeVT() (n int) { if m == nil { return 0 } @@ -6206,47 +6776,266 @@ func (m *RestoreFromBackupResponse) SizeVT() (n int) { return n } -func (m *VDiffRequest) SizeVT() (n int) { +func (m *CreateVReplicationWorkflowRequest) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Keyspace) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } l = len(m.Workflow) if l > 0 { n += 1 + l + sov(uint64(l)) } - l = len(m.Action) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.BinlogSource) > 0 { + for _, e := range m.BinlogSource { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } } - l = len(m.ActionArg) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.Cells) > 0 { + for _, s := range m.Cells { + l = len(s) + n += 1 + l + sov(uint64(l)) + } } - l = len(m.VdiffUuid) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.TabletTypes) > 0 { + l = 0 + for _, e := range m.TabletTypes { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l } - if m.Options != nil { - l = m.Options.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.TabletSelectionPreference != 0 { + n += 1 + sov(uint64(m.TabletSelectionPreference)) + } + if m.WorkflowType != 0 { + n += 1 + sov(uint64(m.WorkflowType)) + } + if m.WorkflowSubType != 0 { + n += 1 + sov(uint64(m.WorkflowSubType)) + } + if m.DeferSecondaryKeys { + n += 2 + } + if m.AutoStart { + n += 2 + } + if m.StopAfterCopy { + n += 2 } n += len(m.unknownFields) return n } -func (m *VDiffResponse) SizeVT() (n int) { +func (m *CreateVReplicationWorkflowResponse) SizeVT() (n int) { if m == nil { return 0 } var l int _ = l - if m.Id != 0 { + if m.Result != nil { + l = m.Result.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DeleteVReplicationWorkflowRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *DeleteVReplicationWorkflowResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result != nil { + l = m.Result.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ReadVReplicationWorkflowRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ReadVReplicationWorkflowResponse_Stream) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Id != 0 { + n += 1 + sov(uint64(m.Id)) + } + if m.Bls != nil { + l = m.Bls.SizeVT() + n += 1 + l + sov(uint64(l)) + } + l = len(m.Pos) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.StopPos) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.MaxTps != 0 { + n += 1 + sov(uint64(m.MaxTps)) + } + if m.MaxReplicationLag != 0 { + n += 1 + sov(uint64(m.MaxReplicationLag)) + } + if m.TimeUpdated != nil { + l = m.TimeUpdated.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.TransactionTimestamp != nil { + l = m.TransactionTimestamp.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.State != 0 { + n += 1 + sov(uint64(m.State)) + } + l = len(m.Message) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.RowsCopied != 0 { + n += 1 + sov(uint64(m.RowsCopied)) + } + if m.TimeHeartbeat != nil { + l = m.TimeHeartbeat.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.TimeThrottled != nil { + l = m.TimeThrottled.SizeVT() + n += 1 + l + sov(uint64(l)) + } + l = len(m.ComponentThrottled) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ReadVReplicationWorkflowResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Cells) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.TabletTypes) > 0 { + l = 0 + for _, e := range m.TabletTypes { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if m.TabletSelectionPreference != 0 { + n += 1 + sov(uint64(m.TabletSelectionPreference)) + } + l = len(m.DbName) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Tags) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.WorkflowType != 0 { + n += 1 + sov(uint64(m.WorkflowType)) + } + if m.WorkflowSubType != 0 { + n += 1 + sov(uint64(m.WorkflowSubType)) + } + if m.DeferSecondaryKeys { + n += 2 + } + if len(m.Streams) > 0 { + for _, e := range m.Streams { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *VDiffRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Action) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.ActionArg) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.VdiffUuid) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.Options != nil { + l = m.Options.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *VDiffResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Id != 0 { n += 1 + sov(uint64(m.Id)) } if m.Output != nil { @@ -6360,7 +7149,7 @@ func (m *VDiffOptions) SizeVT() (n int) { return n } -func (m *UpdateVRWorkflowRequest) SizeVT() (n int) { +func (m *UpdateVReplicationWorkflowRequest) SizeVT() (n int) { if m == nil { return 0 } @@ -6377,65 +7166,1536 @@ func (m *UpdateVRWorkflowRequest) SizeVT() (n int) { } } if len(m.TabletTypes) > 0 { - for _, s := range m.TabletTypes { + l = 0 + for _, e := range m.TabletTypes { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if m.TabletSelectionPreference != 0 { + n += 1 + sov(uint64(m.TabletSelectionPreference)) + } + if m.OnDdl != 0 { + n += 1 + sov(uint64(m.OnDdl)) + } + if m.State != 0 { + n += 1 + sov(uint64(m.State)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpdateVReplicationWorkflowResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result != nil { + l = m.Result.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *ResetSequencesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Tables) > 0 { + for _, s := range m.Tables { l = len(s) n += 1 + l + sov(uint64(l)) } } - if m.OnDdl != 0 { - n += 1 + sov(uint64(m.OnDdl)) - } - n += len(m.unknownFields) - return n -} + n += len(m.unknownFields) + return n +} + +func (m *ResetSequencesResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func sov(x uint64) (n int) { + return (bits.Len64(x|1) + 6) / 7 +} +func soz(x uint64) (n int) { + return sov(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *TableDefinition) 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: TableDefinition: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TableDefinition: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", 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.Schema = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Columns", 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.Columns = append(m.Columns, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryKeyColumns", 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.PrimaryKeyColumns = append(m.PrimaryKeyColumns, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", 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.Type = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DataLength", wireType) + } + m.DataLength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DataLength |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowCount", wireType) + } + m.RowCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RowCount |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Fields", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Fields = append(m.Fields, &query.Field{}) + if err := m.Fields[len(m.Fields)-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 *SchemaDefinition) 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: SchemaDefinition: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SchemaDefinition: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DatabaseSchema", 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.DatabaseSchema = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TableDefinitions", 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.TableDefinitions = append(m.TableDefinitions, &TableDefinition{}) + if err := m.TableDefinitions[len(m.TableDefinitions)-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 *SchemaChangeResult) 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: SchemaChangeResult: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SchemaChangeResult: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeforeSchema", 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.BeforeSchema == nil { + m.BeforeSchema = &SchemaDefinition{} + } + if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", 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.AfterSchema == nil { + m.AfterSchema = &SchemaDefinition{} + } + if err := m.AfterSchema.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 *UserPermission) 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: UserPermission: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UserPermission: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Host", 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.Host = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field User", 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.User = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PasswordChecksum", wireType) + } + m.PasswordChecksum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PasswordChecksum |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Privileges", 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.Privileges == nil { + m.Privileges = make(map[string]string) + } + var mapkey string + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLength + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return ErrInvalidLength + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Privileges[mapkey] = mapvalue + 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 *DbPermission) 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: DbPermission: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DbPermission: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Host", 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.Host = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Db", 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.Db = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field User", 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.User = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Privileges", 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.Privileges == nil { + m.Privileges = make(map[string]string) + } + var mapkey string + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLength + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return ErrInvalidLength + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Privileges[mapkey] = mapvalue + 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 *Permissions) 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: Permissions: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Permissions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UserPermissions", 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.UserPermissions = append(m.UserPermissions, &UserPermission{}) + if err := m.UserPermissions[len(m.UserPermissions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DbPermissions", 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.DbPermissions = append(m.DbPermissions, &DbPermission{}) + if err := m.DbPermissions[len(m.DbPermissions)-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 *PingRequest) 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: PingRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PingRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Payload", 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.Payload = 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 *PingResponse) 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: PingResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PingResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Payload", 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.Payload = 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 *SleepRequest) 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: SleepRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SleepRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) + } + m.Duration = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Duration |= int64(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 { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } -func (m *UpdateVRWorkflowResponse) SizeVT() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Result != nil { - l = m.Result.SizeVT() - n += 1 + l + sov(uint64(l)) + if iNdEx > l { + return io.ErrUnexpectedEOF } - n += len(m.unknownFields) - return n + return nil } - -func (m *ResetSequencesRequest) SizeVT() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Tables) > 0 { - for _, s := range m.Tables { - l = len(s) - n += 1 + l + sov(uint64(l)) +func (m *SleepResponse) 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: SleepResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SleepResponse: 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 } } - n += len(m.unknownFields) - return n -} -func (m *ResetSequencesResponse) SizeVT() (n int) { - if m == nil { - return 0 + if iNdEx > l { + return io.ErrUnexpectedEOF } - var l int - _ = l - n += len(m.unknownFields) - return n -} - -func sov(x uint64) (n int) { - return (bits.Len64(x|1) + 6) / 7 -} -func soz(x uint64) (n int) { - return sov(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + return nil } -func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6458,10 +8718,10 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TableDefinition: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteHookRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TableDefinition: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteHookRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6498,7 +8758,7 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Parameters", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6523,16 +8783,194 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { if postIndex < 0 { return ErrInvalidLength } - if postIndex > l { + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Parameters = append(m.Parameters, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraEnv", 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.ExtraEnv == nil { + m.ExtraEnv = make(map[string]string) + } + var mapkey string + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLength + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return ErrInvalidLength + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.ExtraEnv[mapkey] = mapvalue + 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 *ExecuteHookResponse) 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 } - m.Schema = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Columns", wireType) + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - var stringLen uint64 + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExecuteHookResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecuteHookResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExitStatus", wireType) + } + m.ExitStatus = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -6542,27 +8980,14 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.ExitStatus |= int64(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.Columns = append(m.Columns, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryKeyColumns", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Stdout", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6590,11 +9015,11 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.PrimaryKeyColumns = append(m.PrimaryKeyColumns, string(dAtA[iNdEx:postIndex])) + m.Stdout = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 5: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Stderr", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6622,79 +9047,7 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Type = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DataLength", wireType) - } - m.DataLength = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.DataLength |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RowCount", wireType) - } - m.RowCount = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.RowCount |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Fields", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Fields = append(m.Fields, &query.Field{}) - if err := m.Fields[len(m.Fields)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Stderr = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -6718,7 +9071,7 @@ func (m *TableDefinition) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SchemaDefinition) UnmarshalVT(dAtA []byte) error { +func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6741,15 +9094,15 @@ func (m *SchemaDefinition) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SchemaDefinition: wiretype end group for non-group") + return fmt.Errorf("proto: GetSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SchemaDefinition: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DatabaseSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6777,13 +9130,33 @@ func (m *SchemaDefinition) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.DatabaseSchema = string(dAtA[iNdEx:postIndex]) + m.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", 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.IncludeViews = bool(v != 0) + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TableDefinitions", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -6793,26 +9166,44 @@ func (m *SchemaDefinition) 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 } - m.TableDefinitions = append(m.TableDefinitions, &TableDefinition{}) - if err := m.TableDefinitions[len(m.TableDefinitions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableSchemaOnly", 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.TableSchemaOnly = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -6835,7 +9226,7 @@ func (m *SchemaDefinition) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SchemaChangeResult) UnmarshalVT(dAtA []byte) error { +func (m *GetSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6858,51 +9249,15 @@ func (m *SchemaChangeResult) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SchemaChangeResult: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SchemaChangeResult: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BeforeSchema", 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.BeforeSchema == nil { - m.BeforeSchema = &SchemaDefinition{} - } - if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: + return fmt.Errorf("proto: GetSchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SchemaDefinition", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6929,10 +9284,10 @@ func (m *SchemaChangeResult) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.AfterSchema == nil { - m.AfterSchema = &SchemaDefinition{} + if m.SchemaDefinition == nil { + m.SchemaDefinition = &SchemaDefinition{} } - if err := m.AfterSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.SchemaDefinition.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -6958,7 +9313,7 @@ func (m *SchemaChangeResult) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UserPermission) UnmarshalVT(dAtA []byte) error { +func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6981,221 +9336,98 @@ func (m *UserPermission) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UserPermission: wiretype end group for non-group") + return fmt.Errorf("proto: GetPermissionsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UserPermission: 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 Host", 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.Host = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field User", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.User = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PasswordChecksum", wireType) - } - m.PasswordChecksum = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.PasswordChecksum |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Privileges", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetPermissionsResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - if m.Privileges == nil { - m.Privileges = make(map[string]string) + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - var mapkey string - var mapvalue string - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var stringLenmapvalue uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapvalue |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { - return ErrInvalidLength - } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue < 0 { - return ErrInvalidLength - } - if postStringIndexmapvalue > l { - return io.ErrUnexpectedEOF - } - mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetPermissionsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + 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 Permissions", 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 } } - m.Privileges[mapkey] = mapvalue + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Permissions == nil { + m.Permissions = &Permissions{} + } + if err := m.Permissions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -7219,7 +9451,7 @@ func (m *UserPermission) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DbPermission) UnmarshalVT(dAtA []byte) error { +func (m *SetReadOnlyRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7242,235 +9474,165 @@ func (m *DbPermission) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DbPermission: wiretype end group for non-group") + return fmt.Errorf("proto: SetReadOnlyRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DbPermission: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetReadOnlyRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Host", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Host = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Db", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SetReadOnlyResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - m.Db = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field User", wireType) + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - 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 - } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetReadOnlyResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetReadOnlyResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - if postIndex > l { + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SetReadWriteRequest) 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 } - m.User = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Privileges", wireType) + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - 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 - } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetReadWriteRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetReadWriteRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - if msglen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + msglen - if postIndex < 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 *SetReadWriteResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - if m.Privileges == nil { - m.Privileges = make(map[string]string) - } - var mapkey string - var mapvalue string - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var stringLenmapvalue uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapvalue |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { - return ErrInvalidLength - } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue < 0 { - return ErrInvalidLength - } - if postStringIndexmapvalue > l { - return io.ErrUnexpectedEOF - } - mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - m.Privileges[mapkey] = mapvalue - iNdEx = postIndex + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetReadWriteResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetReadWriteResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -7493,7 +9655,7 @@ func (m *DbPermission) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Permissions) UnmarshalVT(dAtA []byte) error { +func (m *ChangeTypeRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7516,17 +9678,17 @@ func (m *Permissions) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Permissions: wiretype end group for non-group") + return fmt.Errorf("proto: ChangeTypeRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Permissions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ChangeTypeRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UserPermissions", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) } - var msglen int + m.TabletType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -7536,31 +9698,16 @@ func (m *Permissions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.TabletType |= topodata.TabletType(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.UserPermissions = append(m.UserPermissions, &UserPermission{}) - if err := m.UserPermissions[len(m.UserPermissions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbPermissions", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -7570,26 +9717,12 @@ func (m *Permissions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.DbPermissions = append(m.DbPermissions, &DbPermission{}) - if err := m.DbPermissions[len(m.DbPermissions)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -7612,7 +9745,7 @@ func (m *Permissions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PingRequest) UnmarshalVT(dAtA []byte) error { +func (m *ChangeTypeResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7635,44 +9768,12 @@ func (m *PingRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PingRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ChangeTypeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PingRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ChangeTypeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Payload", 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.Payload = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -7695,7 +9796,7 @@ func (m *PingRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PingResponse) UnmarshalVT(dAtA []byte) error { +func (m *RefreshStateRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7718,44 +9819,63 @@ func (m *PingResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PingResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshStateRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PingResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshStateRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Payload", 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 - } + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen - if postIndex < 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 *RefreshStateResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - m.Payload = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + 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: RefreshStateResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RefreshStateResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -7778,7 +9898,7 @@ func (m *PingResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SleepRequest) UnmarshalVT(dAtA []byte) error { +func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7801,31 +9921,12 @@ func (m *SleepRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SleepRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RunHealthCheckRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SleepRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RunHealthCheckRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) - } - m.Duration = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Duration |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -7848,7 +9949,7 @@ func (m *SleepRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SleepResponse) UnmarshalVT(dAtA []byte) error { +func (m *RunHealthCheckResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7871,10 +9972,10 @@ func (m *SleepResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SleepResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RunHealthCheckResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SleepResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RunHealthCheckResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -7899,7 +10000,7 @@ func (m *SleepResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7922,15 +10023,15 @@ func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteHookRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteHookRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7958,167 +10059,59 @@ func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.WaitPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Parameters", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Parameters = append(m.Parameters, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExtraEnv", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ReloadSchemaResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - if m.ExtraEnv == nil { - m.ExtraEnv = make(map[string]string) - } - var mapkey string - var mapvalue string - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var stringLenmapvalue uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapvalue |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { - return ErrInvalidLength - } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue < 0 { - return ErrInvalidLength - } - if postStringIndexmapvalue > l { - return io.ErrUnexpectedEOF - } - mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - m.ExtraEnv[mapkey] = mapvalue - iNdEx = postIndex + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReloadSchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReloadSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -8141,7 +10134,7 @@ func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { +func (m *PreflightSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8164,34 +10157,15 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteHookResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PreflightSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteHookResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PreflightSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ExitStatus", wireType) - } - m.ExitStatus = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ExitStatus |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stdout", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Changes", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -8219,13 +10193,64 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Stdout = string(dAtA[iNdEx:postIndex]) + m.Changes = append(m.Changes, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 3: + 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 *PreflightSchemaResponse) 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: PreflightSchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PreflightSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stderr", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ChangeResults", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -8235,23 +10260,25 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Stderr = string(dAtA[iNdEx:postIndex]) + m.ChangeResults = append(m.ChangeResults, &SchemaChangeResult{}) + if err := m.ChangeResults[len(m.ChangeResults)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -8275,7 +10302,7 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8298,15 +10325,15 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ApplySchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplySchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sql", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -8334,11 +10361,11 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) + m.Sql = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -8355,12 +10382,12 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { break } } - m.IncludeViews = bool(v != 0) + m.Force = bool(v != 0) case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowReplication", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -8370,29 +10397,17 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex + m.AllowReplication = bool(v != 0) case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableSchemaOnly", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeforeSchema", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -8402,66 +10417,31 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.TableSchemaOnly = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + if msglen < 0 { + return ErrInvalidLength } - if (skippy < 0) || (iNdEx+skippy) < 0 { + postIndex := iNdEx + msglen + if postIndex < 0 { return ErrInvalidLength } - if (iNdEx + skippy) > l { + if postIndex > 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 *GetSchemaResponse) 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 + if m.BeforeSchema == nil { + m.BeforeSchema = &SchemaDefinition{} } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: GetSchemaResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SchemaDefinition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -8488,64 +10468,45 @@ func (m *GetSchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.SchemaDefinition == nil { - m.SchemaDefinition = &SchemaDefinition{} + if m.AfterSchema == nil { + m.AfterSchema = &SchemaDefinition{} } - if err := m.SchemaDefinition.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.AfterSchema.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 + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SqlMode", wireType) } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *GetPermissionsRequest) 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 + 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 + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: GetPermissionsRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetPermissionsRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SqlMode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -8568,7 +10529,7 @@ func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { +func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8591,15 +10552,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: ApplySchemaResponse: 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: ApplySchemaResponse: 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 BeforeSchema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -8626,10 +10587,46 @@ func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Permissions == nil { - m.Permissions = &Permissions{} + if m.BeforeSchema == nil { + m.BeforeSchema = &SchemaDefinition{} } - if err := m.Permissions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", 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.AfterSchema == nil { + m.AfterSchema = &SchemaDefinition{} + } + if err := m.AfterSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -8655,7 +10652,7 @@ func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetReadOnlyRequest) UnmarshalVT(dAtA []byte) error { +func (m *LockTablesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8678,10 +10675,10 @@ func (m *SetReadOnlyRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetReadOnlyRequest: wiretype end group for non-group") + return fmt.Errorf("proto: LockTablesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetReadOnlyRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: LockTablesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -8706,7 +10703,7 @@ func (m *SetReadOnlyRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetReadOnlyResponse) UnmarshalVT(dAtA []byte) error { +func (m *LockTablesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8729,10 +10726,10 @@ func (m *SetReadOnlyResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetReadOnlyResponse: wiretype end group for non-group") + return fmt.Errorf("proto: LockTablesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetReadOnlyResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: LockTablesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -8757,7 +10754,7 @@ func (m *SetReadOnlyResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetReadWriteRequest) UnmarshalVT(dAtA []byte) error { +func (m *UnlockTablesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8780,10 +10777,10 @@ func (m *SetReadWriteRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetReadWriteRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UnlockTablesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetReadWriteRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UnlockTablesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -8808,7 +10805,7 @@ func (m *SetReadWriteRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetReadWriteResponse) UnmarshalVT(dAtA []byte) error { +func (m *UnlockTablesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8831,10 +10828,10 @@ func (m *SetReadWriteResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetReadWriteResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UnlockTablesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetReadWriteResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UnlockTablesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -8859,7 +10856,7 @@ func (m *SetReadWriteResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ChangeTypeRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteQueryRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8882,17 +10879,17 @@ func (m *ChangeTypeRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ChangeTypeRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteQueryRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ChangeTypeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteQueryRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } - m.TabletType = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -8902,16 +10899,31 @@ func (m *ChangeTypeRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TabletType |= topodata.TabletType(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } + if byteLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) + if m.Query == nil { + m.Query = []byte{} + } + iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -8921,114 +10933,79 @@ func (m *ChangeTypeRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.SemiSync = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { + intStringLen := int(stringLen) + if intStringLen < 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 *ChangeTypeResponse) 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 + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > 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: ChangeTypeResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ChangeTypeResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + m.DbName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + m.MaxRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxRows |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CallerId", wireType) } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *RefreshStateRequest) 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 + 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 iNdEx >= l { + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.CallerId == nil { + m.CallerId = &vtrpc.CallerID{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: RefreshStateRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if err := m.CallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9051,7 +11028,7 @@ func (m *RefreshStateRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RefreshStateResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteQueryResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9074,63 +11051,48 @@ func (m *RefreshStateResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshStateResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteQueryResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteQueryResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + 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 (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLength } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *RunHealthCheckRequest) 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 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.Result == nil { + m.Result = &query.QueryResult{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: RunHealthCheckRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RunHealthCheckRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9153,7 +11115,7 @@ func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RunHealthCheckResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9176,66 +11138,49 @@ func (m *RunHealthCheckResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RunHealthCheckResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsDbaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RunHealthCheckResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsDbaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if byteLen < 0 { + return ErrInvalidLength } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ReloadSchemaRequest) 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 + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) + if m.Query == nil { + m.Query = []byte{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -9263,8 +11208,67 @@ func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.WaitPosition = string(dAtA[iNdEx:postIndex]) + m.DbName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) + } + m.MaxRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxRows |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DisableBinlogs", 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.DisableBinlogs = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", 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.ReloadSchema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9287,7 +11291,7 @@ func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsDbaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9310,12 +11314,48 @@ func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsDbaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsDbaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", 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.Result == nil { + m.Result = &query.QueryResult{} + } + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9338,7 +11378,7 @@ func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PreflightSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsAllPrivsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9361,15 +11401,49 @@ func (m *PreflightSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PreflightSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsAllPrivsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PreflightSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsAllPrivsRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Changes", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) + if m.Query == nil { + m.Query = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -9397,8 +11471,47 @@ func (m *PreflightSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Changes = append(m.Changes, string(dAtA[iNdEx:postIndex])) + m.DbName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) + } + m.MaxRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxRows |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", 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.ReloadSchema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9421,7 +11534,7 @@ func (m *PreflightSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PreflightSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsAllPrivsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9444,15 +11557,15 @@ func (m *PreflightSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PreflightSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsAllPrivsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PreflightSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsAllPrivsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ChangeResults", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9479,8 +11592,10 @@ func (m *PreflightSchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ChangeResults = append(m.ChangeResults, &SchemaChangeResult{}) - if err := m.ChangeResults[len(m.ChangeResults)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Result == nil { + m.Result = &query.QueryResult{} + } + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -9506,7 +11621,7 @@ func (m *PreflightSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9529,125 +11644,17 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplySchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsAppRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplySchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsAppRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sql", 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.Sql = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", 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.Force = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowReplication", 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.AllowReplication = bool(v != 0) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BeforeSchema", 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.BeforeSchema == nil { - m.BeforeSchema = &SchemaDefinition{} - } - if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -9657,33 +11664,31 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLength } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - if m.AfterSchema == nil { - m.AfterSchema = &SchemaDefinition{} - } - if err := m.AfterSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) + if m.Query == nil { + m.Query = []byte{} } iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SqlMode", wireType) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) } - var stringLen uint64 + m.MaxRows = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -9693,24 +11698,11 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.MaxRows |= 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.SqlMode = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9733,7 +11725,7 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9756,15 +11748,15 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplySchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsAppResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplySchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsAppResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BeforeSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9791,16 +11783,118 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.BeforeSchema == nil { - m.BeforeSchema = &SchemaDefinition{} + if m.Result == nil { + m.Result = &query.QueryResult{} } - if err := m.BeforeSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + 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 *ReplicationStatusRequest) 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: ReplicationStatusRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReplicationStatusRequest: 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 *ReplicationStatusResponse) 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: ReplicationStatusResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReplicationStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AfterSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9827,10 +11921,10 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.AfterSchema == nil { - m.AfterSchema = &SchemaDefinition{} + if m.Status == nil { + m.Status = &replicationdata.Status{} } - if err := m.AfterSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -9856,7 +11950,7 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *LockTablesRequest) UnmarshalVT(dAtA []byte) error { +func (m *PrimaryStatusRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9879,10 +11973,10 @@ func (m *LockTablesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LockTablesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: PrimaryStatusRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LockTablesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrimaryStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -9907,7 +12001,7 @@ func (m *LockTablesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *LockTablesResponse) UnmarshalVT(dAtA []byte) error { +func (m *PrimaryStatusResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9930,12 +12024,48 @@ func (m *LockTablesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LockTablesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PrimaryStatusResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LockTablesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrimaryStatusResponse: 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.PrimaryStatus{} + } + if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -9958,7 +12088,7 @@ func (m *LockTablesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UnlockTablesRequest) UnmarshalVT(dAtA []byte) error { +func (m *PrimaryPositionRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9981,10 +12111,10 @@ func (m *UnlockTablesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UnlockTablesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: PrimaryPositionRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UnlockTablesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrimaryPositionRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -10009,7 +12139,7 @@ func (m *UnlockTablesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UnlockTablesResponse) UnmarshalVT(dAtA []byte) error { +func (m *PrimaryPositionResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10028,16 +12158,48 @@ func (m *UnlockTablesResponse) UnmarshalVT(dAtA []byte) error { if b < 0x80 { break } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: UnlockTablesResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: UnlockTablesResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PrimaryPositionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PrimaryPositionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Position", 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.Position = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -10060,7 +12222,7 @@ func (m *UnlockTablesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteQueryRequest) UnmarshalVT(dAtA []byte) error { +func (m *WaitForPositionRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10083,49 +12245,15 @@ func (m *ExecuteQueryRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteQueryRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WaitForPositionRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteQueryRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WaitForPositionRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) - if m.Query == nil { - m.Query = []byte{} - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -10153,62 +12281,7 @@ func (m *ExecuteQueryRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.DbName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) - } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CallerId", 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.CallerId == nil { - m.CallerId = &vtrpc.CallerID{} - } - if err := m.CallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -10232,7 +12305,7 @@ func (m *ExecuteQueryRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteQueryResponse) UnmarshalVT(dAtA []byte) error { +func (m *WaitForPositionResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10255,48 +12328,63 @@ func (m *ExecuteQueryResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteQueryResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WaitForPositionResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteQueryResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WaitForPositionResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.Result == nil { - m.Result = &query.QueryResult{} + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StopReplicationRequest) 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 err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + if iNdEx >= l { + return io.ErrUnexpectedEOF } - iNdEx = postIndex + 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: StopReplicationRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StopReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -10319,7 +12407,7 @@ func (m *ExecuteQueryResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10342,49 +12430,66 @@ func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsDbaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StopReplicationResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsDbaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StopReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLength + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + byteLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) - if m.Query == nil { - m.Query = []byte{} + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StopReplicationMinimumRequest) 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 } - iNdEx = postIndex - case 2: + 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: StopReplicationMinimumRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StopReplicationMinimumRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -10412,52 +12517,13 @@ func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.DbName = string(dAtA[iNdEx:postIndex]) + m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) - } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DisableBinlogs", 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.DisableBinlogs = bool(v != 0) - case 5: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitTimeout", wireType) } - var v int + m.WaitTimeout = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -10467,12 +12533,11 @@ func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.WaitTimeout |= int64(b&0x7F) << shift if b < 0x80 { break } } - m.ReloadSchema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -10495,7 +12560,7 @@ func (m *ExecuteFetchAsDbaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsDbaResponse) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationMinimumResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10518,17 +12583,17 @@ func (m *ExecuteFetchAsDbaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsDbaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StopReplicationMinimumResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsDbaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StopReplicationMinimumResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -10538,27 +12603,23 @@ func (m *ExecuteFetchAsDbaResponse) 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.Result == nil { - m.Result = &query.QueryResult{} - } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -10582,7 +12643,7 @@ func (m *ExecuteFetchAsDbaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAllPrivsRequest) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10605,100 +12666,15 @@ func (m *ExecuteFetchAsAllPrivsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAllPrivsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAllPrivsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) - if m.Query == nil { - m.Query = []byte{} - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbName", 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.DbName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) - } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -10715,7 +12691,7 @@ func (m *ExecuteFetchAsAllPrivsRequest) UnmarshalVT(dAtA []byte) error { break } } - m.ReloadSchema = bool(v != 0) + m.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -10738,7 +12714,7 @@ func (m *ExecuteFetchAsAllPrivsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAllPrivsResponse) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10761,48 +12737,12 @@ func (m *ExecuteFetchAsAllPrivsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAllPrivsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAllPrivsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", 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.Result == nil { - m.Result = &query.QueryResult{} - } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -10825,7 +12765,7 @@ func (m *ExecuteFetchAsAllPrivsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10848,17 +12788,17 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAppRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationUntilAfterRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAppRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationUntilAfterRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } - var byteLen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -10868,31 +12808,29 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLength } - postIndex := iNdEx + byteLen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Query = append(m.Query[:0], dAtA[iNdEx:postIndex]...) - if m.Query == nil { - m.Query = []byte{} - } + m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitTimeout", wireType) } - m.MaxRows = 0 + m.WaitTimeout = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -10902,7 +12840,7 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxRows |= uint64(b&0x7F) << shift + m.WaitTimeout |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -10929,7 +12867,7 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationUntilAfterResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10952,48 +12890,12 @@ func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAppResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationUntilAfterResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAppResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationUntilAfterResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", 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.Result == nil { - m.Result = &query.QueryResult{} - } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11016,7 +12918,7 @@ func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReplicationStatusRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetReplicasRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11039,10 +12941,10 @@ func (m *ReplicationStatusRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReplicationStatusRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetReplicasRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicationStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetReplicasRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -11067,7 +12969,7 @@ func (m *ReplicationStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReplicationStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetReplicasResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11090,17 +12992,17 @@ func (m *ReplicationStatusResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReplicationStatusResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetReplicasResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicationStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetReplicasResponse: 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 Addrs", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -11110,27 +13012,23 @@ func (m *ReplicationStatusResponse) 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.Status == nil { - m.Status = &replicationdata.Status{} - } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Addrs = append(m.Addrs, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -11154,7 +13052,7 @@ func (m *ReplicationStatusResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PrimaryStatusRequest) UnmarshalVT(dAtA []byte) error { +func (m *ResetReplicationRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11177,10 +13075,10 @@ func (m *PrimaryStatusRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrimaryStatusRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ResetReplicationRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrimaryStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ResetReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -11205,7 +13103,7 @@ func (m *PrimaryStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PrimaryStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *ResetReplicationResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11228,48 +13126,12 @@ func (m *PrimaryStatusResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrimaryStatusResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ResetReplicationResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrimaryStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ResetReplicationResponse: 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.PrimaryStatus{} - } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11292,7 +13154,7 @@ func (m *PrimaryStatusResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PrimaryPositionRequest) UnmarshalVT(dAtA []byte) error { +func (m *VReplicationExecRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11315,12 +13177,44 @@ func (m *PrimaryPositionRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrimaryPositionRequest: wiretype end group for non-group") + return fmt.Errorf("proto: VReplicationExecRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrimaryPositionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VReplicationExecRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Query", 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.Query = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11343,7 +13237,7 @@ func (m *PrimaryPositionRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PrimaryPositionResponse) UnmarshalVT(dAtA []byte) error { +func (m *VReplicationExecResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11366,17 +13260,17 @@ func (m *PrimaryPositionResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrimaryPositionResponse: wiretype end group for non-group") + return fmt.Errorf("proto: VReplicationExecResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrimaryPositionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VReplicationExecResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -11386,23 +13280,27 @@ func (m *PrimaryPositionResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) + if m.Result == nil { + m.Result = &query.QueryResult{} + } + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -11426,7 +13324,7 @@ func (m *PrimaryPositionResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *WaitForPositionRequest) UnmarshalVT(dAtA []byte) error { +func (m *VReplicationWaitForPosRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11449,17 +13347,17 @@ func (m *WaitForPositionRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: WaitForPositionRequest: wiretype end group for non-group") + return fmt.Errorf("proto: VReplicationWaitForPosRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: WaitForPositionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VReplicationWaitForPosRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) } - var stringLen uint64 + m.Id = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -11469,75 +13367,43 @@ func (m *WaitForPositionRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.Id |= int32(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.Position = 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *WaitForPositionResponse) 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 + 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 + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: WaitForPositionResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: WaitForPositionResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Position = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11560,7 +13426,7 @@ func (m *WaitForPositionResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationRequest) UnmarshalVT(dAtA []byte) error { +func (m *VReplicationWaitForPosResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11583,10 +13449,10 @@ func (m *StopReplicationRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationRequest: wiretype end group for non-group") + return fmt.Errorf("proto: VReplicationWaitForPosResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VReplicationWaitForPosResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -11611,7 +13477,7 @@ func (m *StopReplicationRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { +func (m *InitPrimaryRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11634,12 +13500,32 @@ func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationResponse: wiretype end group for non-group") + return fmt.Errorf("proto: InitPrimaryRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitPrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", 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.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11662,7 +13548,7 @@ func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationMinimumRequest) UnmarshalVT(dAtA []byte) error { +func (m *InitPrimaryResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11685,10 +13571,10 @@ func (m *StopReplicationMinimumRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationMinimumRequest: wiretype end group for non-group") + return fmt.Errorf("proto: InitPrimaryResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationMinimumRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitPrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -11723,25 +13609,6 @@ func (m *StopReplicationMinimumRequest) UnmarshalVT(dAtA []byte) error { } m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitTimeout", wireType) - } - m.WaitTimeout = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.WaitTimeout |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11764,7 +13631,7 @@ func (m *StopReplicationMinimumRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationMinimumResponse) UnmarshalVT(dAtA []byte) error { +func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11787,15 +13654,34 @@ func (m *StopReplicationMinimumResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationMinimumResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PopulateReparentJournalRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationMinimumResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PopulateReparentJournalRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) + } + m.TimeCreatedNs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimeCreatedNs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ActionName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -11823,64 +13709,49 @@ func (m *StopReplicationMinimumResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) + m.ActionName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryAlias", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + 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 (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLength } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *StartReplicationRequest) 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 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.PrimaryAlias == nil { + m.PrimaryAlias = &topodata.TabletAlias{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: StartReplicationRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) + if err := m.PrimaryAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - var v int + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReplicationPosition", wireType) + } + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -11890,12 +13761,24 @@ func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.SemiSync = bool(v != 0) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReplicationPosition = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -11918,7 +13801,7 @@ func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { +func (m *PopulateReparentJournalResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11941,10 +13824,10 @@ func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartReplicationResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PopulateReparentJournalResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PopulateReparentJournalResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -11969,7 +13852,7 @@ func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { +func (m *InitReplicaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -11992,15 +13875,51 @@ func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartReplicationUntilAfterRequest: wiretype end group for non-group") + return fmt.Errorf("proto: InitReplicaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationUntilAfterRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitReplicaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Parent", 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.Parent == nil { + m.Parent = &topodata.TabletAlias{} + } + if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReplicationPosition", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -12028,13 +13947,13 @@ func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) + m.ReplicationPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitTimeout", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) } - m.WaitTimeout = 0 + m.TimeCreatedNs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12044,11 +13963,31 @@ func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.WaitTimeout |= int64(b&0x7F) << shift + m.TimeCreatedNs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", 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.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12071,7 +14010,7 @@ func (m *StartReplicationUntilAfterRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StartReplicationUntilAfterResponse) UnmarshalVT(dAtA []byte) error { +func (m *InitReplicaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12094,10 +14033,10 @@ func (m *StartReplicationUntilAfterResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartReplicationUntilAfterResponse: wiretype end group for non-group") + return fmt.Errorf("proto: InitReplicaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationUntilAfterResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitReplicaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -12122,7 +14061,7 @@ func (m *StartReplicationUntilAfterResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetReplicasRequest) UnmarshalVT(dAtA []byte) error { +func (m *DemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12145,10 +14084,10 @@ func (m *GetReplicasRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetReplicasRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DemotePrimaryRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetReplicasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DemotePrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -12173,7 +14112,7 @@ func (m *GetReplicasRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetReplicasResponse) UnmarshalVT(dAtA []byte) error { +func (m *DemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12196,17 +14135,17 @@ func (m *GetReplicasResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetReplicasResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DemotePrimaryResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetReplicasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DemotePrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Addrs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryStatus", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12216,23 +14155,27 @@ func (m *GetReplicasResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Addrs = append(m.Addrs, string(dAtA[iNdEx:postIndex])) + if m.PrimaryStatus == nil { + m.PrimaryStatus = &replicationdata.PrimaryStatus{} + } + if err := m.PrimaryStatus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -12256,7 +14199,7 @@ func (m *GetReplicasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ResetReplicationRequest) UnmarshalVT(dAtA []byte) error { +func (m *UndoDemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12279,12 +14222,32 @@ func (m *ResetReplicationRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResetReplicationRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UndoDemotePrimaryRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResetReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UndoDemotePrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", 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.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12307,7 +14270,7 @@ func (m *ResetReplicationRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ResetReplicationResponse) UnmarshalVT(dAtA []byte) error { +func (m *UndoDemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12330,10 +14293,10 @@ func (m *ResetReplicationResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResetReplicationResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UndoDemotePrimaryResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResetReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UndoDemotePrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -12358,7 +14321,7 @@ func (m *ResetReplicationResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VReplicationExecRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReplicaWasPromotedRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12381,44 +14344,12 @@ func (m *VReplicationExecRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VReplicationExecRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReplicaWasPromotedRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VReplicationExecRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReplicaWasPromotedRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", 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.Query = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12441,7 +14372,7 @@ func (m *VReplicationExecRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VReplicationExecResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReplicaWasPromotedResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12464,48 +14395,12 @@ func (m *VReplicationExecResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VReplicationExecResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReplicaWasPromotedResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VReplicationExecResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReplicaWasPromotedResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", 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.Result == nil { - m.Result = &query.QueryResult{} - } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12528,7 +14423,7 @@ func (m *VReplicationExecResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VReplicationWaitForPosRequest) UnmarshalVT(dAtA []byte) error { +func (m *ResetReplicationParametersRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12541,73 +14436,22 @@ func (m *VReplicationWaitForPosRequest) UnmarshalVT(dAtA []byte) error { 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: VReplicationWaitForPosRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: VReplicationWaitForPosRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) - } - m.Id = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Id |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", 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.Position = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + 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: ResetReplicationParametersRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResetReplicationParametersRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12630,7 +14474,7 @@ func (m *VReplicationWaitForPosRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VReplicationWaitForPosResponse) UnmarshalVT(dAtA []byte) error { +func (m *ResetReplicationParametersResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12653,10 +14497,10 @@ func (m *VReplicationWaitForPosResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VReplicationWaitForPosResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ResetReplicationParametersResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VReplicationWaitForPosResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ResetReplicationParametersResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -12681,7 +14525,7 @@ func (m *VReplicationWaitForPosResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitPrimaryRequest) UnmarshalVT(dAtA []byte) error { +func (m *FullStatusRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12704,32 +14548,12 @@ func (m *InitPrimaryRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitPrimaryRequest: wiretype end group for non-group") + return fmt.Errorf("proto: FullStatusRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitPrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: FullStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", 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.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -12752,7 +14576,7 @@ func (m *InitPrimaryRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitPrimaryResponse) UnmarshalVT(dAtA []byte) error { +func (m *FullStatusResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12775,17 +14599,17 @@ func (m *InitPrimaryResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitPrimaryResponse: wiretype end group for non-group") + return fmt.Errorf("proto: FullStatusResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitPrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: FullStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12795,23 +14619,27 @@ func (m *InitPrimaryResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) + 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 @@ -12835,7 +14663,7 @@ func (m *InitPrimaryResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -12858,17 +14686,17 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PopulateReparentJournalRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetReplicationSourceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PopulateReparentJournalRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetReplicationSourceRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Parent", wireType) } - m.TimeCreatedNs = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12878,16 +14706,33 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TimeCreatedNs |= int64(b&0x7F) << shift + 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.Parent == nil { + m.Parent = &topodata.TabletAlias{} + } + if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ActionName", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) } - var stringLen uint64 + m.TimeCreatedNs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12897,29 +14742,16 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.TimeCreatedNs |= int64(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.ActionName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryAlias", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ForceStartReplication", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -12929,31 +14761,15 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.PrimaryAlias == nil { - m.PrimaryAlias = &topodata.TabletAlias{} - } - if err := m.PrimaryAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.ForceStartReplication = bool(v != 0) case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReplicationPosition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -12981,8 +14797,28 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ReplicationPosition = string(dAtA[iNdEx:postIndex]) + m.WaitPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", 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.SemiSync = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13005,7 +14841,7 @@ func (m *PopulateReparentJournalRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PopulateReparentJournalResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetReplicationSourceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13028,10 +14864,10 @@ func (m *PopulateReparentJournalResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PopulateReparentJournalResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetReplicationSourceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PopulateReparentJournalResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetReplicationSourceResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -13056,7 +14892,7 @@ func (m *PopulateReparentJournalResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitReplicaRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReplicaWasRestartedRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13079,10 +14915,10 @@ func (m *InitReplicaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitReplicaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReplicaWasRestartedRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitReplicaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReplicaWasRestartedRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -13099,99 +14935,28 @@ func (m *InitReplicaRequest) UnmarshalVT(dAtA []byte) error { } 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.Parent == nil { - m.Parent = &topodata.TabletAlias{} - } - if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReplicationPosition", 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.ReplicationPosition = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) - } - m.TimeCreatedNs = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TimeCreatedNs |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) + if msglen < 0 { + return ErrInvalidLength } - 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 - } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength } - m.SemiSync = bool(v != 0) + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Parent == nil { + m.Parent = &topodata.TabletAlias{} + } + if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13214,7 +14979,7 @@ func (m *InitReplicaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitReplicaResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReplicaWasRestartedResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13237,10 +15002,10 @@ func (m *InitReplicaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitReplicaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReplicaWasRestartedResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitReplicaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReplicaWasRestartedResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -13265,7 +15030,7 @@ func (m *InitReplicaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationAndGetStatusRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13288,12 +15053,31 @@ func (m *DemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DemotePrimaryRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StopReplicationAndGetStatusRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DemotePrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StopReplicationAndGetStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StopReplicationMode", wireType) + } + m.StopReplicationMode = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StopReplicationMode |= replicationdata.StopReplicationMode(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13316,7 +15100,7 @@ func (m *DemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationAndGetStatusResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13339,15 +15123,15 @@ func (m *DemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DemotePrimaryResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StopReplicationAndGetStatusResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DemotePrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StopReplicationAndGetStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryStatus", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -13374,10 +15158,10 @@ func (m *DemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.PrimaryStatus == nil { - m.PrimaryStatus = &replicationdata.PrimaryStatus{} + if m.Status == nil { + m.Status = &replicationdata.StopReplicationStatus{} } - if err := m.PrimaryStatus.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -13403,7 +15187,7 @@ func (m *DemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UndoDemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { +func (m *PromoteReplicaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13426,10 +15210,10 @@ func (m *UndoDemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UndoDemotePrimaryRequest: wiretype end group for non-group") + return fmt.Errorf("proto: PromoteReplicaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UndoDemotePrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PromoteReplicaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -13474,7 +15258,7 @@ func (m *UndoDemotePrimaryRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UndoDemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { +func (m *PromoteReplicaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13497,12 +15281,44 @@ func (m *UndoDemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UndoDemotePrimaryResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PromoteReplicaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UndoDemotePrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PromoteReplicaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Position", 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.Position = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13525,7 +15341,7 @@ func (m *UndoDemotePrimaryResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReplicaWasPromotedRequest) UnmarshalVT(dAtA []byte) error { +func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13548,63 +15364,103 @@ func (m *ReplicaWasPromotedRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReplicaWasPromotedRequest: wiretype end group for non-group") + return fmt.Errorf("proto: BackupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicaWasPromotedRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + m.Concurrency = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Concurrency |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ReplicaWasPromotedResponse) 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 + 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 + } } - if iNdEx >= l { + m.AllowPrimary = bool(v != 0) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IncrementalFromPos", 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 } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.IncrementalFromPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ReplicaWasPromotedResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicaWasPromotedResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + 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.UpgradeSafe = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13627,7 +15483,7 @@ func (m *ReplicaWasPromotedResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ResetReplicationParametersRequest) UnmarshalVT(dAtA []byte) error { +func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13650,12 +15506,48 @@ func (m *ResetReplicationParametersRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResetReplicationParametersRequest: wiretype end group for non-group") + return fmt.Errorf("proto: BackupResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResetReplicationParametersRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Event", 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.Event == nil { + m.Event = &logutil.Event{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13678,7 +15570,7 @@ func (m *ResetReplicationParametersRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ResetReplicationParametersResponse) UnmarshalVT(dAtA []byte) error { +func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13701,63 +15593,136 @@ func (m *ResetReplicationParametersResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResetReplicationParametersResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RestoreFromBackupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResetReplicationParametersResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RestoreFromBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackupTime", 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.BackupTime == nil { + m.BackupTime = &vttime.Time{} + } + if err := m.BackupTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestoreToPos", 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 } - if (iNdEx + skippy) > l { + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > 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 *FullStatusRequest) 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 + m.RestoreToPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) } - if iNdEx >= l { + 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.DryRun = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestoreToTimestamp", 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 } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.RestoreToTimestamp == nil { + m.RestoreToTimestamp = &vttime.Time{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: FullStatusRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: FullStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if err := m.RestoreToTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -13780,7 +15745,7 @@ func (m *FullStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *FullStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13803,15 +15768,15 @@ func (m *FullStatusResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: FullStatusResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RestoreFromBackupResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: FullStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RestoreFromBackupResponse: 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 Event", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -13838,10 +15803,10 @@ func (m *FullStatusResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Status == nil { - m.Status = &replicationdata.FullStatus{} + if m.Event == nil { + m.Event = &logutil.Event{} } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -13867,7 +15832,7 @@ func (m *FullStatusResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { +func (m *CreateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13886,19 +15851,51 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { if b < 0x80 { break } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SetReplicationSourceRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SetReplicationSourceRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateVReplicationWorkflowRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", 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.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Parent", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field BinlogSource", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -13925,18 +15922,16 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Parent == nil { - m.Parent = &topodata.TabletAlias{} - } - if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.BinlogSource = append(m.BinlogSource, &binlogdata.BinlogSource{}) + if err := m.BinlogSource[len(m.BinlogSource)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeCreatedNs", wireType) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } - m.TimeCreatedNs = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -13946,16 +15941,98 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TimeCreatedNs |= int64(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 3: + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType == 0 { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) + } + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ForceStartReplication", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) } - var v int + m.TabletSelectionPreference = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -13965,17 +16042,16 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift if b < 0x80 { break } } - m.ForceStartReplication = bool(v != 0) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", wireType) } - var stringLen uint64 + m.WorkflowType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -13985,27 +16061,33 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.WorkflowType |= binlogdata.VReplicationWorkflowType(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", wireType) } - if postIndex > l { - return io.ErrUnexpectedEOF + m.WorkflowSubType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WorkflowSubType |= binlogdata.VReplicationWorkflowSubType(b&0x7F) << shift + if b < 0x80 { + break + } } - m.WaitPosition = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: + case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -14022,58 +16104,47 @@ func (m *SetReplicationSourceRequest) UnmarshalVT(dAtA []byte) error { break } } - m.SemiSync = bool(v != 0) - 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.DeferSecondaryKeys = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AutoStart", wireType) } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SetReplicationSourceResponse) 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 + 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 + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + m.AutoStart = bool(v != 0) + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StopAfterCopy", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + 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 + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SetReplicationSourceResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SetReplicationSourceResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + m.StopAfterCopy = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14096,7 +16167,7 @@ func (m *SetReplicationSourceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReplicaWasRestartedRequest) UnmarshalVT(dAtA []byte) error { +func (m *CreateVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14119,15 +16190,15 @@ func (m *ReplicaWasRestartedRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReplicaWasRestartedRequest: wiretype end group for non-group") + return fmt.Errorf("proto: CreateVReplicationWorkflowResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicaWasRestartedRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CreateVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Parent", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -14154,10 +16225,10 @@ func (m *ReplicaWasRestartedRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Parent == nil { - m.Parent = &topodata.TabletAlias{} + if m.Result == nil { + m.Result = &query.QueryResult{} } - if err := m.Parent.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -14183,7 +16254,7 @@ func (m *ReplicaWasRestartedRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReplicaWasRestartedResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14206,12 +16277,44 @@ func (m *ReplicaWasRestartedResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReplicaWasRestartedResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReplicaWasRestartedResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", 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.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14234,7 +16337,7 @@ func (m *ReplicaWasRestartedResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationAndGetStatusRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14257,17 +16360,17 @@ func (m *StopReplicationAndGetStatusRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationAndGetStatusRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationAndGetStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StopReplicationMode", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } - m.StopReplicationMode = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14277,11 +16380,28 @@ func (m *StopReplicationAndGetStatusRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StopReplicationMode |= replicationdata.StopReplicationMode(b&0x7F) << shift + 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.Result == nil { + m.Result = &query.QueryResult{} + } + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14304,7 +16424,7 @@ func (m *StopReplicationAndGetStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationAndGetStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReadVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14327,17 +16447,17 @@ func (m *StopReplicationAndGetStatusResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationAndGetStatusResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReadVReplicationWorkflowRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationAndGetStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReadVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 2: + 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 Workflow", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14347,27 +16467,23 @@ func (m *StopReplicationAndGetStatusResponse) 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.Status == nil { - m.Status = &replicationdata.StopReplicationStatus{} - } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Workflow = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -14391,7 +16507,7 @@ func (m *StopReplicationAndGetStatusResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PromoteReplicaRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReadVReplicationWorkflowResponse_Stream) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14414,17 +16530,17 @@ func (m *PromoteReplicaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PromoteReplicaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReadVReplicationWorkflowResponse_Stream: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PromoteReplicaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReadVReplicationWorkflowResponse_Stream: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSync", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) } - var v int + m.Id = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14434,66 +16550,243 @@ func (m *PromoteReplicaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.Id |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.SemiSync = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Bls", 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.Bls == nil { + m.Bls = &binlogdata.BinlogSource{} + } + if err := m.Bls.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pos", 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 } - if (iNdEx + skippy) > l { + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > 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 *PromoteReplicaResponse) 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 + m.Pos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StopPos", 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.StopPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxTps", wireType) + } + m.MaxTps = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxTps |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxReplicationLag", wireType) + } + m.MaxReplicationLag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxReplicationLag |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeUpdated", 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.TimeUpdated == nil { + m.TimeUpdated = &vttime.Time{} + } + if err := m.TimeUpdated.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionTimestamp", 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 } - if iNdEx >= l { + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.TransactionTimestamp == nil { + m.TransactionTimestamp = &vttime.Time{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: PromoteReplicaResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PromoteReplicaResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + if err := m.TransactionTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + m.State = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.State |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14521,64 +16814,13 @@ func (m *PromoteReplicaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) + m.Message = 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 *BackupRequest) 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: BackupRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: BackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 11: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RowsCopied", wireType) } - m.Concurrency = 0 + m.RowsCopied = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14588,16 +16830,16 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Concurrency |= int64(b&0x7F) << shift + m.RowsCopied |= int64(b&0x7F) << shift if b < 0x80 { break } } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeHeartbeat", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14607,17 +16849,33 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.AllowPrimary = bool(v != 0) - case 3: + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TimeHeartbeat == nil { + m.TimeHeartbeat = &vttime.Time{} + } + if err := m.TimeHeartbeat.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IncrementalFromPos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TimeThrottled", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14627,29 +16885,33 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.IncrementalFromPos = string(dAtA[iNdEx:postIndex]) + if m.TimeThrottled == nil { + m.TimeThrottled = &vttime.Time{} + } + if err := m.TimeThrottled.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ComponentThrottled", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14659,12 +16921,24 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.UpgradeSafe = bool(v != 0) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ComponentThrottled = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14687,7 +16961,7 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14710,17 +16984,17 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BackupResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReadVReplicationWorkflowResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReadVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14730,84 +17004,149 @@ func (m *BackupResponse) 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.Event == nil { - m.Event = &logutil.Event{} + m.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } - if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + 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 + } } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - if (skippy < 0) || (iNdEx+skippy) < 0 { + postIndex := iNdEx + intStringLen + if postIndex < 0 { return ErrInvalidLength } - if (iNdEx + skippy) > l { + if postIndex > 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 *RestoreFromBackupRequest) 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 + m.Cells = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType == 0 { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } - if iNdEx >= l { - return io.ErrUnexpectedEOF + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.TabletSelectionPreference = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift + if b < 0x80 { + break + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: RestoreFromBackupRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RestoreFromBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BackupTime", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14817,31 +17156,27 @@ func (m *RestoreFromBackupRequest) 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.BackupTime == nil { - m.BackupTime = &vttime.Time{} - } - if err := m.BackupTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.DbName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RestoreToPos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tags", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14869,13 +17204,13 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.RestoreToPos = string(dAtA[iNdEx:postIndex]) + m.Tags = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", wireType) } - var v int + m.WorkflowType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14885,17 +17220,16 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.WorkflowType |= binlogdata.VReplicationWorkflowType(b&0x7F) << shift if b < 0x80 { break } } - m.DryRun = bool(v != 0) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RestoreToTimestamp", wireType) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", wireType) } - var msglen int + m.WorkflowSubType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14905,82 +17239,34 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.WorkflowSubType |= binlogdata.VReplicationWorkflowSubType(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.RestoreToTimestamp == nil { - m.RestoreToTimestamp = &vttime.Time{} - } - if err := m.RestoreToTimestamp.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 *RestoreFromBackupResponse) 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 + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + 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 + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: RestoreFromBackupResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RestoreFromBackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.DeferSecondaryKeys = bool(v != 0) + case 11: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15007,10 +17293,8 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Event == nil { - m.Event = &logutil.Event{} - } - if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Streams = append(m.Streams, &ReadVReplicationWorkflowResponse_Stream{}) + if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -16069,7 +18353,7 @@ func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16092,10 +18376,10 @@ func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateVRWorkflowRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateVRWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -16163,10 +18447,79 @@ func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 3: - if wireType != 2 { + if wireType == 0 { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } + } else { return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } - var stringLen uint64 + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) + } + m.TabletSelectionPreference = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16176,25 +18529,12 @@ func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.TabletSelectionPreference |= TabletSelectionPreference(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.TabletTypes = append(m.TabletTypes, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: + case 5: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", wireType) } @@ -16213,6 +18553,25 @@ func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { break } } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + m.State = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.State |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16235,7 +18594,7 @@ func (m *UpdateVRWorkflowRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateVRWorkflowResponse) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16258,10 +18617,10 @@ func (m *UpdateVRWorkflowResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateVRWorkflowResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateVRWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: diff --git a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go index e42173ae84b..1e409002098 100644 --- a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go +++ b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go @@ -45,7 +45,7 @@ var file_tabletmanagerservice_proto_rawDesc = []byte{ 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x1a, 0x17, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0x9e, 0x28, 0x0a, 0x0d, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xe1, 0x2b, 0x0a, 0x0d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x1e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, @@ -235,143 +235,171 @@ var file_tabletmanagerservice_proto_rawDesc = []byte{ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x6d, 0x0a, 0x10, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, - 0x78, 0x65, 0x63, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x8b, 0x01, 0x0a, 0x1a, 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, 0x12, 0x34, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x8b, 0x01, + 0x0a, 0x1a, 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, 0x12, 0x34, 0x2e, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, + 0x2e, 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, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x18, + 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x32, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, - 0x0a, 0x16, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, - 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x12, 0x30, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, - 0x50, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x74, 0x61, 0x62, + 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x12, 0x30, 0x2e, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, + 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, + 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, 0x0a, 0x1a, 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, 0x12, 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, 0x1a, 0x35, 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, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x4c, 0x0a, 0x05, 0x56, 0x44, 0x69, 0x66, 0x66, 0x12, 0x1f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, - 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x6d, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x12, 0x2a, 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, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, + 0x6d, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 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, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, - 0x0a, 0x05, 0x56, 0x44, 0x69, 0x66, 0x66, 0x12, 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, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, - 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x49, - 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x25, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, - 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, - 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, - 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x31, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, - 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, - 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x74, 0x61, 0x62, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, + 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x25, 0x2e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, + 0x01, 0x0a, 0x17, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x31, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4a, - 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x5e, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, - 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x64, 0x0a, 0x0d, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x12, 0x27, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, - 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, - 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x2b, 0x2e, 0x74, 0x61, + 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x12, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, + 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x12, 0x27, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x55, 0x6e, 0x64, + 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x2b, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, - 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x12, 0x2c, - 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, - 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x74, + 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, + 0x64, 0x12, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, + 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, + 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x8b, 0x01, 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, + 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, + 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, + 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x24, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, - 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, - 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, 0x34, 0x2e, 0x74, + 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, 0x53, + 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x12, 0x2d, 0x2e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0a, 0x46, - 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x24, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, - 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, 0x53, 0x65, 0x74, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x12, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, - 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x12, 0x2d, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, - 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, - 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x1b, - 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, - 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, - 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0e, - 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x28, + 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, + 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, - 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x06, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, - 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x72, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x74, - 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, 0x2b, 0x2e, + 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x67, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x12, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x06, 0x42, 0x61, 0x63, 0x6b, + 0x75, 0x70, 0x12, 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x72, 0x0a, 0x11, 0x52, + 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x12, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 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, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 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, 0x2c, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 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, 0x42, 0x33, 0x5a, 0x31, - 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, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, + 0x33, 0x5a, 0x31, 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, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_tabletmanagerservice_proto_goTypes = []interface{}{ @@ -404,176 +432,188 @@ var file_tabletmanagerservice_proto_goTypes = []interface{}{ (*tabletmanagerdata.StartReplicationRequest)(nil), // 26: tabletmanagerdata.StartReplicationRequest (*tabletmanagerdata.StartReplicationUntilAfterRequest)(nil), // 27: tabletmanagerdata.StartReplicationUntilAfterRequest (*tabletmanagerdata.GetReplicasRequest)(nil), // 28: tabletmanagerdata.GetReplicasRequest - (*tabletmanagerdata.VReplicationExecRequest)(nil), // 29: tabletmanagerdata.VReplicationExecRequest - (*tabletmanagerdata.VReplicationWaitForPosRequest)(nil), // 30: tabletmanagerdata.VReplicationWaitForPosRequest - (*tabletmanagerdata.UpdateVRWorkflowRequest)(nil), // 31: tabletmanagerdata.UpdateVRWorkflowRequest - (*tabletmanagerdata.VDiffRequest)(nil), // 32: tabletmanagerdata.VDiffRequest - (*tabletmanagerdata.ResetReplicationRequest)(nil), // 33: tabletmanagerdata.ResetReplicationRequest - (*tabletmanagerdata.InitPrimaryRequest)(nil), // 34: tabletmanagerdata.InitPrimaryRequest - (*tabletmanagerdata.PopulateReparentJournalRequest)(nil), // 35: tabletmanagerdata.PopulateReparentJournalRequest - (*tabletmanagerdata.InitReplicaRequest)(nil), // 36: tabletmanagerdata.InitReplicaRequest - (*tabletmanagerdata.DemotePrimaryRequest)(nil), // 37: tabletmanagerdata.DemotePrimaryRequest - (*tabletmanagerdata.UndoDemotePrimaryRequest)(nil), // 38: tabletmanagerdata.UndoDemotePrimaryRequest - (*tabletmanagerdata.ReplicaWasPromotedRequest)(nil), // 39: tabletmanagerdata.ReplicaWasPromotedRequest - (*tabletmanagerdata.ResetReplicationParametersRequest)(nil), // 40: tabletmanagerdata.ResetReplicationParametersRequest - (*tabletmanagerdata.FullStatusRequest)(nil), // 41: tabletmanagerdata.FullStatusRequest - (*tabletmanagerdata.SetReplicationSourceRequest)(nil), // 42: tabletmanagerdata.SetReplicationSourceRequest - (*tabletmanagerdata.ReplicaWasRestartedRequest)(nil), // 43: tabletmanagerdata.ReplicaWasRestartedRequest - (*tabletmanagerdata.StopReplicationAndGetStatusRequest)(nil), // 44: tabletmanagerdata.StopReplicationAndGetStatusRequest - (*tabletmanagerdata.PromoteReplicaRequest)(nil), // 45: tabletmanagerdata.PromoteReplicaRequest - (*tabletmanagerdata.BackupRequest)(nil), // 46: tabletmanagerdata.BackupRequest - (*tabletmanagerdata.RestoreFromBackupRequest)(nil), // 47: tabletmanagerdata.RestoreFromBackupRequest - (*tabletmanagerdata.PingResponse)(nil), // 48: tabletmanagerdata.PingResponse - (*tabletmanagerdata.SleepResponse)(nil), // 49: tabletmanagerdata.SleepResponse - (*tabletmanagerdata.ExecuteHookResponse)(nil), // 50: tabletmanagerdata.ExecuteHookResponse - (*tabletmanagerdata.GetSchemaResponse)(nil), // 51: tabletmanagerdata.GetSchemaResponse - (*tabletmanagerdata.GetPermissionsResponse)(nil), // 52: tabletmanagerdata.GetPermissionsResponse - (*tabletmanagerdata.SetReadOnlyResponse)(nil), // 53: tabletmanagerdata.SetReadOnlyResponse - (*tabletmanagerdata.SetReadWriteResponse)(nil), // 54: tabletmanagerdata.SetReadWriteResponse - (*tabletmanagerdata.ChangeTypeResponse)(nil), // 55: tabletmanagerdata.ChangeTypeResponse - (*tabletmanagerdata.RefreshStateResponse)(nil), // 56: tabletmanagerdata.RefreshStateResponse - (*tabletmanagerdata.RunHealthCheckResponse)(nil), // 57: tabletmanagerdata.RunHealthCheckResponse - (*tabletmanagerdata.ReloadSchemaResponse)(nil), // 58: tabletmanagerdata.ReloadSchemaResponse - (*tabletmanagerdata.PreflightSchemaResponse)(nil), // 59: tabletmanagerdata.PreflightSchemaResponse - (*tabletmanagerdata.ApplySchemaResponse)(nil), // 60: tabletmanagerdata.ApplySchemaResponse - (*tabletmanagerdata.ResetSequencesResponse)(nil), // 61: tabletmanagerdata.ResetSequencesResponse - (*tabletmanagerdata.LockTablesResponse)(nil), // 62: tabletmanagerdata.LockTablesResponse - (*tabletmanagerdata.UnlockTablesResponse)(nil), // 63: tabletmanagerdata.UnlockTablesResponse - (*tabletmanagerdata.ExecuteQueryResponse)(nil), // 64: tabletmanagerdata.ExecuteQueryResponse - (*tabletmanagerdata.ExecuteFetchAsDbaResponse)(nil), // 65: tabletmanagerdata.ExecuteFetchAsDbaResponse - (*tabletmanagerdata.ExecuteFetchAsAllPrivsResponse)(nil), // 66: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse - (*tabletmanagerdata.ExecuteFetchAsAppResponse)(nil), // 67: tabletmanagerdata.ExecuteFetchAsAppResponse - (*tabletmanagerdata.ReplicationStatusResponse)(nil), // 68: tabletmanagerdata.ReplicationStatusResponse - (*tabletmanagerdata.PrimaryStatusResponse)(nil), // 69: tabletmanagerdata.PrimaryStatusResponse - (*tabletmanagerdata.PrimaryPositionResponse)(nil), // 70: tabletmanagerdata.PrimaryPositionResponse - (*tabletmanagerdata.WaitForPositionResponse)(nil), // 71: tabletmanagerdata.WaitForPositionResponse - (*tabletmanagerdata.StopReplicationResponse)(nil), // 72: tabletmanagerdata.StopReplicationResponse - (*tabletmanagerdata.StopReplicationMinimumResponse)(nil), // 73: tabletmanagerdata.StopReplicationMinimumResponse - (*tabletmanagerdata.StartReplicationResponse)(nil), // 74: tabletmanagerdata.StartReplicationResponse - (*tabletmanagerdata.StartReplicationUntilAfterResponse)(nil), // 75: tabletmanagerdata.StartReplicationUntilAfterResponse - (*tabletmanagerdata.GetReplicasResponse)(nil), // 76: tabletmanagerdata.GetReplicasResponse - (*tabletmanagerdata.VReplicationExecResponse)(nil), // 77: tabletmanagerdata.VReplicationExecResponse - (*tabletmanagerdata.VReplicationWaitForPosResponse)(nil), // 78: tabletmanagerdata.VReplicationWaitForPosResponse - (*tabletmanagerdata.UpdateVRWorkflowResponse)(nil), // 79: tabletmanagerdata.UpdateVRWorkflowResponse - (*tabletmanagerdata.VDiffResponse)(nil), // 80: tabletmanagerdata.VDiffResponse - (*tabletmanagerdata.ResetReplicationResponse)(nil), // 81: tabletmanagerdata.ResetReplicationResponse - (*tabletmanagerdata.InitPrimaryResponse)(nil), // 82: tabletmanagerdata.InitPrimaryResponse - (*tabletmanagerdata.PopulateReparentJournalResponse)(nil), // 83: tabletmanagerdata.PopulateReparentJournalResponse - (*tabletmanagerdata.InitReplicaResponse)(nil), // 84: tabletmanagerdata.InitReplicaResponse - (*tabletmanagerdata.DemotePrimaryResponse)(nil), // 85: tabletmanagerdata.DemotePrimaryResponse - (*tabletmanagerdata.UndoDemotePrimaryResponse)(nil), // 86: tabletmanagerdata.UndoDemotePrimaryResponse - (*tabletmanagerdata.ReplicaWasPromotedResponse)(nil), // 87: tabletmanagerdata.ReplicaWasPromotedResponse - (*tabletmanagerdata.ResetReplicationParametersResponse)(nil), // 88: tabletmanagerdata.ResetReplicationParametersResponse - (*tabletmanagerdata.FullStatusResponse)(nil), // 89: tabletmanagerdata.FullStatusResponse - (*tabletmanagerdata.SetReplicationSourceResponse)(nil), // 90: tabletmanagerdata.SetReplicationSourceResponse - (*tabletmanagerdata.ReplicaWasRestartedResponse)(nil), // 91: tabletmanagerdata.ReplicaWasRestartedResponse - (*tabletmanagerdata.StopReplicationAndGetStatusResponse)(nil), // 92: tabletmanagerdata.StopReplicationAndGetStatusResponse - (*tabletmanagerdata.PromoteReplicaResponse)(nil), // 93: tabletmanagerdata.PromoteReplicaResponse - (*tabletmanagerdata.BackupResponse)(nil), // 94: tabletmanagerdata.BackupResponse - (*tabletmanagerdata.RestoreFromBackupResponse)(nil), // 95: tabletmanagerdata.RestoreFromBackupResponse + (*tabletmanagerdata.CreateVReplicationWorkflowRequest)(nil), // 29: tabletmanagerdata.CreateVReplicationWorkflowRequest + (*tabletmanagerdata.DeleteVReplicationWorkflowRequest)(nil), // 30: tabletmanagerdata.DeleteVReplicationWorkflowRequest + (*tabletmanagerdata.ReadVReplicationWorkflowRequest)(nil), // 31: tabletmanagerdata.ReadVReplicationWorkflowRequest + (*tabletmanagerdata.VReplicationExecRequest)(nil), // 32: tabletmanagerdata.VReplicationExecRequest + (*tabletmanagerdata.VReplicationWaitForPosRequest)(nil), // 33: tabletmanagerdata.VReplicationWaitForPosRequest + (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 34: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*tabletmanagerdata.VDiffRequest)(nil), // 35: tabletmanagerdata.VDiffRequest + (*tabletmanagerdata.ResetReplicationRequest)(nil), // 36: tabletmanagerdata.ResetReplicationRequest + (*tabletmanagerdata.InitPrimaryRequest)(nil), // 37: tabletmanagerdata.InitPrimaryRequest + (*tabletmanagerdata.PopulateReparentJournalRequest)(nil), // 38: tabletmanagerdata.PopulateReparentJournalRequest + (*tabletmanagerdata.InitReplicaRequest)(nil), // 39: tabletmanagerdata.InitReplicaRequest + (*tabletmanagerdata.DemotePrimaryRequest)(nil), // 40: tabletmanagerdata.DemotePrimaryRequest + (*tabletmanagerdata.UndoDemotePrimaryRequest)(nil), // 41: tabletmanagerdata.UndoDemotePrimaryRequest + (*tabletmanagerdata.ReplicaWasPromotedRequest)(nil), // 42: tabletmanagerdata.ReplicaWasPromotedRequest + (*tabletmanagerdata.ResetReplicationParametersRequest)(nil), // 43: tabletmanagerdata.ResetReplicationParametersRequest + (*tabletmanagerdata.FullStatusRequest)(nil), // 44: tabletmanagerdata.FullStatusRequest + (*tabletmanagerdata.SetReplicationSourceRequest)(nil), // 45: tabletmanagerdata.SetReplicationSourceRequest + (*tabletmanagerdata.ReplicaWasRestartedRequest)(nil), // 46: tabletmanagerdata.ReplicaWasRestartedRequest + (*tabletmanagerdata.StopReplicationAndGetStatusRequest)(nil), // 47: tabletmanagerdata.StopReplicationAndGetStatusRequest + (*tabletmanagerdata.PromoteReplicaRequest)(nil), // 48: tabletmanagerdata.PromoteReplicaRequest + (*tabletmanagerdata.BackupRequest)(nil), // 49: tabletmanagerdata.BackupRequest + (*tabletmanagerdata.RestoreFromBackupRequest)(nil), // 50: tabletmanagerdata.RestoreFromBackupRequest + (*tabletmanagerdata.PingResponse)(nil), // 51: tabletmanagerdata.PingResponse + (*tabletmanagerdata.SleepResponse)(nil), // 52: tabletmanagerdata.SleepResponse + (*tabletmanagerdata.ExecuteHookResponse)(nil), // 53: tabletmanagerdata.ExecuteHookResponse + (*tabletmanagerdata.GetSchemaResponse)(nil), // 54: tabletmanagerdata.GetSchemaResponse + (*tabletmanagerdata.GetPermissionsResponse)(nil), // 55: tabletmanagerdata.GetPermissionsResponse + (*tabletmanagerdata.SetReadOnlyResponse)(nil), // 56: tabletmanagerdata.SetReadOnlyResponse + (*tabletmanagerdata.SetReadWriteResponse)(nil), // 57: tabletmanagerdata.SetReadWriteResponse + (*tabletmanagerdata.ChangeTypeResponse)(nil), // 58: tabletmanagerdata.ChangeTypeResponse + (*tabletmanagerdata.RefreshStateResponse)(nil), // 59: tabletmanagerdata.RefreshStateResponse + (*tabletmanagerdata.RunHealthCheckResponse)(nil), // 60: tabletmanagerdata.RunHealthCheckResponse + (*tabletmanagerdata.ReloadSchemaResponse)(nil), // 61: tabletmanagerdata.ReloadSchemaResponse + (*tabletmanagerdata.PreflightSchemaResponse)(nil), // 62: tabletmanagerdata.PreflightSchemaResponse + (*tabletmanagerdata.ApplySchemaResponse)(nil), // 63: tabletmanagerdata.ApplySchemaResponse + (*tabletmanagerdata.ResetSequencesResponse)(nil), // 64: tabletmanagerdata.ResetSequencesResponse + (*tabletmanagerdata.LockTablesResponse)(nil), // 65: tabletmanagerdata.LockTablesResponse + (*tabletmanagerdata.UnlockTablesResponse)(nil), // 66: tabletmanagerdata.UnlockTablesResponse + (*tabletmanagerdata.ExecuteQueryResponse)(nil), // 67: tabletmanagerdata.ExecuteQueryResponse + (*tabletmanagerdata.ExecuteFetchAsDbaResponse)(nil), // 68: tabletmanagerdata.ExecuteFetchAsDbaResponse + (*tabletmanagerdata.ExecuteFetchAsAllPrivsResponse)(nil), // 69: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse + (*tabletmanagerdata.ExecuteFetchAsAppResponse)(nil), // 70: tabletmanagerdata.ExecuteFetchAsAppResponse + (*tabletmanagerdata.ReplicationStatusResponse)(nil), // 71: tabletmanagerdata.ReplicationStatusResponse + (*tabletmanagerdata.PrimaryStatusResponse)(nil), // 72: tabletmanagerdata.PrimaryStatusResponse + (*tabletmanagerdata.PrimaryPositionResponse)(nil), // 73: tabletmanagerdata.PrimaryPositionResponse + (*tabletmanagerdata.WaitForPositionResponse)(nil), // 74: tabletmanagerdata.WaitForPositionResponse + (*tabletmanagerdata.StopReplicationResponse)(nil), // 75: tabletmanagerdata.StopReplicationResponse + (*tabletmanagerdata.StopReplicationMinimumResponse)(nil), // 76: tabletmanagerdata.StopReplicationMinimumResponse + (*tabletmanagerdata.StartReplicationResponse)(nil), // 77: tabletmanagerdata.StartReplicationResponse + (*tabletmanagerdata.StartReplicationUntilAfterResponse)(nil), // 78: tabletmanagerdata.StartReplicationUntilAfterResponse + (*tabletmanagerdata.GetReplicasResponse)(nil), // 79: tabletmanagerdata.GetReplicasResponse + (*tabletmanagerdata.CreateVReplicationWorkflowResponse)(nil), // 80: tabletmanagerdata.CreateVReplicationWorkflowResponse + (*tabletmanagerdata.DeleteVReplicationWorkflowResponse)(nil), // 81: tabletmanagerdata.DeleteVReplicationWorkflowResponse + (*tabletmanagerdata.ReadVReplicationWorkflowResponse)(nil), // 82: tabletmanagerdata.ReadVReplicationWorkflowResponse + (*tabletmanagerdata.VReplicationExecResponse)(nil), // 83: tabletmanagerdata.VReplicationExecResponse + (*tabletmanagerdata.VReplicationWaitForPosResponse)(nil), // 84: tabletmanagerdata.VReplicationWaitForPosResponse + (*tabletmanagerdata.UpdateVReplicationWorkflowResponse)(nil), // 85: tabletmanagerdata.UpdateVReplicationWorkflowResponse + (*tabletmanagerdata.VDiffResponse)(nil), // 86: tabletmanagerdata.VDiffResponse + (*tabletmanagerdata.ResetReplicationResponse)(nil), // 87: tabletmanagerdata.ResetReplicationResponse + (*tabletmanagerdata.InitPrimaryResponse)(nil), // 88: tabletmanagerdata.InitPrimaryResponse + (*tabletmanagerdata.PopulateReparentJournalResponse)(nil), // 89: tabletmanagerdata.PopulateReparentJournalResponse + (*tabletmanagerdata.InitReplicaResponse)(nil), // 90: tabletmanagerdata.InitReplicaResponse + (*tabletmanagerdata.DemotePrimaryResponse)(nil), // 91: tabletmanagerdata.DemotePrimaryResponse + (*tabletmanagerdata.UndoDemotePrimaryResponse)(nil), // 92: tabletmanagerdata.UndoDemotePrimaryResponse + (*tabletmanagerdata.ReplicaWasPromotedResponse)(nil), // 93: tabletmanagerdata.ReplicaWasPromotedResponse + (*tabletmanagerdata.ResetReplicationParametersResponse)(nil), // 94: tabletmanagerdata.ResetReplicationParametersResponse + (*tabletmanagerdata.FullStatusResponse)(nil), // 95: tabletmanagerdata.FullStatusResponse + (*tabletmanagerdata.SetReplicationSourceResponse)(nil), // 96: tabletmanagerdata.SetReplicationSourceResponse + (*tabletmanagerdata.ReplicaWasRestartedResponse)(nil), // 97: tabletmanagerdata.ReplicaWasRestartedResponse + (*tabletmanagerdata.StopReplicationAndGetStatusResponse)(nil), // 98: tabletmanagerdata.StopReplicationAndGetStatusResponse + (*tabletmanagerdata.PromoteReplicaResponse)(nil), // 99: tabletmanagerdata.PromoteReplicaResponse + (*tabletmanagerdata.BackupResponse)(nil), // 100: tabletmanagerdata.BackupResponse + (*tabletmanagerdata.RestoreFromBackupResponse)(nil), // 101: tabletmanagerdata.RestoreFromBackupResponse } var file_tabletmanagerservice_proto_depIdxs = []int32{ - 0, // 0: tabletmanagerservice.TabletManager.Ping:input_type -> tabletmanagerdata.PingRequest - 1, // 1: tabletmanagerservice.TabletManager.Sleep:input_type -> tabletmanagerdata.SleepRequest - 2, // 2: tabletmanagerservice.TabletManager.ExecuteHook:input_type -> tabletmanagerdata.ExecuteHookRequest - 3, // 3: tabletmanagerservice.TabletManager.GetSchema:input_type -> tabletmanagerdata.GetSchemaRequest - 4, // 4: tabletmanagerservice.TabletManager.GetPermissions:input_type -> tabletmanagerdata.GetPermissionsRequest - 5, // 5: tabletmanagerservice.TabletManager.SetReadOnly:input_type -> tabletmanagerdata.SetReadOnlyRequest - 6, // 6: tabletmanagerservice.TabletManager.SetReadWrite:input_type -> tabletmanagerdata.SetReadWriteRequest - 7, // 7: tabletmanagerservice.TabletManager.ChangeType:input_type -> tabletmanagerdata.ChangeTypeRequest - 8, // 8: tabletmanagerservice.TabletManager.RefreshState:input_type -> tabletmanagerdata.RefreshStateRequest - 9, // 9: tabletmanagerservice.TabletManager.RunHealthCheck:input_type -> tabletmanagerdata.RunHealthCheckRequest - 10, // 10: tabletmanagerservice.TabletManager.ReloadSchema:input_type -> tabletmanagerdata.ReloadSchemaRequest - 11, // 11: tabletmanagerservice.TabletManager.PreflightSchema:input_type -> tabletmanagerdata.PreflightSchemaRequest - 12, // 12: tabletmanagerservice.TabletManager.ApplySchema:input_type -> tabletmanagerdata.ApplySchemaRequest - 13, // 13: tabletmanagerservice.TabletManager.ResetSequences:input_type -> tabletmanagerdata.ResetSequencesRequest - 14, // 14: tabletmanagerservice.TabletManager.LockTables:input_type -> tabletmanagerdata.LockTablesRequest - 15, // 15: tabletmanagerservice.TabletManager.UnlockTables:input_type -> tabletmanagerdata.UnlockTablesRequest - 16, // 16: tabletmanagerservice.TabletManager.ExecuteQuery:input_type -> tabletmanagerdata.ExecuteQueryRequest - 17, // 17: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:input_type -> tabletmanagerdata.ExecuteFetchAsDbaRequest - 18, // 18: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:input_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsRequest - 19, // 19: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:input_type -> tabletmanagerdata.ExecuteFetchAsAppRequest - 20, // 20: tabletmanagerservice.TabletManager.ReplicationStatus:input_type -> tabletmanagerdata.ReplicationStatusRequest - 21, // 21: tabletmanagerservice.TabletManager.PrimaryStatus:input_type -> tabletmanagerdata.PrimaryStatusRequest - 22, // 22: tabletmanagerservice.TabletManager.PrimaryPosition:input_type -> tabletmanagerdata.PrimaryPositionRequest - 23, // 23: tabletmanagerservice.TabletManager.WaitForPosition:input_type -> tabletmanagerdata.WaitForPositionRequest - 24, // 24: tabletmanagerservice.TabletManager.StopReplication:input_type -> tabletmanagerdata.StopReplicationRequest - 25, // 25: tabletmanagerservice.TabletManager.StopReplicationMinimum:input_type -> tabletmanagerdata.StopReplicationMinimumRequest - 26, // 26: tabletmanagerservice.TabletManager.StartReplication:input_type -> tabletmanagerdata.StartReplicationRequest - 27, // 27: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:input_type -> tabletmanagerdata.StartReplicationUntilAfterRequest - 28, // 28: tabletmanagerservice.TabletManager.GetReplicas:input_type -> tabletmanagerdata.GetReplicasRequest - 29, // 29: tabletmanagerservice.TabletManager.VReplicationExec:input_type -> tabletmanagerdata.VReplicationExecRequest - 30, // 30: tabletmanagerservice.TabletManager.VReplicationWaitForPos:input_type -> tabletmanagerdata.VReplicationWaitForPosRequest - 31, // 31: tabletmanagerservice.TabletManager.UpdateVRWorkflow:input_type -> tabletmanagerdata.UpdateVRWorkflowRequest - 32, // 32: tabletmanagerservice.TabletManager.VDiff:input_type -> tabletmanagerdata.VDiffRequest - 33, // 33: tabletmanagerservice.TabletManager.ResetReplication:input_type -> tabletmanagerdata.ResetReplicationRequest - 34, // 34: tabletmanagerservice.TabletManager.InitPrimary:input_type -> tabletmanagerdata.InitPrimaryRequest - 35, // 35: tabletmanagerservice.TabletManager.PopulateReparentJournal:input_type -> tabletmanagerdata.PopulateReparentJournalRequest - 36, // 36: tabletmanagerservice.TabletManager.InitReplica:input_type -> tabletmanagerdata.InitReplicaRequest - 37, // 37: tabletmanagerservice.TabletManager.DemotePrimary:input_type -> tabletmanagerdata.DemotePrimaryRequest - 38, // 38: tabletmanagerservice.TabletManager.UndoDemotePrimary:input_type -> tabletmanagerdata.UndoDemotePrimaryRequest - 39, // 39: tabletmanagerservice.TabletManager.ReplicaWasPromoted:input_type -> tabletmanagerdata.ReplicaWasPromotedRequest - 40, // 40: tabletmanagerservice.TabletManager.ResetReplicationParameters:input_type -> tabletmanagerdata.ResetReplicationParametersRequest - 41, // 41: tabletmanagerservice.TabletManager.FullStatus:input_type -> tabletmanagerdata.FullStatusRequest - 42, // 42: tabletmanagerservice.TabletManager.SetReplicationSource:input_type -> tabletmanagerdata.SetReplicationSourceRequest - 43, // 43: tabletmanagerservice.TabletManager.ReplicaWasRestarted:input_type -> tabletmanagerdata.ReplicaWasRestartedRequest - 44, // 44: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:input_type -> tabletmanagerdata.StopReplicationAndGetStatusRequest - 45, // 45: tabletmanagerservice.TabletManager.PromoteReplica:input_type -> tabletmanagerdata.PromoteReplicaRequest - 46, // 46: tabletmanagerservice.TabletManager.Backup:input_type -> tabletmanagerdata.BackupRequest - 47, // 47: tabletmanagerservice.TabletManager.RestoreFromBackup:input_type -> tabletmanagerdata.RestoreFromBackupRequest - 48, // 48: tabletmanagerservice.TabletManager.Ping:output_type -> tabletmanagerdata.PingResponse - 49, // 49: tabletmanagerservice.TabletManager.Sleep:output_type -> tabletmanagerdata.SleepResponse - 50, // 50: tabletmanagerservice.TabletManager.ExecuteHook:output_type -> tabletmanagerdata.ExecuteHookResponse - 51, // 51: tabletmanagerservice.TabletManager.GetSchema:output_type -> tabletmanagerdata.GetSchemaResponse - 52, // 52: tabletmanagerservice.TabletManager.GetPermissions:output_type -> tabletmanagerdata.GetPermissionsResponse - 53, // 53: tabletmanagerservice.TabletManager.SetReadOnly:output_type -> tabletmanagerdata.SetReadOnlyResponse - 54, // 54: tabletmanagerservice.TabletManager.SetReadWrite:output_type -> tabletmanagerdata.SetReadWriteResponse - 55, // 55: tabletmanagerservice.TabletManager.ChangeType:output_type -> tabletmanagerdata.ChangeTypeResponse - 56, // 56: tabletmanagerservice.TabletManager.RefreshState:output_type -> tabletmanagerdata.RefreshStateResponse - 57, // 57: tabletmanagerservice.TabletManager.RunHealthCheck:output_type -> tabletmanagerdata.RunHealthCheckResponse - 58, // 58: tabletmanagerservice.TabletManager.ReloadSchema:output_type -> tabletmanagerdata.ReloadSchemaResponse - 59, // 59: tabletmanagerservice.TabletManager.PreflightSchema:output_type -> tabletmanagerdata.PreflightSchemaResponse - 60, // 60: tabletmanagerservice.TabletManager.ApplySchema:output_type -> tabletmanagerdata.ApplySchemaResponse - 61, // 61: tabletmanagerservice.TabletManager.ResetSequences:output_type -> tabletmanagerdata.ResetSequencesResponse - 62, // 62: tabletmanagerservice.TabletManager.LockTables:output_type -> tabletmanagerdata.LockTablesResponse - 63, // 63: tabletmanagerservice.TabletManager.UnlockTables:output_type -> tabletmanagerdata.UnlockTablesResponse - 64, // 64: tabletmanagerservice.TabletManager.ExecuteQuery:output_type -> tabletmanagerdata.ExecuteQueryResponse - 65, // 65: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:output_type -> tabletmanagerdata.ExecuteFetchAsDbaResponse - 66, // 66: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:output_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsResponse - 67, // 67: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:output_type -> tabletmanagerdata.ExecuteFetchAsAppResponse - 68, // 68: tabletmanagerservice.TabletManager.ReplicationStatus:output_type -> tabletmanagerdata.ReplicationStatusResponse - 69, // 69: tabletmanagerservice.TabletManager.PrimaryStatus:output_type -> tabletmanagerdata.PrimaryStatusResponse - 70, // 70: tabletmanagerservice.TabletManager.PrimaryPosition:output_type -> tabletmanagerdata.PrimaryPositionResponse - 71, // 71: tabletmanagerservice.TabletManager.WaitForPosition:output_type -> tabletmanagerdata.WaitForPositionResponse - 72, // 72: tabletmanagerservice.TabletManager.StopReplication:output_type -> tabletmanagerdata.StopReplicationResponse - 73, // 73: tabletmanagerservice.TabletManager.StopReplicationMinimum:output_type -> tabletmanagerdata.StopReplicationMinimumResponse - 74, // 74: tabletmanagerservice.TabletManager.StartReplication:output_type -> tabletmanagerdata.StartReplicationResponse - 75, // 75: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:output_type -> tabletmanagerdata.StartReplicationUntilAfterResponse - 76, // 76: tabletmanagerservice.TabletManager.GetReplicas:output_type -> tabletmanagerdata.GetReplicasResponse - 77, // 77: tabletmanagerservice.TabletManager.VReplicationExec:output_type -> tabletmanagerdata.VReplicationExecResponse - 78, // 78: tabletmanagerservice.TabletManager.VReplicationWaitForPos:output_type -> tabletmanagerdata.VReplicationWaitForPosResponse - 79, // 79: tabletmanagerservice.TabletManager.UpdateVRWorkflow:output_type -> tabletmanagerdata.UpdateVRWorkflowResponse - 80, // 80: tabletmanagerservice.TabletManager.VDiff:output_type -> tabletmanagerdata.VDiffResponse - 81, // 81: tabletmanagerservice.TabletManager.ResetReplication:output_type -> tabletmanagerdata.ResetReplicationResponse - 82, // 82: tabletmanagerservice.TabletManager.InitPrimary:output_type -> tabletmanagerdata.InitPrimaryResponse - 83, // 83: tabletmanagerservice.TabletManager.PopulateReparentJournal:output_type -> tabletmanagerdata.PopulateReparentJournalResponse - 84, // 84: tabletmanagerservice.TabletManager.InitReplica:output_type -> tabletmanagerdata.InitReplicaResponse - 85, // 85: tabletmanagerservice.TabletManager.DemotePrimary:output_type -> tabletmanagerdata.DemotePrimaryResponse - 86, // 86: tabletmanagerservice.TabletManager.UndoDemotePrimary:output_type -> tabletmanagerdata.UndoDemotePrimaryResponse - 87, // 87: tabletmanagerservice.TabletManager.ReplicaWasPromoted:output_type -> tabletmanagerdata.ReplicaWasPromotedResponse - 88, // 88: tabletmanagerservice.TabletManager.ResetReplicationParameters:output_type -> tabletmanagerdata.ResetReplicationParametersResponse - 89, // 89: tabletmanagerservice.TabletManager.FullStatus:output_type -> tabletmanagerdata.FullStatusResponse - 90, // 90: tabletmanagerservice.TabletManager.SetReplicationSource:output_type -> tabletmanagerdata.SetReplicationSourceResponse - 91, // 91: tabletmanagerservice.TabletManager.ReplicaWasRestarted:output_type -> tabletmanagerdata.ReplicaWasRestartedResponse - 92, // 92: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:output_type -> tabletmanagerdata.StopReplicationAndGetStatusResponse - 93, // 93: tabletmanagerservice.TabletManager.PromoteReplica:output_type -> tabletmanagerdata.PromoteReplicaResponse - 94, // 94: tabletmanagerservice.TabletManager.Backup:output_type -> tabletmanagerdata.BackupResponse - 95, // 95: tabletmanagerservice.TabletManager.RestoreFromBackup:output_type -> tabletmanagerdata.RestoreFromBackupResponse - 48, // [48:96] is the sub-list for method output_type - 0, // [0:48] 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 + 0, // 0: tabletmanagerservice.TabletManager.Ping:input_type -> tabletmanagerdata.PingRequest + 1, // 1: tabletmanagerservice.TabletManager.Sleep:input_type -> tabletmanagerdata.SleepRequest + 2, // 2: tabletmanagerservice.TabletManager.ExecuteHook:input_type -> tabletmanagerdata.ExecuteHookRequest + 3, // 3: tabletmanagerservice.TabletManager.GetSchema:input_type -> tabletmanagerdata.GetSchemaRequest + 4, // 4: tabletmanagerservice.TabletManager.GetPermissions:input_type -> tabletmanagerdata.GetPermissionsRequest + 5, // 5: tabletmanagerservice.TabletManager.SetReadOnly:input_type -> tabletmanagerdata.SetReadOnlyRequest + 6, // 6: tabletmanagerservice.TabletManager.SetReadWrite:input_type -> tabletmanagerdata.SetReadWriteRequest + 7, // 7: tabletmanagerservice.TabletManager.ChangeType:input_type -> tabletmanagerdata.ChangeTypeRequest + 8, // 8: tabletmanagerservice.TabletManager.RefreshState:input_type -> tabletmanagerdata.RefreshStateRequest + 9, // 9: tabletmanagerservice.TabletManager.RunHealthCheck:input_type -> tabletmanagerdata.RunHealthCheckRequest + 10, // 10: tabletmanagerservice.TabletManager.ReloadSchema:input_type -> tabletmanagerdata.ReloadSchemaRequest + 11, // 11: tabletmanagerservice.TabletManager.PreflightSchema:input_type -> tabletmanagerdata.PreflightSchemaRequest + 12, // 12: tabletmanagerservice.TabletManager.ApplySchema:input_type -> tabletmanagerdata.ApplySchemaRequest + 13, // 13: tabletmanagerservice.TabletManager.ResetSequences:input_type -> tabletmanagerdata.ResetSequencesRequest + 14, // 14: tabletmanagerservice.TabletManager.LockTables:input_type -> tabletmanagerdata.LockTablesRequest + 15, // 15: tabletmanagerservice.TabletManager.UnlockTables:input_type -> tabletmanagerdata.UnlockTablesRequest + 16, // 16: tabletmanagerservice.TabletManager.ExecuteQuery:input_type -> tabletmanagerdata.ExecuteQueryRequest + 17, // 17: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:input_type -> tabletmanagerdata.ExecuteFetchAsDbaRequest + 18, // 18: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:input_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsRequest + 19, // 19: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:input_type -> tabletmanagerdata.ExecuteFetchAsAppRequest + 20, // 20: tabletmanagerservice.TabletManager.ReplicationStatus:input_type -> tabletmanagerdata.ReplicationStatusRequest + 21, // 21: tabletmanagerservice.TabletManager.PrimaryStatus:input_type -> tabletmanagerdata.PrimaryStatusRequest + 22, // 22: tabletmanagerservice.TabletManager.PrimaryPosition:input_type -> tabletmanagerdata.PrimaryPositionRequest + 23, // 23: tabletmanagerservice.TabletManager.WaitForPosition:input_type -> tabletmanagerdata.WaitForPositionRequest + 24, // 24: tabletmanagerservice.TabletManager.StopReplication:input_type -> tabletmanagerdata.StopReplicationRequest + 25, // 25: tabletmanagerservice.TabletManager.StopReplicationMinimum:input_type -> tabletmanagerdata.StopReplicationMinimumRequest + 26, // 26: tabletmanagerservice.TabletManager.StartReplication:input_type -> tabletmanagerdata.StartReplicationRequest + 27, // 27: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:input_type -> tabletmanagerdata.StartReplicationUntilAfterRequest + 28, // 28: tabletmanagerservice.TabletManager.GetReplicas:input_type -> tabletmanagerdata.GetReplicasRequest + 29, // 29: tabletmanagerservice.TabletManager.CreateVReplicationWorkflow:input_type -> tabletmanagerdata.CreateVReplicationWorkflowRequest + 30, // 30: tabletmanagerservice.TabletManager.DeleteVReplicationWorkflow:input_type -> tabletmanagerdata.DeleteVReplicationWorkflowRequest + 31, // 31: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:input_type -> tabletmanagerdata.ReadVReplicationWorkflowRequest + 32, // 32: tabletmanagerservice.TabletManager.VReplicationExec:input_type -> tabletmanagerdata.VReplicationExecRequest + 33, // 33: tabletmanagerservice.TabletManager.VReplicationWaitForPos:input_type -> tabletmanagerdata.VReplicationWaitForPosRequest + 34, // 34: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:input_type -> tabletmanagerdata.UpdateVReplicationWorkflowRequest + 35, // 35: tabletmanagerservice.TabletManager.VDiff:input_type -> tabletmanagerdata.VDiffRequest + 36, // 36: tabletmanagerservice.TabletManager.ResetReplication:input_type -> tabletmanagerdata.ResetReplicationRequest + 37, // 37: tabletmanagerservice.TabletManager.InitPrimary:input_type -> tabletmanagerdata.InitPrimaryRequest + 38, // 38: tabletmanagerservice.TabletManager.PopulateReparentJournal:input_type -> tabletmanagerdata.PopulateReparentJournalRequest + 39, // 39: tabletmanagerservice.TabletManager.InitReplica:input_type -> tabletmanagerdata.InitReplicaRequest + 40, // 40: tabletmanagerservice.TabletManager.DemotePrimary:input_type -> tabletmanagerdata.DemotePrimaryRequest + 41, // 41: tabletmanagerservice.TabletManager.UndoDemotePrimary:input_type -> tabletmanagerdata.UndoDemotePrimaryRequest + 42, // 42: tabletmanagerservice.TabletManager.ReplicaWasPromoted:input_type -> tabletmanagerdata.ReplicaWasPromotedRequest + 43, // 43: tabletmanagerservice.TabletManager.ResetReplicationParameters:input_type -> tabletmanagerdata.ResetReplicationParametersRequest + 44, // 44: tabletmanagerservice.TabletManager.FullStatus:input_type -> tabletmanagerdata.FullStatusRequest + 45, // 45: tabletmanagerservice.TabletManager.SetReplicationSource:input_type -> tabletmanagerdata.SetReplicationSourceRequest + 46, // 46: tabletmanagerservice.TabletManager.ReplicaWasRestarted:input_type -> tabletmanagerdata.ReplicaWasRestartedRequest + 47, // 47: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:input_type -> tabletmanagerdata.StopReplicationAndGetStatusRequest + 48, // 48: tabletmanagerservice.TabletManager.PromoteReplica:input_type -> tabletmanagerdata.PromoteReplicaRequest + 49, // 49: tabletmanagerservice.TabletManager.Backup:input_type -> tabletmanagerdata.BackupRequest + 50, // 50: tabletmanagerservice.TabletManager.RestoreFromBackup:input_type -> tabletmanagerdata.RestoreFromBackupRequest + 51, // 51: tabletmanagerservice.TabletManager.Ping:output_type -> tabletmanagerdata.PingResponse + 52, // 52: tabletmanagerservice.TabletManager.Sleep:output_type -> tabletmanagerdata.SleepResponse + 53, // 53: tabletmanagerservice.TabletManager.ExecuteHook:output_type -> tabletmanagerdata.ExecuteHookResponse + 54, // 54: tabletmanagerservice.TabletManager.GetSchema:output_type -> tabletmanagerdata.GetSchemaResponse + 55, // 55: tabletmanagerservice.TabletManager.GetPermissions:output_type -> tabletmanagerdata.GetPermissionsResponse + 56, // 56: tabletmanagerservice.TabletManager.SetReadOnly:output_type -> tabletmanagerdata.SetReadOnlyResponse + 57, // 57: tabletmanagerservice.TabletManager.SetReadWrite:output_type -> tabletmanagerdata.SetReadWriteResponse + 58, // 58: tabletmanagerservice.TabletManager.ChangeType:output_type -> tabletmanagerdata.ChangeTypeResponse + 59, // 59: tabletmanagerservice.TabletManager.RefreshState:output_type -> tabletmanagerdata.RefreshStateResponse + 60, // 60: tabletmanagerservice.TabletManager.RunHealthCheck:output_type -> tabletmanagerdata.RunHealthCheckResponse + 61, // 61: tabletmanagerservice.TabletManager.ReloadSchema:output_type -> tabletmanagerdata.ReloadSchemaResponse + 62, // 62: tabletmanagerservice.TabletManager.PreflightSchema:output_type -> tabletmanagerdata.PreflightSchemaResponse + 63, // 63: tabletmanagerservice.TabletManager.ApplySchema:output_type -> tabletmanagerdata.ApplySchemaResponse + 64, // 64: tabletmanagerservice.TabletManager.ResetSequences:output_type -> tabletmanagerdata.ResetSequencesResponse + 65, // 65: tabletmanagerservice.TabletManager.LockTables:output_type -> tabletmanagerdata.LockTablesResponse + 66, // 66: tabletmanagerservice.TabletManager.UnlockTables:output_type -> tabletmanagerdata.UnlockTablesResponse + 67, // 67: tabletmanagerservice.TabletManager.ExecuteQuery:output_type -> tabletmanagerdata.ExecuteQueryResponse + 68, // 68: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:output_type -> tabletmanagerdata.ExecuteFetchAsDbaResponse + 69, // 69: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:output_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsResponse + 70, // 70: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:output_type -> tabletmanagerdata.ExecuteFetchAsAppResponse + 71, // 71: tabletmanagerservice.TabletManager.ReplicationStatus:output_type -> tabletmanagerdata.ReplicationStatusResponse + 72, // 72: tabletmanagerservice.TabletManager.PrimaryStatus:output_type -> tabletmanagerdata.PrimaryStatusResponse + 73, // 73: tabletmanagerservice.TabletManager.PrimaryPosition:output_type -> tabletmanagerdata.PrimaryPositionResponse + 74, // 74: tabletmanagerservice.TabletManager.WaitForPosition:output_type -> tabletmanagerdata.WaitForPositionResponse + 75, // 75: tabletmanagerservice.TabletManager.StopReplication:output_type -> tabletmanagerdata.StopReplicationResponse + 76, // 76: tabletmanagerservice.TabletManager.StopReplicationMinimum:output_type -> tabletmanagerdata.StopReplicationMinimumResponse + 77, // 77: tabletmanagerservice.TabletManager.StartReplication:output_type -> tabletmanagerdata.StartReplicationResponse + 78, // 78: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:output_type -> tabletmanagerdata.StartReplicationUntilAfterResponse + 79, // 79: tabletmanagerservice.TabletManager.GetReplicas:output_type -> tabletmanagerdata.GetReplicasResponse + 80, // 80: tabletmanagerservice.TabletManager.CreateVReplicationWorkflow:output_type -> tabletmanagerdata.CreateVReplicationWorkflowResponse + 81, // 81: tabletmanagerservice.TabletManager.DeleteVReplicationWorkflow:output_type -> tabletmanagerdata.DeleteVReplicationWorkflowResponse + 82, // 82: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:output_type -> tabletmanagerdata.ReadVReplicationWorkflowResponse + 83, // 83: tabletmanagerservice.TabletManager.VReplicationExec:output_type -> tabletmanagerdata.VReplicationExecResponse + 84, // 84: tabletmanagerservice.TabletManager.VReplicationWaitForPos:output_type -> tabletmanagerdata.VReplicationWaitForPosResponse + 85, // 85: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:output_type -> tabletmanagerdata.UpdateVReplicationWorkflowResponse + 86, // 86: tabletmanagerservice.TabletManager.VDiff:output_type -> tabletmanagerdata.VDiffResponse + 87, // 87: tabletmanagerservice.TabletManager.ResetReplication:output_type -> tabletmanagerdata.ResetReplicationResponse + 88, // 88: tabletmanagerservice.TabletManager.InitPrimary:output_type -> tabletmanagerdata.InitPrimaryResponse + 89, // 89: tabletmanagerservice.TabletManager.PopulateReparentJournal:output_type -> tabletmanagerdata.PopulateReparentJournalResponse + 90, // 90: tabletmanagerservice.TabletManager.InitReplica:output_type -> tabletmanagerdata.InitReplicaResponse + 91, // 91: tabletmanagerservice.TabletManager.DemotePrimary:output_type -> tabletmanagerdata.DemotePrimaryResponse + 92, // 92: tabletmanagerservice.TabletManager.UndoDemotePrimary:output_type -> tabletmanagerdata.UndoDemotePrimaryResponse + 93, // 93: tabletmanagerservice.TabletManager.ReplicaWasPromoted:output_type -> tabletmanagerdata.ReplicaWasPromotedResponse + 94, // 94: tabletmanagerservice.TabletManager.ResetReplicationParameters:output_type -> tabletmanagerdata.ResetReplicationParametersResponse + 95, // 95: tabletmanagerservice.TabletManager.FullStatus:output_type -> tabletmanagerdata.FullStatusResponse + 96, // 96: tabletmanagerservice.TabletManager.SetReplicationSource:output_type -> tabletmanagerdata.SetReplicationSourceResponse + 97, // 97: tabletmanagerservice.TabletManager.ReplicaWasRestarted:output_type -> tabletmanagerdata.ReplicaWasRestartedResponse + 98, // 98: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:output_type -> tabletmanagerdata.StopReplicationAndGetStatusResponse + 99, // 99: tabletmanagerservice.TabletManager.PromoteReplica:output_type -> tabletmanagerdata.PromoteReplicaResponse + 100, // 100: tabletmanagerservice.TabletManager.Backup:output_type -> tabletmanagerdata.BackupResponse + 101, // 101: tabletmanagerservice.TabletManager.RestoreFromBackup:output_type -> tabletmanagerdata.RestoreFromBackupResponse + 51, // [51:102] is the sub-list for method output_type + 0, // [0:51] 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 } func init() { file_tabletmanagerservice_proto_init() } diff --git a/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go b/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go index bc2e912c606..fbbd0deb2b2 100644 --- a/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go +++ b/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go @@ -70,9 +70,12 @@ type TabletManagerClient interface { // GetReplicas asks for the list of mysql replicas GetReplicas(ctx context.Context, in *tabletmanagerdata.GetReplicasRequest, opts ...grpc.CallOption) (*tabletmanagerdata.GetReplicasResponse, error) // VReplication API + CreateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.CreateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) + DeleteVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.DeleteVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) + ReadVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) VReplicationExec(ctx context.Context, in *tabletmanagerdata.VReplicationExecRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationExecResponse, error) VReplicationWaitForPos(ctx context.Context, in *tabletmanagerdata.VReplicationWaitForPosRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationWaitForPosResponse, error) - UpdateVRWorkflow(ctx context.Context, in *tabletmanagerdata.UpdateVRWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVRWorkflowResponse, error) + UpdateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.UpdateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) // VDiff API VDiff(ctx context.Context, in *tabletmanagerdata.VDiffRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VDiffResponse, error) // ResetReplication makes the target not replicating @@ -377,6 +380,33 @@ func (c *tabletManagerClient) GetReplicas(ctx context.Context, in *tabletmanager return out, nil } +func (c *tabletManagerClient) CreateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.CreateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) { + out := new(tabletmanagerdata.CreateVReplicationWorkflowResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/CreateVReplicationWorkflow", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tabletManagerClient) DeleteVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.DeleteVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) { + out := new(tabletmanagerdata.DeleteVReplicationWorkflowResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/DeleteVReplicationWorkflow", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tabletManagerClient) ReadVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) { + out := new(tabletmanagerdata.ReadVReplicationWorkflowResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/ReadVReplicationWorkflow", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tabletManagerClient) VReplicationExec(ctx context.Context, in *tabletmanagerdata.VReplicationExecRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationExecResponse, error) { out := new(tabletmanagerdata.VReplicationExecResponse) err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/VReplicationExec", in, out, opts...) @@ -395,9 +425,9 @@ func (c *tabletManagerClient) VReplicationWaitForPos(ctx context.Context, in *ta return out, nil } -func (c *tabletManagerClient) UpdateVRWorkflow(ctx context.Context, in *tabletmanagerdata.UpdateVRWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVRWorkflowResponse, error) { - out := new(tabletmanagerdata.UpdateVRWorkflowResponse) - err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/UpdateVRWorkflow", in, out, opts...) +func (c *tabletManagerClient) UpdateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.UpdateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) { + out := new(tabletmanagerdata.UpdateVReplicationWorkflowResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/UpdateVReplicationWorkflow", in, out, opts...) if err != nil { return nil, err } @@ -645,9 +675,12 @@ type TabletManagerServer interface { // GetReplicas asks for the list of mysql replicas GetReplicas(context.Context, *tabletmanagerdata.GetReplicasRequest) (*tabletmanagerdata.GetReplicasResponse, error) // VReplication API + CreateVReplicationWorkflow(context.Context, *tabletmanagerdata.CreateVReplicationWorkflowRequest) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) + DeleteVReplicationWorkflow(context.Context, *tabletmanagerdata.DeleteVReplicationWorkflowRequest) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) + ReadVReplicationWorkflow(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowRequest) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) VReplicationExec(context.Context, *tabletmanagerdata.VReplicationExecRequest) (*tabletmanagerdata.VReplicationExecResponse, error) VReplicationWaitForPos(context.Context, *tabletmanagerdata.VReplicationWaitForPosRequest) (*tabletmanagerdata.VReplicationWaitForPosResponse, error) - UpdateVRWorkflow(context.Context, *tabletmanagerdata.UpdateVRWorkflowRequest) (*tabletmanagerdata.UpdateVRWorkflowResponse, error) + UpdateVReplicationWorkflow(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) // VDiff API VDiff(context.Context, *tabletmanagerdata.VDiffRequest) (*tabletmanagerdata.VDiffResponse, error) // ResetReplication makes the target not replicating @@ -775,14 +808,23 @@ func (UnimplementedTabletManagerServer) StartReplicationUntilAfter(context.Conte func (UnimplementedTabletManagerServer) GetReplicas(context.Context, *tabletmanagerdata.GetReplicasRequest) (*tabletmanagerdata.GetReplicasResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetReplicas not implemented") } +func (UnimplementedTabletManagerServer) CreateVReplicationWorkflow(context.Context, *tabletmanagerdata.CreateVReplicationWorkflowRequest) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateVReplicationWorkflow not implemented") +} +func (UnimplementedTabletManagerServer) DeleteVReplicationWorkflow(context.Context, *tabletmanagerdata.DeleteVReplicationWorkflowRequest) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteVReplicationWorkflow not implemented") +} +func (UnimplementedTabletManagerServer) ReadVReplicationWorkflow(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowRequest) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReadVReplicationWorkflow not implemented") +} func (UnimplementedTabletManagerServer) VReplicationExec(context.Context, *tabletmanagerdata.VReplicationExecRequest) (*tabletmanagerdata.VReplicationExecResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method VReplicationExec not implemented") } func (UnimplementedTabletManagerServer) VReplicationWaitForPos(context.Context, *tabletmanagerdata.VReplicationWaitForPosRequest) (*tabletmanagerdata.VReplicationWaitForPosResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method VReplicationWaitForPos not implemented") } -func (UnimplementedTabletManagerServer) UpdateVRWorkflow(context.Context, *tabletmanagerdata.UpdateVRWorkflowRequest) (*tabletmanagerdata.UpdateVRWorkflowResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method UpdateVRWorkflow not implemented") +func (UnimplementedTabletManagerServer) UpdateVReplicationWorkflow(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateVReplicationWorkflow not implemented") } func (UnimplementedTabletManagerServer) VDiff(context.Context, *tabletmanagerdata.VDiffRequest) (*tabletmanagerdata.VDiffResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method VDiff not implemented") @@ -1367,6 +1409,60 @@ func _TabletManager_GetReplicas_Handler(srv interface{}, ctx context.Context, de return interceptor(ctx, in, info, handler) } +func _TabletManager_CreateVReplicationWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.CreateVReplicationWorkflowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).CreateVReplicationWorkflow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/CreateVReplicationWorkflow", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).CreateVReplicationWorkflow(ctx, req.(*tabletmanagerdata.CreateVReplicationWorkflowRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TabletManager_DeleteVReplicationWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.DeleteVReplicationWorkflowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).DeleteVReplicationWorkflow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/DeleteVReplicationWorkflow", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).DeleteVReplicationWorkflow(ctx, req.(*tabletmanagerdata.DeleteVReplicationWorkflowRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TabletManager_ReadVReplicationWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.ReadVReplicationWorkflowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).ReadVReplicationWorkflow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/ReadVReplicationWorkflow", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).ReadVReplicationWorkflow(ctx, req.(*tabletmanagerdata.ReadVReplicationWorkflowRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _TabletManager_VReplicationExec_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(tabletmanagerdata.VReplicationExecRequest) if err := dec(in); err != nil { @@ -1403,20 +1499,20 @@ func _TabletManager_VReplicationWaitForPos_Handler(srv interface{}, ctx context. return interceptor(ctx, in, info, handler) } -func _TabletManager_UpdateVRWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(tabletmanagerdata.UpdateVRWorkflowRequest) +func _TabletManager_UpdateVReplicationWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.UpdateVReplicationWorkflowRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(TabletManagerServer).UpdateVRWorkflow(ctx, in) + return srv.(TabletManagerServer).UpdateVReplicationWorkflow(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/tabletmanagerservice.TabletManager/UpdateVRWorkflow", + FullMethod: "/tabletmanagerservice.TabletManager/UpdateVReplicationWorkflow", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(TabletManagerServer).UpdateVRWorkflow(ctx, req.(*tabletmanagerdata.UpdateVRWorkflowRequest)) + return srv.(TabletManagerServer).UpdateVReplicationWorkflow(ctx, req.(*tabletmanagerdata.UpdateVReplicationWorkflowRequest)) } return interceptor(ctx, in, info, handler) } @@ -1838,6 +1934,18 @@ var TabletManager_ServiceDesc = grpc.ServiceDesc{ MethodName: "GetReplicas", Handler: _TabletManager_GetReplicas_Handler, }, + { + MethodName: "CreateVReplicationWorkflow", + Handler: _TabletManager_CreateVReplicationWorkflow_Handler, + }, + { + MethodName: "DeleteVReplicationWorkflow", + Handler: _TabletManager_DeleteVReplicationWorkflow_Handler, + }, + { + MethodName: "ReadVReplicationWorkflow", + Handler: _TabletManager_ReadVReplicationWorkflow_Handler, + }, { MethodName: "VReplicationExec", Handler: _TabletManager_VReplicationExec_Handler, @@ -1847,8 +1955,8 @@ var TabletManager_ServiceDesc = grpc.ServiceDesc{ Handler: _TabletManager_VReplicationWaitForPos_Handler, }, { - MethodName: "UpdateVRWorkflow", - Handler: _TabletManager_UpdateVRWorkflow_Handler, + MethodName: "UpdateVReplicationWorkflow", + Handler: _TabletManager_UpdateVReplicationWorkflow_Handler, }, { MethodName: "VDiff", diff --git a/go/vt/proto/vtctldata/vtctldata.pb.go b/go/vt/proto/vtctldata/vtctldata.pb.go index 885cb34c207..214778c47d9 100644 --- a/go/vt/proto/vtctldata/vtctldata.pb.go +++ b/go/vt/proto/vtctldata/vtctldata.pb.go @@ -304,7 +304,8 @@ type MaterializeSettings struct { // OnDdl specifies the action to be taken when a DDL is encountered. OnDdl string `protobuf:"bytes,13,opt,name=on_ddl,json=onDdl,proto3" json:"on_ddl,omitempty"` // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. - DeferSecondaryKeys bool `protobuf:"varint,14,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` + 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"` } func (x *MaterializeSettings) Reset() { @@ -437,6 +438,13 @@ func (x *MaterializeSettings) GetDeferSecondaryKeys() bool { return false } +func (x *MaterializeSettings) GetTabletSelectionPreference() tabletmanagerdata.TabletSelectionPreference { + if x != nil { + return x.TabletSelectionPreference + } + return tabletmanagerdata.TabletSelectionPreference(0) +} + type Keyspace struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5600,6 +5608,8 @@ type GetWorkflowsRequest struct { Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` ActiveOnly bool `protobuf:"varint,2,opt,name=active_only,json=activeOnly,proto3" json:"active_only,omitempty"` NameOnly bool `protobuf:"varint,3,opt,name=name_only,json=nameOnly,proto3" json:"name_only,omitempty"` + // If you only want a specific workflow then set this field. + Workflow string `protobuf:"bytes,4,opt,name=workflow,proto3" json:"workflow,omitempty"` } func (x *GetWorkflowsRequest) Reset() { @@ -5655,6 +5665,13 @@ func (x *GetWorkflowsRequest) GetNameOnly() bool { return false } +func (x *GetWorkflowsRequest) GetWorkflow() string { + if x != nil { + return x.Workflow + } + return "" +} + type GetWorkflowsResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5828,16 +5845,41 @@ func (x *InitShardPrimaryResponse) GetEvents() []*logutil.Event { return nil } -type PingTabletRequest struct { +type MoveTablesCreateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - TabletAlias *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet_alias,json=tabletAlias,proto3" json:"tablet_alias,omitempty"` + // The necessary info gets passed on to each primary tablet involved + // in the workflow via the CreateVReplicationWorkflow tabletmanager RPC. + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` + SourceKeyspace string `protobuf:"bytes,2,opt,name=source_keyspace,json=sourceKeyspace,proto3" json:"source_keyspace,omitempty"` + TargetKeyspace string `protobuf:"bytes,3,opt,name=target_keyspace,json=targetKeyspace,proto3" json:"target_keyspace,omitempty"` + Cells []string `protobuf:"bytes,4,rep,name=cells,proto3" json:"cells,omitempty"` + TabletTypes []topodata.TabletType `protobuf:"varint,5,rep,packed,name=tablet_types,json=tabletTypes,proto3,enum=topodata.TabletType" json:"tablet_types,omitempty"` + TabletSelectionPreference tabletmanagerdata.TabletSelectionPreference `protobuf:"varint,6,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` + SourceShards []string `protobuf:"bytes,7,rep,name=source_shards,json=sourceShards,proto3" json:"source_shards,omitempty"` + AllTables bool `protobuf:"varint,8,opt,name=all_tables,json=allTables,proto3" json:"all_tables,omitempty"` + IncludeTables []string `protobuf:"bytes,9,rep,name=include_tables,json=includeTables,proto3" json:"include_tables,omitempty"` + ExcludeTables []string `protobuf:"bytes,10,rep,name=exclude_tables,json=excludeTables,proto3" json:"exclude_tables,omitempty"` + // The name of the external cluster mounted in topo server. + ExternalClusterName string `protobuf:"bytes,11,opt,name=external_cluster_name,json=externalClusterName,proto3" json:"external_cluster_name,omitempty"` + // SourceTimeZone is the time zone in which datetimes on the source were stored, provided as an option in MoveTables + SourceTimeZone string `protobuf:"bytes,12,opt,name=source_time_zone,json=sourceTimeZone,proto3" json:"source_time_zone,omitempty"` + // OnDdl specifies the action to be taken when a DDL is encountered. + OnDdl string `protobuf:"bytes,13,opt,name=on_ddl,json=onDdl,proto3" json:"on_ddl,omitempty"` + // StopAfterCopy specifies if vreplication should be stopped after copying. + StopAfterCopy bool `protobuf:"varint,14,opt,name=stop_after_copy,json=stopAfterCopy,proto3" json:"stop_after_copy,omitempty"` + // DropForeignKeys specifies if foreign key constraints should be elided on the target. + DropForeignKeys bool `protobuf:"varint,15,opt,name=drop_foreign_keys,json=dropForeignKeys,proto3" json:"drop_foreign_keys,omitempty"` + // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. + DeferSecondaryKeys bool `protobuf:"varint,16,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` + // Start the workflow after creating it. + AutoStart bool `protobuf:"varint,17,opt,name=auto_start,json=autoStart,proto3" json:"auto_start,omitempty"` } -func (x *PingTabletRequest) Reset() { - *x = PingTabletRequest{} +func (x *MoveTablesCreateRequest) Reset() { + *x = MoveTablesCreateRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5845,13 +5887,13 @@ func (x *PingTabletRequest) Reset() { } } -func (x *PingTabletRequest) String() string { +func (x *MoveTablesCreateRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PingTabletRequest) ProtoMessage() {} +func (*MoveTablesCreateRequest) ProtoMessage() {} -func (x *PingTabletRequest) ProtoReflect() protoreflect.Message { +func (x *MoveTablesCreateRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[99] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5863,26 +5905,141 @@ func (x *PingTabletRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PingTabletRequest.ProtoReflect.Descriptor instead. -func (*PingTabletRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use MoveTablesCreateRequest.ProtoReflect.Descriptor instead. +func (*MoveTablesCreateRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{99} } -func (x *PingTabletRequest) GetTabletAlias() *topodata.TabletAlias { +func (x *MoveTablesCreateRequest) GetWorkflow() string { if x != nil { - return x.TabletAlias + return x.Workflow + } + return "" +} + +func (x *MoveTablesCreateRequest) GetSourceKeyspace() string { + if x != nil { + return x.SourceKeyspace + } + return "" +} + +func (x *MoveTablesCreateRequest) GetTargetKeyspace() string { + if x != nil { + return x.TargetKeyspace + } + return "" +} + +func (x *MoveTablesCreateRequest) GetCells() []string { + if x != nil { + return x.Cells } return nil } -type PingTabletResponse struct { +func (x *MoveTablesCreateRequest) GetTabletTypes() []topodata.TabletType { + if x != nil { + return x.TabletTypes + } + return nil +} + +func (x *MoveTablesCreateRequest) GetTabletSelectionPreference() tabletmanagerdata.TabletSelectionPreference { + if x != nil { + return x.TabletSelectionPreference + } + return tabletmanagerdata.TabletSelectionPreference(0) +} + +func (x *MoveTablesCreateRequest) GetSourceShards() []string { + if x != nil { + return x.SourceShards + } + return nil +} + +func (x *MoveTablesCreateRequest) GetAllTables() bool { + if x != nil { + return x.AllTables + } + return false +} + +func (x *MoveTablesCreateRequest) GetIncludeTables() []string { + if x != nil { + return x.IncludeTables + } + return nil +} + +func (x *MoveTablesCreateRequest) GetExcludeTables() []string { + if x != nil { + return x.ExcludeTables + } + return nil +} + +func (x *MoveTablesCreateRequest) GetExternalClusterName() string { + if x != nil { + return x.ExternalClusterName + } + return "" +} + +func (x *MoveTablesCreateRequest) GetSourceTimeZone() string { + if x != nil { + return x.SourceTimeZone + } + return "" +} + +func (x *MoveTablesCreateRequest) GetOnDdl() string { + if x != nil { + return x.OnDdl + } + return "" +} + +func (x *MoveTablesCreateRequest) GetStopAfterCopy() bool { + if x != nil { + return x.StopAfterCopy + } + return false +} + +func (x *MoveTablesCreateRequest) GetDropForeignKeys() bool { + if x != nil { + return x.DropForeignKeys + } + return false +} + +func (x *MoveTablesCreateRequest) GetDeferSecondaryKeys() bool { + if x != nil { + return x.DeferSecondaryKeys + } + return false +} + +func (x *MoveTablesCreateRequest) GetAutoStart() bool { + if x != nil { + return x.AutoStart + } + return false +} + +type MoveTablesCreateResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + + Summary string `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` + Details []*MoveTablesCreateResponse_TabletInfo `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty"` } -func (x *PingTabletResponse) Reset() { - *x = PingTabletResponse{} +func (x *MoveTablesCreateResponse) Reset() { + *x = MoveTablesCreateResponse{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[100] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5890,13 +6047,13 @@ func (x *PingTabletResponse) Reset() { } } -func (x *PingTabletResponse) String() string { +func (x *MoveTablesCreateResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PingTabletResponse) ProtoMessage() {} +func (*MoveTablesCreateResponse) ProtoMessage() {} -func (x *PingTabletResponse) ProtoReflect() protoreflect.Message { +func (x *MoveTablesCreateResponse) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[100] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5908,42 +6065,40 @@ func (x *PingTabletResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PingTabletResponse.ProtoReflect.Descriptor instead. -func (*PingTabletResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use MoveTablesCreateResponse.ProtoReflect.Descriptor instead. +func (*MoveTablesCreateResponse) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{100} } -type PlannedReparentShardRequest struct { +func (x *MoveTablesCreateResponse) GetSummary() string { + if x != nil { + return x.Summary + } + return "" +} + +func (x *MoveTablesCreateResponse) GetDetails() []*MoveTablesCreateResponse_TabletInfo { + if x != nil { + return x.Details + } + return nil +} + +type MoveTablesCompleteRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // Keyspace is the name of the keyspace to perform the Planned Reparent in. - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - // Shard is the name of the shard to perform teh Planned Reparent in. - Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` - // NewPrimary is the alias of the tablet to promote to shard primary. If not - // specified, the vtctld will select the most up-to-date candidate to promote. - // - // It is an error to set NewPrimary and AvoidPrimary to the same alias. - NewPrimary *topodata.TabletAlias `protobuf:"bytes,3,opt,name=new_primary,json=newPrimary,proto3" json:"new_primary,omitempty"` - // AvoidPrimary is the alias of the tablet to demote. In other words, - // specifying an AvoidPrimary alias tells the vtctld to promote any replica - // other than this one. A shard whose current primary is not this one is then - // a no-op. - // - // It is an error to set NewPrimary and AvoidPrimary to the same alias. - AvoidPrimary *topodata.TabletAlias `protobuf:"bytes,4,opt,name=avoid_primary,json=avoidPrimary,proto3" json:"avoid_primary,omitempty"` - // WaitReplicasTimeout is the duration of time to wait for replicas to catch - // up in replication both before and after the reparent. The timeout is not - // cumulative across both wait periods, meaning that the replicas have - // WaitReplicasTimeout time to catch up before the reparent, and an additional - // WaitReplicasTimeout time to catch up after the reparent. - WaitReplicasTimeout *vttime.Duration `protobuf:"bytes,5,opt,name=wait_replicas_timeout,json=waitReplicasTimeout,proto3" json:"wait_replicas_timeout,omitempty"` + Workflow string `protobuf:"bytes,1,opt,name=workflow,proto3" json:"workflow,omitempty"` + TargetKeyspace string `protobuf:"bytes,3,opt,name=target_keyspace,json=targetKeyspace,proto3" json:"target_keyspace,omitempty"` + KeepData bool `protobuf:"varint,4,opt,name=keep_data,json=keepData,proto3" json:"keep_data,omitempty"` + KeepRoutingRules bool `protobuf:"varint,5,opt,name=keep_routing_rules,json=keepRoutingRules,proto3" json:"keep_routing_rules,omitempty"` + RenameTables bool `protobuf:"varint,6,opt,name=rename_tables,json=renameTables,proto3" json:"rename_tables,omitempty"` + DryRun bool `protobuf:"varint,7,opt,name=dry_run,json=dryRun,proto3" json:"dry_run,omitempty"` } -func (x *PlannedReparentShardRequest) Reset() { - *x = PlannedReparentShardRequest{} +func (x *MoveTablesCompleteRequest) Reset() { + *x = MoveTablesCompleteRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5951,13 +6106,13 @@ func (x *PlannedReparentShardRequest) Reset() { } } -func (x *PlannedReparentShardRequest) String() string { +func (x *MoveTablesCompleteRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PlannedReparentShardRequest) ProtoMessage() {} +func (*MoveTablesCompleteRequest) ProtoMessage() {} -func (x *PlannedReparentShardRequest) ProtoReflect() protoreflect.Message { +func (x *MoveTablesCompleteRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[101] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5969,65 +6124,64 @@ func (x *PlannedReparentShardRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PlannedReparentShardRequest.ProtoReflect.Descriptor instead. -func (*PlannedReparentShardRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use MoveTablesCompleteRequest.ProtoReflect.Descriptor instead. +func (*MoveTablesCompleteRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{101} } -func (x *PlannedReparentShardRequest) GetKeyspace() string { +func (x *MoveTablesCompleteRequest) GetWorkflow() string { if x != nil { - return x.Keyspace + return x.Workflow } return "" } -func (x *PlannedReparentShardRequest) GetShard() string { +func (x *MoveTablesCompleteRequest) GetTargetKeyspace() string { if x != nil { - return x.Shard + return x.TargetKeyspace } return "" } -func (x *PlannedReparentShardRequest) GetNewPrimary() *topodata.TabletAlias { +func (x *MoveTablesCompleteRequest) GetKeepData() bool { if x != nil { - return x.NewPrimary + return x.KeepData } - return nil + return false } -func (x *PlannedReparentShardRequest) GetAvoidPrimary() *topodata.TabletAlias { +func (x *MoveTablesCompleteRequest) GetKeepRoutingRules() bool { if x != nil { - return x.AvoidPrimary + return x.KeepRoutingRules } - return nil + return false } -func (x *PlannedReparentShardRequest) GetWaitReplicasTimeout() *vttime.Duration { +func (x *MoveTablesCompleteRequest) GetRenameTables() bool { if x != nil { - return x.WaitReplicasTimeout + return x.RenameTables } - return nil + return false } -type PlannedReparentShardResponse struct { +func (x *MoveTablesCompleteRequest) GetDryRun() bool { + if x != nil { + return x.DryRun + } + return false +} + +type MoveTablesCompleteResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // Keyspace is the name of the keyspace the Planned Reparent took place in. - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - // Shard is the name of the shard the Planned Reparent took place in. - Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` - // PromotedPrimary is the alias of the tablet that was promoted to shard - // primary. If NewPrimary was set in the request, then this will be the same - // alias. Otherwise, it will be the alias of the tablet found to be most - // up-to-date. - PromotedPrimary *topodata.TabletAlias `protobuf:"bytes,3,opt,name=promoted_primary,json=promotedPrimary,proto3" json:"promoted_primary,omitempty"` - Events []*logutil.Event `protobuf:"bytes,4,rep,name=events,proto3" json:"events,omitempty"` + Summary string `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` + DryRunResults []string `protobuf:"bytes,2,rep,name=dry_run_results,json=dryRunResults,proto3" json:"dry_run_results,omitempty"` } -func (x *PlannedReparentShardResponse) Reset() { - *x = PlannedReparentShardResponse{} +func (x *MoveTablesCompleteResponse) Reset() { + *x = MoveTablesCompleteResponse{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[102] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6035,13 +6189,13 @@ func (x *PlannedReparentShardResponse) Reset() { } } -func (x *PlannedReparentShardResponse) String() string { +func (x *MoveTablesCompleteResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PlannedReparentShardResponse) ProtoMessage() {} +func (*MoveTablesCompleteResponse) ProtoMessage() {} -func (x *PlannedReparentShardResponse) ProtoReflect() protoreflect.Message { +func (x *MoveTablesCompleteResponse) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[102] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6053,53 +6207,35 @@ func (x *PlannedReparentShardResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PlannedReparentShardResponse.ProtoReflect.Descriptor instead. -func (*PlannedReparentShardResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use MoveTablesCompleteResponse.ProtoReflect.Descriptor instead. +func (*MoveTablesCompleteResponse) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{102} } -func (x *PlannedReparentShardResponse) GetKeyspace() string { - if x != nil { - return x.Keyspace - } - return "" -} - -func (x *PlannedReparentShardResponse) GetShard() string { +func (x *MoveTablesCompleteResponse) GetSummary() string { if x != nil { - return x.Shard + return x.Summary } return "" } -func (x *PlannedReparentShardResponse) GetPromotedPrimary() *topodata.TabletAlias { - if x != nil { - return x.PromotedPrimary - } - return nil -} - -func (x *PlannedReparentShardResponse) GetEvents() []*logutil.Event { +func (x *MoveTablesCompleteResponse) GetDryRunResults() []string { if x != nil { - return x.Events + return x.DryRunResults } return nil } -type RebuildKeyspaceGraphRequest struct { +type PingTabletRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - Cells []string `protobuf:"bytes,2,rep,name=cells,proto3" json:"cells,omitempty"` - // AllowPartial, when set, allows a SNAPSHOT keyspace to serve with an - // incomplete set of shards. It is ignored for all other keyspace types. - AllowPartial bool `protobuf:"varint,3,opt,name=allow_partial,json=allowPartial,proto3" json:"allow_partial,omitempty"` + TabletAlias *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet_alias,json=tabletAlias,proto3" json:"tablet_alias,omitempty"` } -func (x *RebuildKeyspaceGraphRequest) Reset() { - *x = RebuildKeyspaceGraphRequest{} +func (x *PingTabletRequest) Reset() { + *x = PingTabletRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[103] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6107,13 +6243,13 @@ func (x *RebuildKeyspaceGraphRequest) Reset() { } } -func (x *RebuildKeyspaceGraphRequest) String() string { +func (x *PingTabletRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RebuildKeyspaceGraphRequest) ProtoMessage() {} +func (*PingTabletRequest) ProtoMessage() {} -func (x *RebuildKeyspaceGraphRequest) ProtoReflect() protoreflect.Message { +func (x *PingTabletRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[103] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6125,40 +6261,26 @@ func (x *RebuildKeyspaceGraphRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RebuildKeyspaceGraphRequest.ProtoReflect.Descriptor instead. -func (*RebuildKeyspaceGraphRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use PingTabletRequest.ProtoReflect.Descriptor instead. +func (*PingTabletRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{103} } -func (x *RebuildKeyspaceGraphRequest) GetKeyspace() string { +func (x *PingTabletRequest) GetTabletAlias() *topodata.TabletAlias { if x != nil { - return x.Keyspace + return x.TabletAlias } - return "" + return nil } -func (x *RebuildKeyspaceGraphRequest) GetCells() []string { - if x != nil { - return x.Cells - } - return nil -} - -func (x *RebuildKeyspaceGraphRequest) GetAllowPartial() bool { - if x != nil { - return x.AllowPartial - } - return false -} - -type RebuildKeyspaceGraphResponse struct { +type PingTabletResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields } -func (x *RebuildKeyspaceGraphResponse) Reset() { - *x = RebuildKeyspaceGraphResponse{} +func (x *PingTabletResponse) Reset() { + *x = PingTabletResponse{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6166,13 +6288,13 @@ func (x *RebuildKeyspaceGraphResponse) Reset() { } } -func (x *RebuildKeyspaceGraphResponse) String() string { +func (x *PingTabletResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RebuildKeyspaceGraphResponse) ProtoMessage() {} +func (*PingTabletResponse) ProtoMessage() {} -func (x *RebuildKeyspaceGraphResponse) ProtoReflect() protoreflect.Message { +func (x *PingTabletResponse) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[104] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6184,23 +6306,42 @@ func (x *RebuildKeyspaceGraphResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RebuildKeyspaceGraphResponse.ProtoReflect.Descriptor instead. -func (*RebuildKeyspaceGraphResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use PingTabletResponse.ProtoReflect.Descriptor instead. +func (*PingTabletResponse) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{104} } -type RebuildVSchemaGraphRequest struct { +type PlannedReparentShardRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // Cells specifies the cells to rebuild the SrvVSchema objects for. If empty, - // RebuildVSchemaGraph rebuilds the SrvVSchema for every cell in the topo. - Cells []string `protobuf:"bytes,1,rep,name=cells,proto3" json:"cells,omitempty"` + // Keyspace is the name of the keyspace to perform the Planned Reparent in. + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + // Shard is the name of the shard to perform teh Planned Reparent in. + Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` + // NewPrimary is the alias of the tablet to promote to shard primary. If not + // specified, the vtctld will select the most up-to-date candidate to promote. + // + // It is an error to set NewPrimary and AvoidPrimary to the same alias. + NewPrimary *topodata.TabletAlias `protobuf:"bytes,3,opt,name=new_primary,json=newPrimary,proto3" json:"new_primary,omitempty"` + // AvoidPrimary is the alias of the tablet to demote. In other words, + // specifying an AvoidPrimary alias tells the vtctld to promote any replica + // other than this one. A shard whose current primary is not this one is then + // a no-op. + // + // It is an error to set NewPrimary and AvoidPrimary to the same alias. + AvoidPrimary *topodata.TabletAlias `protobuf:"bytes,4,opt,name=avoid_primary,json=avoidPrimary,proto3" json:"avoid_primary,omitempty"` + // WaitReplicasTimeout is the duration of time to wait for replicas to catch + // up in replication both before and after the reparent. The timeout is not + // cumulative across both wait periods, meaning that the replicas have + // WaitReplicasTimeout time to catch up before the reparent, and an additional + // WaitReplicasTimeout time to catch up after the reparent. + WaitReplicasTimeout *vttime.Duration `protobuf:"bytes,5,opt,name=wait_replicas_timeout,json=waitReplicasTimeout,proto3" json:"wait_replicas_timeout,omitempty"` } -func (x *RebuildVSchemaGraphRequest) Reset() { - *x = RebuildVSchemaGraphRequest{} +func (x *PlannedReparentShardRequest) Reset() { + *x = PlannedReparentShardRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6208,13 +6349,13 @@ func (x *RebuildVSchemaGraphRequest) Reset() { } } -func (x *RebuildVSchemaGraphRequest) String() string { +func (x *PlannedReparentShardRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RebuildVSchemaGraphRequest) ProtoMessage() {} +func (*PlannedReparentShardRequest) ProtoMessage() {} -func (x *RebuildVSchemaGraphRequest) ProtoReflect() protoreflect.Message { +func (x *PlannedReparentShardRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[105] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6226,26 +6367,65 @@ func (x *RebuildVSchemaGraphRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RebuildVSchemaGraphRequest.ProtoReflect.Descriptor instead. -func (*RebuildVSchemaGraphRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use PlannedReparentShardRequest.ProtoReflect.Descriptor instead. +func (*PlannedReparentShardRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{105} } -func (x *RebuildVSchemaGraphRequest) GetCells() []string { +func (x *PlannedReparentShardRequest) GetKeyspace() string { if x != nil { - return x.Cells + return x.Keyspace + } + return "" +} + +func (x *PlannedReparentShardRequest) GetShard() string { + if x != nil { + return x.Shard + } + return "" +} + +func (x *PlannedReparentShardRequest) GetNewPrimary() *topodata.TabletAlias { + if x != nil { + return x.NewPrimary } return nil } -type RebuildVSchemaGraphResponse struct { +func (x *PlannedReparentShardRequest) GetAvoidPrimary() *topodata.TabletAlias { + if x != nil { + return x.AvoidPrimary + } + return nil +} + +func (x *PlannedReparentShardRequest) GetWaitReplicasTimeout() *vttime.Duration { + if x != nil { + return x.WaitReplicasTimeout + } + return nil +} + +type PlannedReparentShardResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + + // Keyspace is the name of the keyspace the Planned Reparent took place in. + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + // Shard is the name of the shard the Planned Reparent took place in. + Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` + // PromotedPrimary is the alias of the tablet that was promoted to shard + // primary. If NewPrimary was set in the request, then this will be the same + // alias. Otherwise, it will be the alias of the tablet found to be most + // up-to-date. + PromotedPrimary *topodata.TabletAlias `protobuf:"bytes,3,opt,name=promoted_primary,json=promotedPrimary,proto3" json:"promoted_primary,omitempty"` + Events []*logutil.Event `protobuf:"bytes,4,rep,name=events,proto3" json:"events,omitempty"` } -func (x *RebuildVSchemaGraphResponse) Reset() { - *x = RebuildVSchemaGraphResponse{} +func (x *PlannedReparentShardResponse) Reset() { + *x = PlannedReparentShardResponse{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6253,13 +6433,13 @@ func (x *RebuildVSchemaGraphResponse) Reset() { } } -func (x *RebuildVSchemaGraphResponse) String() string { +func (x *PlannedReparentShardResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RebuildVSchemaGraphResponse) ProtoMessage() {} +func (*PlannedReparentShardResponse) ProtoMessage() {} -func (x *RebuildVSchemaGraphResponse) ProtoReflect() protoreflect.Message { +func (x *PlannedReparentShardResponse) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[106] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6271,21 +6451,53 @@ func (x *RebuildVSchemaGraphResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RebuildVSchemaGraphResponse.ProtoReflect.Descriptor instead. -func (*RebuildVSchemaGraphResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use PlannedReparentShardResponse.ProtoReflect.Descriptor instead. +func (*PlannedReparentShardResponse) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{106} } -type RefreshStateRequest struct { +func (x *PlannedReparentShardResponse) GetKeyspace() string { + if x != nil { + return x.Keyspace + } + return "" +} + +func (x *PlannedReparentShardResponse) GetShard() string { + if x != nil { + return x.Shard + } + return "" +} + +func (x *PlannedReparentShardResponse) GetPromotedPrimary() *topodata.TabletAlias { + if x != nil { + return x.PromotedPrimary + } + return nil +} + +func (x *PlannedReparentShardResponse) GetEvents() []*logutil.Event { + if x != nil { + return x.Events + } + return nil +} + +type RebuildKeyspaceGraphRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - TabletAlias *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet_alias,json=tabletAlias,proto3" json:"tablet_alias,omitempty"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Cells []string `protobuf:"bytes,2,rep,name=cells,proto3" json:"cells,omitempty"` + // AllowPartial, when set, allows a SNAPSHOT keyspace to serve with an + // incomplete set of shards. It is ignored for all other keyspace types. + AllowPartial bool `protobuf:"varint,3,opt,name=allow_partial,json=allowPartial,proto3" json:"allow_partial,omitempty"` } -func (x *RefreshStateRequest) Reset() { - *x = RefreshStateRequest{} +func (x *RebuildKeyspaceGraphRequest) Reset() { + *x = RebuildKeyspaceGraphRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6293,13 +6505,13 @@ func (x *RefreshStateRequest) Reset() { } } -func (x *RefreshStateRequest) String() string { +func (x *RebuildKeyspaceGraphRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RefreshStateRequest) ProtoMessage() {} +func (*RebuildKeyspaceGraphRequest) ProtoMessage() {} -func (x *RefreshStateRequest) ProtoReflect() protoreflect.Message { +func (x *RebuildKeyspaceGraphRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[107] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6311,26 +6523,40 @@ func (x *RefreshStateRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RefreshStateRequest.ProtoReflect.Descriptor instead. -func (*RefreshStateRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use RebuildKeyspaceGraphRequest.ProtoReflect.Descriptor instead. +func (*RebuildKeyspaceGraphRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{107} } -func (x *RefreshStateRequest) GetTabletAlias() *topodata.TabletAlias { +func (x *RebuildKeyspaceGraphRequest) GetKeyspace() string { if x != nil { - return x.TabletAlias + return x.Keyspace + } + return "" +} + +func (x *RebuildKeyspaceGraphRequest) GetCells() []string { + if x != nil { + return x.Cells } return nil } -type RefreshStateResponse struct { +func (x *RebuildKeyspaceGraphRequest) GetAllowPartial() bool { + if x != nil { + return x.AllowPartial + } + return false +} + +type RebuildKeyspaceGraphResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields } -func (x *RefreshStateResponse) Reset() { - *x = RefreshStateResponse{} +func (x *RebuildKeyspaceGraphResponse) Reset() { + *x = RebuildKeyspaceGraphResponse{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6338,13 +6564,13 @@ func (x *RefreshStateResponse) Reset() { } } -func (x *RefreshStateResponse) String() string { +func (x *RebuildKeyspaceGraphResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RefreshStateResponse) ProtoMessage() {} +func (*RebuildKeyspaceGraphResponse) ProtoMessage() {} -func (x *RefreshStateResponse) ProtoReflect() protoreflect.Message { +func (x *RebuildKeyspaceGraphResponse) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[108] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6356,23 +6582,23 @@ func (x *RefreshStateResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RefreshStateResponse.ProtoReflect.Descriptor instead. -func (*RefreshStateResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use RebuildKeyspaceGraphResponse.ProtoReflect.Descriptor instead. +func (*RebuildKeyspaceGraphResponse) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{108} } -type RefreshStateByShardRequest struct { +type RebuildVSchemaGraphRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` - Cells []string `protobuf:"bytes,3,rep,name=cells,proto3" json:"cells,omitempty"` + // Cells specifies the cells to rebuild the SrvVSchema objects for. If empty, + // RebuildVSchemaGraph rebuilds the SrvVSchema for every cell in the topo. + Cells []string `protobuf:"bytes,1,rep,name=cells,proto3" json:"cells,omitempty"` } -func (x *RefreshStateByShardRequest) Reset() { - *x = RefreshStateByShardRequest{} +func (x *RebuildVSchemaGraphRequest) Reset() { + *x = RebuildVSchemaGraphRequest{} if protoimpl.UnsafeEnabled { mi := &file_vtctldata_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6380,13 +6606,13 @@ func (x *RefreshStateByShardRequest) Reset() { } } -func (x *RefreshStateByShardRequest) String() string { +func (x *RebuildVSchemaGraphRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*RefreshStateByShardRequest) ProtoMessage() {} +func (*RebuildVSchemaGraphRequest) ProtoMessage() {} -func (x *RefreshStateByShardRequest) ProtoReflect() protoreflect.Message { +func (x *RebuildVSchemaGraphRequest) ProtoReflect() protoreflect.Message { mi := &file_vtctldata_proto_msgTypes[109] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -6398,46 +6624,218 @@ func (x *RefreshStateByShardRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RefreshStateByShardRequest.ProtoReflect.Descriptor instead. -func (*RefreshStateByShardRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use RebuildVSchemaGraphRequest.ProtoReflect.Descriptor instead. +func (*RebuildVSchemaGraphRequest) Descriptor() ([]byte, []int) { return file_vtctldata_proto_rawDescGZIP(), []int{109} } -func (x *RefreshStateByShardRequest) GetKeyspace() string { - if x != nil { - return x.Keyspace - } - return "" -} - -func (x *RefreshStateByShardRequest) GetShard() string { - if x != nil { - return x.Shard - } - return "" -} - -func (x *RefreshStateByShardRequest) GetCells() []string { +func (x *RebuildVSchemaGraphRequest) GetCells() []string { if x != nil { return x.Cells } return nil } -type RefreshStateByShardResponse struct { +type RebuildVSchemaGraphResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - - IsPartialRefresh bool `protobuf:"varint,1,opt,name=is_partial_refresh,json=isPartialRefresh,proto3" json:"is_partial_refresh,omitempty"` - // This explains why we had a partial refresh (if we did) - PartialRefreshDetails string `protobuf:"bytes,2,opt,name=partial_refresh_details,json=partialRefreshDetails,proto3" json:"partial_refresh_details,omitempty"` +} + +func (x *RebuildVSchemaGraphResponse) Reset() { + *x = RebuildVSchemaGraphResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[110] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RebuildVSchemaGraphResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RebuildVSchemaGraphResponse) ProtoMessage() {} + +func (x *RebuildVSchemaGraphResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[110] + 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 RebuildVSchemaGraphResponse.ProtoReflect.Descriptor instead. +func (*RebuildVSchemaGraphResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{110} +} + +type RefreshStateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TabletAlias *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet_alias,json=tabletAlias,proto3" json:"tablet_alias,omitempty"` +} + +func (x *RefreshStateRequest) Reset() { + *x = RefreshStateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[111] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RefreshStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RefreshStateRequest) ProtoMessage() {} + +func (x *RefreshStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[111] + 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 RefreshStateRequest.ProtoReflect.Descriptor instead. +func (*RefreshStateRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{111} +} + +func (x *RefreshStateRequest) GetTabletAlias() *topodata.TabletAlias { + if x != nil { + return x.TabletAlias + } + return nil +} + +type RefreshStateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RefreshStateResponse) Reset() { + *x = RefreshStateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[112] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RefreshStateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RefreshStateResponse) ProtoMessage() {} + +func (x *RefreshStateResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[112] + 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 RefreshStateResponse.ProtoReflect.Descriptor instead. +func (*RefreshStateResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{112} +} + +type RefreshStateByShardRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` + Cells []string `protobuf:"bytes,3,rep,name=cells,proto3" json:"cells,omitempty"` +} + +func (x *RefreshStateByShardRequest) Reset() { + *x = RefreshStateByShardRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[113] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RefreshStateByShardRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RefreshStateByShardRequest) ProtoMessage() {} + +func (x *RefreshStateByShardRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[113] + 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 RefreshStateByShardRequest.ProtoReflect.Descriptor instead. +func (*RefreshStateByShardRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{113} +} + +func (x *RefreshStateByShardRequest) GetKeyspace() string { + if x != nil { + return x.Keyspace + } + return "" +} + +func (x *RefreshStateByShardRequest) GetShard() string { + if x != nil { + return x.Shard + } + return "" +} + +func (x *RefreshStateByShardRequest) GetCells() []string { + if x != nil { + return x.Cells + } + return nil +} + +type RefreshStateByShardResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IsPartialRefresh bool `protobuf:"varint,1,opt,name=is_partial_refresh,json=isPartialRefresh,proto3" json:"is_partial_refresh,omitempty"` + // This explains why we had a partial refresh (if we did) + PartialRefreshDetails string `protobuf:"bytes,2,opt,name=partial_refresh_details,json=partialRefreshDetails,proto3" json:"partial_refresh_details,omitempty"` } func (x *RefreshStateByShardResponse) Reset() { *x = RefreshStateByShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[110] + mi := &file_vtctldata_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6450,7 +6848,7 @@ func (x *RefreshStateByShardResponse) String() string { func (*RefreshStateByShardResponse) ProtoMessage() {} func (x *RefreshStateByShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[110] + mi := &file_vtctldata_proto_msgTypes[114] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6463,7 +6861,7 @@ func (x *RefreshStateByShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RefreshStateByShardResponse.ProtoReflect.Descriptor instead. func (*RefreshStateByShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{110} + return file_vtctldata_proto_rawDescGZIP(), []int{114} } func (x *RefreshStateByShardResponse) GetIsPartialRefresh() bool { @@ -6491,7 +6889,7 @@ type ReloadSchemaRequest struct { func (x *ReloadSchemaRequest) Reset() { *x = ReloadSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[111] + mi := &file_vtctldata_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6504,7 +6902,7 @@ func (x *ReloadSchemaRequest) String() string { func (*ReloadSchemaRequest) ProtoMessage() {} func (x *ReloadSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[111] + mi := &file_vtctldata_proto_msgTypes[115] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6517,7 +6915,7 @@ func (x *ReloadSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{111} + return file_vtctldata_proto_rawDescGZIP(), []int{115} } func (x *ReloadSchemaRequest) GetTabletAlias() *topodata.TabletAlias { @@ -6536,7 +6934,7 @@ type ReloadSchemaResponse struct { func (x *ReloadSchemaResponse) Reset() { *x = ReloadSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[112] + mi := &file_vtctldata_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6549,7 +6947,7 @@ func (x *ReloadSchemaResponse) String() string { func (*ReloadSchemaResponse) ProtoMessage() {} func (x *ReloadSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[112] + mi := &file_vtctldata_proto_msgTypes[116] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6562,7 +6960,7 @@ func (x *ReloadSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{112} + return file_vtctldata_proto_rawDescGZIP(), []int{116} } type ReloadSchemaKeyspaceRequest struct { @@ -6582,7 +6980,7 @@ type ReloadSchemaKeyspaceRequest struct { func (x *ReloadSchemaKeyspaceRequest) Reset() { *x = ReloadSchemaKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[113] + mi := &file_vtctldata_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6595,7 +6993,7 @@ func (x *ReloadSchemaKeyspaceRequest) String() string { func (*ReloadSchemaKeyspaceRequest) ProtoMessage() {} func (x *ReloadSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[113] + mi := &file_vtctldata_proto_msgTypes[117] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6608,7 +7006,7 @@ func (x *ReloadSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{113} + return file_vtctldata_proto_rawDescGZIP(), []int{117} } func (x *ReloadSchemaKeyspaceRequest) GetKeyspace() string { @@ -6650,7 +7048,7 @@ type ReloadSchemaKeyspaceResponse struct { func (x *ReloadSchemaKeyspaceResponse) Reset() { *x = ReloadSchemaKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[114] + mi := &file_vtctldata_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6663,7 +7061,7 @@ func (x *ReloadSchemaKeyspaceResponse) String() string { func (*ReloadSchemaKeyspaceResponse) ProtoMessage() {} func (x *ReloadSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[114] + mi := &file_vtctldata_proto_msgTypes[118] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6676,7 +7074,7 @@ func (x *ReloadSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{114} + return file_vtctldata_proto_rawDescGZIP(), []int{118} } func (x *ReloadSchemaKeyspaceResponse) GetEvents() []*logutil.Event { @@ -6702,7 +7100,7 @@ type ReloadSchemaShardRequest struct { func (x *ReloadSchemaShardRequest) Reset() { *x = ReloadSchemaShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[115] + mi := &file_vtctldata_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6715,7 +7113,7 @@ func (x *ReloadSchemaShardRequest) String() string { func (*ReloadSchemaShardRequest) ProtoMessage() {} func (x *ReloadSchemaShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[115] + mi := &file_vtctldata_proto_msgTypes[119] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6728,7 +7126,7 @@ func (x *ReloadSchemaShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaShardRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{115} + return file_vtctldata_proto_rawDescGZIP(), []int{119} } func (x *ReloadSchemaShardRequest) GetKeyspace() string { @@ -6777,7 +7175,7 @@ type ReloadSchemaShardResponse struct { func (x *ReloadSchemaShardResponse) Reset() { *x = ReloadSchemaShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[116] + mi := &file_vtctldata_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6790,7 +7188,7 @@ func (x *ReloadSchemaShardResponse) String() string { func (*ReloadSchemaShardResponse) ProtoMessage() {} func (x *ReloadSchemaShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[116] + mi := &file_vtctldata_proto_msgTypes[120] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6803,7 +7201,7 @@ func (x *ReloadSchemaShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaShardResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{116} + return file_vtctldata_proto_rawDescGZIP(), []int{120} } func (x *ReloadSchemaShardResponse) GetEvents() []*logutil.Event { @@ -6826,7 +7224,7 @@ type RemoveBackupRequest struct { func (x *RemoveBackupRequest) Reset() { *x = RemoveBackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[117] + mi := &file_vtctldata_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6839,7 +7237,7 @@ func (x *RemoveBackupRequest) String() string { func (*RemoveBackupRequest) ProtoMessage() {} func (x *RemoveBackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[117] + mi := &file_vtctldata_proto_msgTypes[121] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6852,7 +7250,7 @@ func (x *RemoveBackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveBackupRequest.ProtoReflect.Descriptor instead. func (*RemoveBackupRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{117} + return file_vtctldata_proto_rawDescGZIP(), []int{121} } func (x *RemoveBackupRequest) GetKeyspace() string { @@ -6885,7 +7283,7 @@ type RemoveBackupResponse struct { func (x *RemoveBackupResponse) Reset() { *x = RemoveBackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[118] + mi := &file_vtctldata_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6898,7 +7296,7 @@ func (x *RemoveBackupResponse) String() string { func (*RemoveBackupResponse) ProtoMessage() {} func (x *RemoveBackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[118] + mi := &file_vtctldata_proto_msgTypes[122] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6911,7 +7309,7 @@ func (x *RemoveBackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveBackupResponse.ProtoReflect.Descriptor instead. func (*RemoveBackupResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{118} + return file_vtctldata_proto_rawDescGZIP(), []int{122} } type RemoveKeyspaceCellRequest struct { @@ -6933,7 +7331,7 @@ type RemoveKeyspaceCellRequest struct { func (x *RemoveKeyspaceCellRequest) Reset() { *x = RemoveKeyspaceCellRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[119] + mi := &file_vtctldata_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6946,7 +7344,7 @@ func (x *RemoveKeyspaceCellRequest) String() string { func (*RemoveKeyspaceCellRequest) ProtoMessage() {} func (x *RemoveKeyspaceCellRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[119] + mi := &file_vtctldata_proto_msgTypes[123] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6959,7 +7357,7 @@ func (x *RemoveKeyspaceCellRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveKeyspaceCellRequest.ProtoReflect.Descriptor instead. func (*RemoveKeyspaceCellRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{119} + return file_vtctldata_proto_rawDescGZIP(), []int{123} } func (x *RemoveKeyspaceCellRequest) GetKeyspace() string { @@ -6999,7 +7397,7 @@ type RemoveKeyspaceCellResponse struct { func (x *RemoveKeyspaceCellResponse) Reset() { *x = RemoveKeyspaceCellResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[120] + mi := &file_vtctldata_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7012,7 +7410,7 @@ func (x *RemoveKeyspaceCellResponse) String() string { func (*RemoveKeyspaceCellResponse) ProtoMessage() {} func (x *RemoveKeyspaceCellResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[120] + mi := &file_vtctldata_proto_msgTypes[124] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7025,7 +7423,7 @@ func (x *RemoveKeyspaceCellResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveKeyspaceCellResponse.ProtoReflect.Descriptor instead. func (*RemoveKeyspaceCellResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{120} + return file_vtctldata_proto_rawDescGZIP(), []int{124} } type RemoveShardCellRequest struct { @@ -7048,7 +7446,7 @@ type RemoveShardCellRequest struct { func (x *RemoveShardCellRequest) Reset() { *x = RemoveShardCellRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[121] + mi := &file_vtctldata_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7061,7 +7459,7 @@ func (x *RemoveShardCellRequest) String() string { func (*RemoveShardCellRequest) ProtoMessage() {} func (x *RemoveShardCellRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[121] + mi := &file_vtctldata_proto_msgTypes[125] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7074,7 +7472,7 @@ func (x *RemoveShardCellRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveShardCellRequest.ProtoReflect.Descriptor instead. func (*RemoveShardCellRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{121} + return file_vtctldata_proto_rawDescGZIP(), []int{125} } func (x *RemoveShardCellRequest) GetKeyspace() string { @@ -7121,7 +7519,7 @@ type RemoveShardCellResponse struct { func (x *RemoveShardCellResponse) Reset() { *x = RemoveShardCellResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[122] + mi := &file_vtctldata_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7134,7 +7532,7 @@ func (x *RemoveShardCellResponse) String() string { func (*RemoveShardCellResponse) ProtoMessage() {} func (x *RemoveShardCellResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[122] + mi := &file_vtctldata_proto_msgTypes[126] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7147,7 +7545,7 @@ func (x *RemoveShardCellResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveShardCellResponse.ProtoReflect.Descriptor instead. func (*RemoveShardCellResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{122} + return file_vtctldata_proto_rawDescGZIP(), []int{126} } type ReparentTabletRequest struct { @@ -7163,7 +7561,7 @@ type ReparentTabletRequest struct { func (x *ReparentTabletRequest) Reset() { *x = ReparentTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[123] + mi := &file_vtctldata_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7176,7 +7574,7 @@ func (x *ReparentTabletRequest) String() string { func (*ReparentTabletRequest) ProtoMessage() {} func (x *ReparentTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[123] + mi := &file_vtctldata_proto_msgTypes[127] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7189,7 +7587,7 @@ func (x *ReparentTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReparentTabletRequest.ProtoReflect.Descriptor instead. func (*ReparentTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{123} + return file_vtctldata_proto_rawDescGZIP(), []int{127} } func (x *ReparentTabletRequest) GetTablet() *topodata.TabletAlias { @@ -7215,7 +7613,7 @@ type ReparentTabletResponse struct { func (x *ReparentTabletResponse) Reset() { *x = ReparentTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[124] + mi := &file_vtctldata_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7228,7 +7626,7 @@ func (x *ReparentTabletResponse) String() string { func (*ReparentTabletResponse) ProtoMessage() {} func (x *ReparentTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[124] + mi := &file_vtctldata_proto_msgTypes[128] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7241,7 +7639,7 @@ func (x *ReparentTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReparentTabletResponse.ProtoReflect.Descriptor instead. func (*ReparentTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{124} + return file_vtctldata_proto_rawDescGZIP(), []int{128} } func (x *ReparentTabletResponse) GetKeyspace() string { @@ -7288,7 +7686,7 @@ type RestoreFromBackupRequest struct { func (x *RestoreFromBackupRequest) Reset() { *x = RestoreFromBackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[125] + mi := &file_vtctldata_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7301,7 +7699,7 @@ func (x *RestoreFromBackupRequest) String() string { func (*RestoreFromBackupRequest) ProtoMessage() {} func (x *RestoreFromBackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[125] + mi := &file_vtctldata_proto_msgTypes[129] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7314,7 +7712,7 @@ func (x *RestoreFromBackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RestoreFromBackupRequest.ProtoReflect.Descriptor instead. func (*RestoreFromBackupRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{125} + return file_vtctldata_proto_rawDescGZIP(), []int{129} } func (x *RestoreFromBackupRequest) GetTabletAlias() *topodata.TabletAlias { @@ -7367,7 +7765,7 @@ type RestoreFromBackupResponse struct { func (x *RestoreFromBackupResponse) Reset() { *x = RestoreFromBackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[126] + mi := &file_vtctldata_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7380,7 +7778,7 @@ func (x *RestoreFromBackupResponse) String() string { func (*RestoreFromBackupResponse) ProtoMessage() {} func (x *RestoreFromBackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[126] + mi := &file_vtctldata_proto_msgTypes[130] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7393,7 +7791,7 @@ func (x *RestoreFromBackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RestoreFromBackupResponse.ProtoReflect.Descriptor instead. func (*RestoreFromBackupResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{126} + return file_vtctldata_proto_rawDescGZIP(), []int{130} } func (x *RestoreFromBackupResponse) GetTabletAlias() *topodata.TabletAlias { @@ -7435,7 +7833,7 @@ type RunHealthCheckRequest struct { func (x *RunHealthCheckRequest) Reset() { *x = RunHealthCheckRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[127] + mi := &file_vtctldata_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7448,7 +7846,7 @@ func (x *RunHealthCheckRequest) String() string { func (*RunHealthCheckRequest) ProtoMessage() {} func (x *RunHealthCheckRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[127] + mi := &file_vtctldata_proto_msgTypes[131] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7461,7 +7859,7 @@ func (x *RunHealthCheckRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RunHealthCheckRequest.ProtoReflect.Descriptor instead. func (*RunHealthCheckRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{127} + return file_vtctldata_proto_rawDescGZIP(), []int{131} } func (x *RunHealthCheckRequest) GetTabletAlias() *topodata.TabletAlias { @@ -7480,7 +7878,7 @@ type RunHealthCheckResponse struct { func (x *RunHealthCheckResponse) Reset() { *x = RunHealthCheckResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[128] + mi := &file_vtctldata_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7493,7 +7891,7 @@ func (x *RunHealthCheckResponse) String() string { func (*RunHealthCheckResponse) ProtoMessage() {} func (x *RunHealthCheckResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[128] + mi := &file_vtctldata_proto_msgTypes[132] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7506,7 +7904,7 @@ func (x *RunHealthCheckResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RunHealthCheckResponse.ProtoReflect.Descriptor instead. func (*RunHealthCheckResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{128} + return file_vtctldata_proto_rawDescGZIP(), []int{132} } type SetKeyspaceDurabilityPolicyRequest struct { @@ -7521,7 +7919,7 @@ type SetKeyspaceDurabilityPolicyRequest struct { func (x *SetKeyspaceDurabilityPolicyRequest) Reset() { *x = SetKeyspaceDurabilityPolicyRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[129] + mi := &file_vtctldata_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7534,7 +7932,7 @@ func (x *SetKeyspaceDurabilityPolicyRequest) String() string { func (*SetKeyspaceDurabilityPolicyRequest) ProtoMessage() {} func (x *SetKeyspaceDurabilityPolicyRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[129] + mi := &file_vtctldata_proto_msgTypes[133] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7547,7 +7945,7 @@ func (x *SetKeyspaceDurabilityPolicyRequest) ProtoReflect() protoreflect.Message // Deprecated: Use SetKeyspaceDurabilityPolicyRequest.ProtoReflect.Descriptor instead. func (*SetKeyspaceDurabilityPolicyRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{129} + return file_vtctldata_proto_rawDescGZIP(), []int{133} } func (x *SetKeyspaceDurabilityPolicyRequest) GetKeyspace() string { @@ -7576,7 +7974,7 @@ type SetKeyspaceDurabilityPolicyResponse struct { func (x *SetKeyspaceDurabilityPolicyResponse) Reset() { *x = SetKeyspaceDurabilityPolicyResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[130] + mi := &file_vtctldata_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7589,7 +7987,7 @@ func (x *SetKeyspaceDurabilityPolicyResponse) String() string { func (*SetKeyspaceDurabilityPolicyResponse) ProtoMessage() {} func (x *SetKeyspaceDurabilityPolicyResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[130] + mi := &file_vtctldata_proto_msgTypes[134] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7602,7 +8000,7 @@ func (x *SetKeyspaceDurabilityPolicyResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use SetKeyspaceDurabilityPolicyResponse.ProtoReflect.Descriptor instead. func (*SetKeyspaceDurabilityPolicyResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{130} + return file_vtctldata_proto_rawDescGZIP(), []int{134} } func (x *SetKeyspaceDurabilityPolicyResponse) GetKeyspace() *topodata.Keyspace { @@ -7627,7 +8025,7 @@ type SetKeyspaceServedFromRequest struct { func (x *SetKeyspaceServedFromRequest) Reset() { *x = SetKeyspaceServedFromRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[131] + mi := &file_vtctldata_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7640,7 +8038,7 @@ func (x *SetKeyspaceServedFromRequest) String() string { func (*SetKeyspaceServedFromRequest) ProtoMessage() {} func (x *SetKeyspaceServedFromRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[131] + mi := &file_vtctldata_proto_msgTypes[135] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7653,7 +8051,7 @@ func (x *SetKeyspaceServedFromRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceServedFromRequest.ProtoReflect.Descriptor instead. func (*SetKeyspaceServedFromRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{131} + return file_vtctldata_proto_rawDescGZIP(), []int{135} } func (x *SetKeyspaceServedFromRequest) GetKeyspace() string { @@ -7703,7 +8101,7 @@ type SetKeyspaceServedFromResponse struct { func (x *SetKeyspaceServedFromResponse) Reset() { *x = SetKeyspaceServedFromResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[132] + mi := &file_vtctldata_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7716,7 +8114,7 @@ func (x *SetKeyspaceServedFromResponse) String() string { func (*SetKeyspaceServedFromResponse) ProtoMessage() {} func (x *SetKeyspaceServedFromResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[132] + mi := &file_vtctldata_proto_msgTypes[136] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7729,7 +8127,7 @@ func (x *SetKeyspaceServedFromResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceServedFromResponse.ProtoReflect.Descriptor instead. func (*SetKeyspaceServedFromResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{132} + return file_vtctldata_proto_rawDescGZIP(), []int{136} } func (x *SetKeyspaceServedFromResponse) GetKeyspace() *topodata.Keyspace { @@ -7751,7 +8149,7 @@ type SetKeyspaceShardingInfoRequest struct { func (x *SetKeyspaceShardingInfoRequest) Reset() { *x = SetKeyspaceShardingInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[133] + mi := &file_vtctldata_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7764,7 +8162,7 @@ func (x *SetKeyspaceShardingInfoRequest) String() string { func (*SetKeyspaceShardingInfoRequest) ProtoMessage() {} func (x *SetKeyspaceShardingInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[133] + mi := &file_vtctldata_proto_msgTypes[137] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7777,7 +8175,7 @@ func (x *SetKeyspaceShardingInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceShardingInfoRequest.ProtoReflect.Descriptor instead. func (*SetKeyspaceShardingInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{133} + return file_vtctldata_proto_rawDescGZIP(), []int{137} } func (x *SetKeyspaceShardingInfoRequest) GetKeyspace() string { @@ -7806,7 +8204,7 @@ type SetKeyspaceShardingInfoResponse struct { func (x *SetKeyspaceShardingInfoResponse) Reset() { *x = SetKeyspaceShardingInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[134] + mi := &file_vtctldata_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7819,7 +8217,7 @@ func (x *SetKeyspaceShardingInfoResponse) String() string { func (*SetKeyspaceShardingInfoResponse) ProtoMessage() {} func (x *SetKeyspaceShardingInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[134] + mi := &file_vtctldata_proto_msgTypes[138] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7832,7 +8230,7 @@ func (x *SetKeyspaceShardingInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceShardingInfoResponse.ProtoReflect.Descriptor instead. func (*SetKeyspaceShardingInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{134} + return file_vtctldata_proto_rawDescGZIP(), []int{138} } func (x *SetKeyspaceShardingInfoResponse) GetKeyspace() *topodata.Keyspace { @@ -7855,7 +8253,7 @@ type SetShardIsPrimaryServingRequest struct { func (x *SetShardIsPrimaryServingRequest) Reset() { *x = SetShardIsPrimaryServingRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[135] + mi := &file_vtctldata_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7868,7 +8266,7 @@ func (x *SetShardIsPrimaryServingRequest) String() string { func (*SetShardIsPrimaryServingRequest) ProtoMessage() {} func (x *SetShardIsPrimaryServingRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[135] + mi := &file_vtctldata_proto_msgTypes[139] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7881,7 +8279,7 @@ func (x *SetShardIsPrimaryServingRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardIsPrimaryServingRequest.ProtoReflect.Descriptor instead. func (*SetShardIsPrimaryServingRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{135} + return file_vtctldata_proto_rawDescGZIP(), []int{139} } func (x *SetShardIsPrimaryServingRequest) GetKeyspace() string { @@ -7917,7 +8315,7 @@ type SetShardIsPrimaryServingResponse struct { func (x *SetShardIsPrimaryServingResponse) Reset() { *x = SetShardIsPrimaryServingResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[136] + mi := &file_vtctldata_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7930,7 +8328,7 @@ func (x *SetShardIsPrimaryServingResponse) String() string { func (*SetShardIsPrimaryServingResponse) ProtoMessage() {} func (x *SetShardIsPrimaryServingResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[136] + mi := &file_vtctldata_proto_msgTypes[140] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7943,7 +8341,7 @@ func (x *SetShardIsPrimaryServingResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardIsPrimaryServingResponse.ProtoReflect.Descriptor instead. func (*SetShardIsPrimaryServingResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{136} + return file_vtctldata_proto_rawDescGZIP(), []int{140} } func (x *SetShardIsPrimaryServingResponse) GetShard() *topodata.Shard { @@ -7984,7 +8382,7 @@ type SetShardTabletControlRequest struct { func (x *SetShardTabletControlRequest) Reset() { *x = SetShardTabletControlRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[137] + mi := &file_vtctldata_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7997,7 +8395,7 @@ func (x *SetShardTabletControlRequest) String() string { func (*SetShardTabletControlRequest) ProtoMessage() {} func (x *SetShardTabletControlRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[137] + mi := &file_vtctldata_proto_msgTypes[141] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8010,7 +8408,7 @@ func (x *SetShardTabletControlRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardTabletControlRequest.ProtoReflect.Descriptor instead. func (*SetShardTabletControlRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{137} + return file_vtctldata_proto_rawDescGZIP(), []int{141} } func (x *SetShardTabletControlRequest) GetKeyspace() string { @@ -8074,7 +8472,7 @@ type SetShardTabletControlResponse struct { func (x *SetShardTabletControlResponse) Reset() { *x = SetShardTabletControlResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[138] + mi := &file_vtctldata_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8087,7 +8485,7 @@ func (x *SetShardTabletControlResponse) String() string { func (*SetShardTabletControlResponse) ProtoMessage() {} func (x *SetShardTabletControlResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[138] + mi := &file_vtctldata_proto_msgTypes[142] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8100,7 +8498,7 @@ func (x *SetShardTabletControlResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardTabletControlResponse.ProtoReflect.Descriptor instead. func (*SetShardTabletControlResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{138} + return file_vtctldata_proto_rawDescGZIP(), []int{142} } func (x *SetShardTabletControlResponse) GetShard() *topodata.Shard { @@ -8122,7 +8520,7 @@ type SetWritableRequest struct { func (x *SetWritableRequest) Reset() { *x = SetWritableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[139] + mi := &file_vtctldata_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8135,7 +8533,7 @@ func (x *SetWritableRequest) String() string { func (*SetWritableRequest) ProtoMessage() {} func (x *SetWritableRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[139] + mi := &file_vtctldata_proto_msgTypes[143] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8148,7 +8546,7 @@ func (x *SetWritableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetWritableRequest.ProtoReflect.Descriptor instead. func (*SetWritableRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{139} + return file_vtctldata_proto_rawDescGZIP(), []int{143} } func (x *SetWritableRequest) GetTabletAlias() *topodata.TabletAlias { @@ -8174,7 +8572,7 @@ type SetWritableResponse struct { func (x *SetWritableResponse) Reset() { *x = SetWritableResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[140] + mi := &file_vtctldata_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8187,7 +8585,7 @@ func (x *SetWritableResponse) String() string { func (*SetWritableResponse) ProtoMessage() {} func (x *SetWritableResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[140] + mi := &file_vtctldata_proto_msgTypes[144] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8200,7 +8598,7 @@ func (x *SetWritableResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetWritableResponse.ProtoReflect.Descriptor instead. func (*SetWritableResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{140} + return file_vtctldata_proto_rawDescGZIP(), []int{144} } type ShardReplicationAddRequest struct { @@ -8216,7 +8614,7 @@ type ShardReplicationAddRequest struct { func (x *ShardReplicationAddRequest) Reset() { *x = ShardReplicationAddRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[141] + mi := &file_vtctldata_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8229,7 +8627,7 @@ func (x *ShardReplicationAddRequest) String() string { func (*ShardReplicationAddRequest) ProtoMessage() {} func (x *ShardReplicationAddRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[141] + mi := &file_vtctldata_proto_msgTypes[145] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8242,7 +8640,7 @@ func (x *ShardReplicationAddRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationAddRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationAddRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{141} + return file_vtctldata_proto_rawDescGZIP(), []int{145} } func (x *ShardReplicationAddRequest) GetKeyspace() string { @@ -8275,7 +8673,7 @@ type ShardReplicationAddResponse struct { func (x *ShardReplicationAddResponse) Reset() { *x = ShardReplicationAddResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[142] + mi := &file_vtctldata_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8288,7 +8686,7 @@ func (x *ShardReplicationAddResponse) String() string { func (*ShardReplicationAddResponse) ProtoMessage() {} func (x *ShardReplicationAddResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[142] + mi := &file_vtctldata_proto_msgTypes[146] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8301,7 +8699,7 @@ func (x *ShardReplicationAddResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationAddResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationAddResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{142} + return file_vtctldata_proto_rawDescGZIP(), []int{146} } type ShardReplicationFixRequest struct { @@ -8317,7 +8715,7 @@ type ShardReplicationFixRequest struct { func (x *ShardReplicationFixRequest) Reset() { *x = ShardReplicationFixRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[143] + mi := &file_vtctldata_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8330,7 +8728,7 @@ func (x *ShardReplicationFixRequest) String() string { func (*ShardReplicationFixRequest) ProtoMessage() {} func (x *ShardReplicationFixRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[143] + mi := &file_vtctldata_proto_msgTypes[147] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8343,7 +8741,7 @@ func (x *ShardReplicationFixRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationFixRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationFixRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{143} + return file_vtctldata_proto_rawDescGZIP(), []int{147} } func (x *ShardReplicationFixRequest) GetKeyspace() string { @@ -8381,7 +8779,7 @@ type ShardReplicationFixResponse struct { func (x *ShardReplicationFixResponse) Reset() { *x = ShardReplicationFixResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[144] + mi := &file_vtctldata_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8394,7 +8792,7 @@ func (x *ShardReplicationFixResponse) String() string { func (*ShardReplicationFixResponse) ProtoMessage() {} func (x *ShardReplicationFixResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[144] + mi := &file_vtctldata_proto_msgTypes[148] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8407,7 +8805,7 @@ func (x *ShardReplicationFixResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationFixResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationFixResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{144} + return file_vtctldata_proto_rawDescGZIP(), []int{148} } func (x *ShardReplicationFixResponse) GetError() *topodata.ShardReplicationError { @@ -8429,7 +8827,7 @@ type ShardReplicationPositionsRequest struct { func (x *ShardReplicationPositionsRequest) Reset() { *x = ShardReplicationPositionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[145] + mi := &file_vtctldata_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8442,7 +8840,7 @@ func (x *ShardReplicationPositionsRequest) String() string { func (*ShardReplicationPositionsRequest) ProtoMessage() {} func (x *ShardReplicationPositionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[145] + mi := &file_vtctldata_proto_msgTypes[149] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8455,7 +8853,7 @@ func (x *ShardReplicationPositionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationPositionsRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationPositionsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{145} + return file_vtctldata_proto_rawDescGZIP(), []int{149} } func (x *ShardReplicationPositionsRequest) GetKeyspace() string { @@ -8488,7 +8886,7 @@ type ShardReplicationPositionsResponse struct { func (x *ShardReplicationPositionsResponse) Reset() { *x = ShardReplicationPositionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[146] + mi := &file_vtctldata_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8501,7 +8899,7 @@ func (x *ShardReplicationPositionsResponse) String() string { func (*ShardReplicationPositionsResponse) ProtoMessage() {} func (x *ShardReplicationPositionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[146] + mi := &file_vtctldata_proto_msgTypes[150] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8514,7 +8912,7 @@ func (x *ShardReplicationPositionsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use ShardReplicationPositionsResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationPositionsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{146} + return file_vtctldata_proto_rawDescGZIP(), []int{150} } func (x *ShardReplicationPositionsResponse) GetReplicationStatuses() map[string]*replicationdata.Status { @@ -8544,7 +8942,7 @@ type ShardReplicationRemoveRequest struct { func (x *ShardReplicationRemoveRequest) Reset() { *x = ShardReplicationRemoveRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[147] + mi := &file_vtctldata_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8557,7 +8955,7 @@ func (x *ShardReplicationRemoveRequest) String() string { func (*ShardReplicationRemoveRequest) ProtoMessage() {} func (x *ShardReplicationRemoveRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[147] + mi := &file_vtctldata_proto_msgTypes[151] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8570,7 +8968,7 @@ func (x *ShardReplicationRemoveRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationRemoveRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationRemoveRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{147} + return file_vtctldata_proto_rawDescGZIP(), []int{151} } func (x *ShardReplicationRemoveRequest) GetKeyspace() string { @@ -8603,7 +9001,7 @@ type ShardReplicationRemoveResponse struct { func (x *ShardReplicationRemoveResponse) Reset() { *x = ShardReplicationRemoveResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[148] + mi := &file_vtctldata_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8616,7 +9014,7 @@ func (x *ShardReplicationRemoveResponse) String() string { func (*ShardReplicationRemoveResponse) ProtoMessage() {} func (x *ShardReplicationRemoveResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[148] + mi := &file_vtctldata_proto_msgTypes[152] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8629,7 +9027,7 @@ func (x *ShardReplicationRemoveResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationRemoveResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationRemoveResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{148} + return file_vtctldata_proto_rawDescGZIP(), []int{152} } type SleepTabletRequest struct { @@ -8644,7 +9042,7 @@ type SleepTabletRequest struct { func (x *SleepTabletRequest) Reset() { *x = SleepTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[149] + mi := &file_vtctldata_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8657,7 +9055,7 @@ func (x *SleepTabletRequest) String() string { func (*SleepTabletRequest) ProtoMessage() {} func (x *SleepTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[149] + mi := &file_vtctldata_proto_msgTypes[153] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8670,7 +9068,7 @@ func (x *SleepTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SleepTabletRequest.ProtoReflect.Descriptor instead. func (*SleepTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{149} + return file_vtctldata_proto_rawDescGZIP(), []int{153} } func (x *SleepTabletRequest) GetTabletAlias() *topodata.TabletAlias { @@ -8696,7 +9094,7 @@ type SleepTabletResponse struct { func (x *SleepTabletResponse) Reset() { *x = SleepTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[150] + mi := &file_vtctldata_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8709,7 +9107,7 @@ func (x *SleepTabletResponse) String() string { func (*SleepTabletResponse) ProtoMessage() {} func (x *SleepTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[150] + mi := &file_vtctldata_proto_msgTypes[154] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8722,7 +9120,7 @@ func (x *SleepTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SleepTabletResponse.ProtoReflect.Descriptor instead. func (*SleepTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{150} + return file_vtctldata_proto_rawDescGZIP(), []int{154} } type SourceShardAddRequest struct { @@ -8746,7 +9144,7 @@ type SourceShardAddRequest struct { func (x *SourceShardAddRequest) Reset() { *x = SourceShardAddRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[151] + mi := &file_vtctldata_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8759,7 +9157,7 @@ func (x *SourceShardAddRequest) String() string { func (*SourceShardAddRequest) ProtoMessage() {} func (x *SourceShardAddRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[151] + mi := &file_vtctldata_proto_msgTypes[155] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8772,7 +9170,7 @@ func (x *SourceShardAddRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardAddRequest.ProtoReflect.Descriptor instead. func (*SourceShardAddRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{151} + return file_vtctldata_proto_rawDescGZIP(), []int{155} } func (x *SourceShardAddRequest) GetKeyspace() string { @@ -8836,7 +9234,7 @@ type SourceShardAddResponse struct { func (x *SourceShardAddResponse) Reset() { *x = SourceShardAddResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[152] + mi := &file_vtctldata_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8849,7 +9247,7 @@ func (x *SourceShardAddResponse) String() string { func (*SourceShardAddResponse) ProtoMessage() {} func (x *SourceShardAddResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[152] + mi := &file_vtctldata_proto_msgTypes[156] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8862,7 +9260,7 @@ func (x *SourceShardAddResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardAddResponse.ProtoReflect.Descriptor instead. func (*SourceShardAddResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{152} + return file_vtctldata_proto_rawDescGZIP(), []int{156} } func (x *SourceShardAddResponse) GetShard() *topodata.Shard { @@ -8885,7 +9283,7 @@ type SourceShardDeleteRequest struct { func (x *SourceShardDeleteRequest) Reset() { *x = SourceShardDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[153] + mi := &file_vtctldata_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8898,7 +9296,7 @@ func (x *SourceShardDeleteRequest) String() string { func (*SourceShardDeleteRequest) ProtoMessage() {} func (x *SourceShardDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[153] + mi := &file_vtctldata_proto_msgTypes[157] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8911,7 +9309,7 @@ func (x *SourceShardDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardDeleteRequest.ProtoReflect.Descriptor instead. func (*SourceShardDeleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{153} + return file_vtctldata_proto_rawDescGZIP(), []int{157} } func (x *SourceShardDeleteRequest) GetKeyspace() string { @@ -8947,7 +9345,7 @@ type SourceShardDeleteResponse struct { func (x *SourceShardDeleteResponse) Reset() { *x = SourceShardDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[154] + mi := &file_vtctldata_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8960,7 +9358,7 @@ func (x *SourceShardDeleteResponse) String() string { func (*SourceShardDeleteResponse) ProtoMessage() {} func (x *SourceShardDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[154] + mi := &file_vtctldata_proto_msgTypes[158] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8973,7 +9371,7 @@ func (x *SourceShardDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardDeleteResponse.ProtoReflect.Descriptor instead. func (*SourceShardDeleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{154} + return file_vtctldata_proto_rawDescGZIP(), []int{158} } func (x *SourceShardDeleteResponse) GetShard() *topodata.Shard { @@ -8994,7 +9392,7 @@ type StartReplicationRequest struct { func (x *StartReplicationRequest) Reset() { *x = StartReplicationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[155] + mi := &file_vtctldata_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9007,7 +9405,7 @@ func (x *StartReplicationRequest) String() string { func (*StartReplicationRequest) ProtoMessage() {} func (x *StartReplicationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[155] + mi := &file_vtctldata_proto_msgTypes[159] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9020,7 +9418,7 @@ func (x *StartReplicationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StartReplicationRequest.ProtoReflect.Descriptor instead. func (*StartReplicationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{155} + return file_vtctldata_proto_rawDescGZIP(), []int{159} } func (x *StartReplicationRequest) GetTabletAlias() *topodata.TabletAlias { @@ -9039,7 +9437,7 @@ type StartReplicationResponse struct { func (x *StartReplicationResponse) Reset() { *x = StartReplicationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[156] + mi := &file_vtctldata_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9052,7 +9450,7 @@ func (x *StartReplicationResponse) String() string { func (*StartReplicationResponse) ProtoMessage() {} func (x *StartReplicationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[156] + mi := &file_vtctldata_proto_msgTypes[160] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9065,7 +9463,7 @@ func (x *StartReplicationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StartReplicationResponse.ProtoReflect.Descriptor instead. func (*StartReplicationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{156} + return file_vtctldata_proto_rawDescGZIP(), []int{160} } type StopReplicationRequest struct { @@ -9079,7 +9477,7 @@ type StopReplicationRequest struct { func (x *StopReplicationRequest) Reset() { *x = StopReplicationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[157] + mi := &file_vtctldata_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9092,7 +9490,7 @@ func (x *StopReplicationRequest) String() string { func (*StopReplicationRequest) ProtoMessage() {} func (x *StopReplicationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[157] + mi := &file_vtctldata_proto_msgTypes[161] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9105,7 +9503,7 @@ func (x *StopReplicationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StopReplicationRequest.ProtoReflect.Descriptor instead. func (*StopReplicationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{157} + return file_vtctldata_proto_rawDescGZIP(), []int{161} } func (x *StopReplicationRequest) GetTabletAlias() *topodata.TabletAlias { @@ -9124,7 +9522,7 @@ type StopReplicationResponse struct { func (x *StopReplicationResponse) Reset() { *x = StopReplicationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[158] + mi := &file_vtctldata_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9137,7 +9535,7 @@ func (x *StopReplicationResponse) String() string { func (*StopReplicationResponse) ProtoMessage() {} func (x *StopReplicationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[158] + mi := &file_vtctldata_proto_msgTypes[162] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9150,7 +9548,7 @@ func (x *StopReplicationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StopReplicationResponse.ProtoReflect.Descriptor instead. func (*StopReplicationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{158} + return file_vtctldata_proto_rawDescGZIP(), []int{162} } type TabletExternallyReparentedRequest struct { @@ -9166,7 +9564,7 @@ type TabletExternallyReparentedRequest struct { func (x *TabletExternallyReparentedRequest) Reset() { *x = TabletExternallyReparentedRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[159] + mi := &file_vtctldata_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9179,7 +9577,7 @@ func (x *TabletExternallyReparentedRequest) String() string { func (*TabletExternallyReparentedRequest) ProtoMessage() {} func (x *TabletExternallyReparentedRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[159] + mi := &file_vtctldata_proto_msgTypes[163] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9192,7 +9590,7 @@ func (x *TabletExternallyReparentedRequest) ProtoReflect() protoreflect.Message // Deprecated: Use TabletExternallyReparentedRequest.ProtoReflect.Descriptor instead. func (*TabletExternallyReparentedRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{159} + return file_vtctldata_proto_rawDescGZIP(), []int{163} } func (x *TabletExternallyReparentedRequest) GetTablet() *topodata.TabletAlias { @@ -9216,7 +9614,7 @@ type TabletExternallyReparentedResponse struct { func (x *TabletExternallyReparentedResponse) Reset() { *x = TabletExternallyReparentedResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[160] + mi := &file_vtctldata_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9229,7 +9627,7 @@ func (x *TabletExternallyReparentedResponse) String() string { func (*TabletExternallyReparentedResponse) ProtoMessage() {} func (x *TabletExternallyReparentedResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[160] + mi := &file_vtctldata_proto_msgTypes[164] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9242,7 +9640,7 @@ func (x *TabletExternallyReparentedResponse) ProtoReflect() protoreflect.Message // Deprecated: Use TabletExternallyReparentedResponse.ProtoReflect.Descriptor instead. func (*TabletExternallyReparentedResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{160} + return file_vtctldata_proto_rawDescGZIP(), []int{164} } func (x *TabletExternallyReparentedResponse) GetKeyspace() string { @@ -9285,7 +9683,7 @@ type UpdateCellInfoRequest struct { func (x *UpdateCellInfoRequest) Reset() { *x = UpdateCellInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[161] + mi := &file_vtctldata_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9298,7 +9696,7 @@ func (x *UpdateCellInfoRequest) String() string { func (*UpdateCellInfoRequest) ProtoMessage() {} func (x *UpdateCellInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[161] + mi := &file_vtctldata_proto_msgTypes[165] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9311,7 +9709,7 @@ func (x *UpdateCellInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellInfoRequest.ProtoReflect.Descriptor instead. func (*UpdateCellInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{161} + return file_vtctldata_proto_rawDescGZIP(), []int{165} } func (x *UpdateCellInfoRequest) GetName() string { @@ -9340,7 +9738,7 @@ type UpdateCellInfoResponse struct { func (x *UpdateCellInfoResponse) Reset() { *x = UpdateCellInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[162] + mi := &file_vtctldata_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9353,7 +9751,7 @@ func (x *UpdateCellInfoResponse) String() string { func (*UpdateCellInfoResponse) ProtoMessage() {} func (x *UpdateCellInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[162] + mi := &file_vtctldata_proto_msgTypes[166] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9366,7 +9764,7 @@ func (x *UpdateCellInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellInfoResponse.ProtoReflect.Descriptor instead. func (*UpdateCellInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{162} + return file_vtctldata_proto_rawDescGZIP(), []int{166} } func (x *UpdateCellInfoResponse) GetName() string { @@ -9395,7 +9793,7 @@ type UpdateCellsAliasRequest struct { func (x *UpdateCellsAliasRequest) Reset() { *x = UpdateCellsAliasRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[163] + mi := &file_vtctldata_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9408,7 +9806,7 @@ func (x *UpdateCellsAliasRequest) String() string { func (*UpdateCellsAliasRequest) ProtoMessage() {} func (x *UpdateCellsAliasRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[163] + mi := &file_vtctldata_proto_msgTypes[167] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9421,7 +9819,7 @@ func (x *UpdateCellsAliasRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellsAliasRequest.ProtoReflect.Descriptor instead. func (*UpdateCellsAliasRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{163} + return file_vtctldata_proto_rawDescGZIP(), []int{167} } func (x *UpdateCellsAliasRequest) GetName() string { @@ -9450,7 +9848,7 @@ type UpdateCellsAliasResponse struct { func (x *UpdateCellsAliasResponse) Reset() { *x = UpdateCellsAliasResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[164] + mi := &file_vtctldata_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9463,7 +9861,7 @@ func (x *UpdateCellsAliasResponse) String() string { func (*UpdateCellsAliasResponse) ProtoMessage() {} func (x *UpdateCellsAliasResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[164] + mi := &file_vtctldata_proto_msgTypes[168] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9476,7 +9874,7 @@ func (x *UpdateCellsAliasResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellsAliasResponse.ProtoReflect.Descriptor instead. func (*UpdateCellsAliasResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{164} + return file_vtctldata_proto_rawDescGZIP(), []int{168} } func (x *UpdateCellsAliasResponse) GetName() string { @@ -9504,7 +9902,7 @@ type ValidateRequest struct { func (x *ValidateRequest) Reset() { *x = ValidateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[165] + mi := &file_vtctldata_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9517,7 +9915,7 @@ func (x *ValidateRequest) String() string { func (*ValidateRequest) ProtoMessage() {} func (x *ValidateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[165] + mi := &file_vtctldata_proto_msgTypes[169] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9530,7 +9928,7 @@ func (x *ValidateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateRequest.ProtoReflect.Descriptor instead. func (*ValidateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{165} + return file_vtctldata_proto_rawDescGZIP(), []int{169} } func (x *ValidateRequest) GetPingTablets() bool { @@ -9552,7 +9950,7 @@ type ValidateResponse struct { func (x *ValidateResponse) Reset() { *x = ValidateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[166] + mi := &file_vtctldata_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9565,7 +9963,7 @@ func (x *ValidateResponse) String() string { func (*ValidateResponse) ProtoMessage() {} func (x *ValidateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[166] + mi := &file_vtctldata_proto_msgTypes[170] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9578,7 +9976,7 @@ func (x *ValidateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateResponse.ProtoReflect.Descriptor instead. func (*ValidateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{166} + return file_vtctldata_proto_rawDescGZIP(), []int{170} } func (x *ValidateResponse) GetResults() []string { @@ -9607,7 +10005,7 @@ type ValidateKeyspaceRequest struct { func (x *ValidateKeyspaceRequest) Reset() { *x = ValidateKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[167] + mi := &file_vtctldata_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9620,7 +10018,7 @@ func (x *ValidateKeyspaceRequest) String() string { func (*ValidateKeyspaceRequest) ProtoMessage() {} func (x *ValidateKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[167] + mi := &file_vtctldata_proto_msgTypes[171] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9633,7 +10031,7 @@ func (x *ValidateKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{167} + return file_vtctldata_proto_rawDescGZIP(), []int{171} } func (x *ValidateKeyspaceRequest) GetKeyspace() string { @@ -9662,7 +10060,7 @@ type ValidateKeyspaceResponse struct { func (x *ValidateKeyspaceResponse) Reset() { *x = ValidateKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[168] + mi := &file_vtctldata_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9675,7 +10073,7 @@ func (x *ValidateKeyspaceResponse) String() string { func (*ValidateKeyspaceResponse) ProtoMessage() {} func (x *ValidateKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[168] + mi := &file_vtctldata_proto_msgTypes[172] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9688,7 +10086,7 @@ func (x *ValidateKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{168} + return file_vtctldata_proto_rawDescGZIP(), []int{172} } func (x *ValidateKeyspaceResponse) GetResults() []string { @@ -9720,7 +10118,7 @@ type ValidateSchemaKeyspaceRequest struct { func (x *ValidateSchemaKeyspaceRequest) Reset() { *x = ValidateSchemaKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[169] + mi := &file_vtctldata_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9733,7 +10131,7 @@ func (x *ValidateSchemaKeyspaceRequest) String() string { func (*ValidateSchemaKeyspaceRequest) ProtoMessage() {} func (x *ValidateSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[169] + mi := &file_vtctldata_proto_msgTypes[173] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9746,7 +10144,7 @@ func (x *ValidateSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateSchemaKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateSchemaKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{169} + return file_vtctldata_proto_rawDescGZIP(), []int{173} } func (x *ValidateSchemaKeyspaceRequest) GetKeyspace() string { @@ -9796,7 +10194,7 @@ type ValidateSchemaKeyspaceResponse struct { func (x *ValidateSchemaKeyspaceResponse) Reset() { *x = ValidateSchemaKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[170] + mi := &file_vtctldata_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9809,7 +10207,7 @@ func (x *ValidateSchemaKeyspaceResponse) String() string { func (*ValidateSchemaKeyspaceResponse) ProtoMessage() {} func (x *ValidateSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[170] + mi := &file_vtctldata_proto_msgTypes[174] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9822,7 +10220,7 @@ func (x *ValidateSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateSchemaKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateSchemaKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{170} + return file_vtctldata_proto_rawDescGZIP(), []int{174} } func (x *ValidateSchemaKeyspaceResponse) GetResults() []string { @@ -9852,7 +10250,7 @@ type ValidateShardRequest struct { func (x *ValidateShardRequest) Reset() { *x = ValidateShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[171] + mi := &file_vtctldata_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9865,7 +10263,7 @@ func (x *ValidateShardRequest) String() string { func (*ValidateShardRequest) ProtoMessage() {} func (x *ValidateShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[171] + mi := &file_vtctldata_proto_msgTypes[175] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9878,7 +10276,7 @@ func (x *ValidateShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateShardRequest.ProtoReflect.Descriptor instead. func (*ValidateShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{171} + return file_vtctldata_proto_rawDescGZIP(), []int{175} } func (x *ValidateShardRequest) GetKeyspace() string { @@ -9913,7 +10311,7 @@ type ValidateShardResponse struct { func (x *ValidateShardResponse) Reset() { *x = ValidateShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[172] + mi := &file_vtctldata_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9926,7 +10324,7 @@ func (x *ValidateShardResponse) String() string { func (*ValidateShardResponse) ProtoMessage() {} func (x *ValidateShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[172] + mi := &file_vtctldata_proto_msgTypes[176] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9939,7 +10337,7 @@ func (x *ValidateShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateShardResponse.ProtoReflect.Descriptor instead. func (*ValidateShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{172} + return file_vtctldata_proto_rawDescGZIP(), []int{176} } func (x *ValidateShardResponse) GetResults() []string { @@ -9960,7 +10358,7 @@ type ValidateVersionKeyspaceRequest struct { func (x *ValidateVersionKeyspaceRequest) Reset() { *x = ValidateVersionKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[173] + mi := &file_vtctldata_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9973,7 +10371,7 @@ func (x *ValidateVersionKeyspaceRequest) String() string { func (*ValidateVersionKeyspaceRequest) ProtoMessage() {} func (x *ValidateVersionKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[173] + mi := &file_vtctldata_proto_msgTypes[177] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9986,7 +10384,7 @@ func (x *ValidateVersionKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateVersionKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{173} + return file_vtctldata_proto_rawDescGZIP(), []int{177} } func (x *ValidateVersionKeyspaceRequest) GetKeyspace() string { @@ -10008,7 +10406,7 @@ type ValidateVersionKeyspaceResponse struct { func (x *ValidateVersionKeyspaceResponse) Reset() { *x = ValidateVersionKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[174] + mi := &file_vtctldata_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10021,7 +10419,7 @@ func (x *ValidateVersionKeyspaceResponse) String() string { func (*ValidateVersionKeyspaceResponse) ProtoMessage() {} func (x *ValidateVersionKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[174] + mi := &file_vtctldata_proto_msgTypes[178] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10034,7 +10432,7 @@ func (x *ValidateVersionKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateVersionKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{174} + return file_vtctldata_proto_rawDescGZIP(), []int{178} } func (x *ValidateVersionKeyspaceResponse) GetResults() []string { @@ -10063,7 +10461,7 @@ type ValidateVersionShardRequest struct { func (x *ValidateVersionShardRequest) Reset() { *x = ValidateVersionShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[175] + mi := &file_vtctldata_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10076,7 +10474,7 @@ func (x *ValidateVersionShardRequest) String() string { func (*ValidateVersionShardRequest) ProtoMessage() {} func (x *ValidateVersionShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[175] + mi := &file_vtctldata_proto_msgTypes[179] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10089,7 +10487,7 @@ func (x *ValidateVersionShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionShardRequest.ProtoReflect.Descriptor instead. func (*ValidateVersionShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{175} + return file_vtctldata_proto_rawDescGZIP(), []int{179} } func (x *ValidateVersionShardRequest) GetKeyspace() string { @@ -10117,7 +10515,7 @@ type ValidateVersionShardResponse struct { func (x *ValidateVersionShardResponse) Reset() { *x = ValidateVersionShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[176] + mi := &file_vtctldata_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10130,7 +10528,7 @@ func (x *ValidateVersionShardResponse) String() string { func (*ValidateVersionShardResponse) ProtoMessage() {} func (x *ValidateVersionShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[176] + mi := &file_vtctldata_proto_msgTypes[180] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10143,7 +10541,7 @@ func (x *ValidateVersionShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionShardResponse.ProtoReflect.Descriptor instead. func (*ValidateVersionShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{176} + return file_vtctldata_proto_rawDescGZIP(), []int{180} } func (x *ValidateVersionShardResponse) GetResults() []string { @@ -10167,7 +10565,7 @@ type ValidateVSchemaRequest struct { func (x *ValidateVSchemaRequest) Reset() { *x = ValidateVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[177] + mi := &file_vtctldata_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10180,7 +10578,7 @@ func (x *ValidateVSchemaRequest) String() string { func (*ValidateVSchemaRequest) ProtoMessage() {} func (x *ValidateVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[177] + mi := &file_vtctldata_proto_msgTypes[181] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10193,7 +10591,7 @@ func (x *ValidateVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVSchemaRequest.ProtoReflect.Descriptor instead. func (*ValidateVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{177} + return file_vtctldata_proto_rawDescGZIP(), []int{181} } func (x *ValidateVSchemaRequest) GetKeyspace() string { @@ -10236,7 +10634,7 @@ type ValidateVSchemaResponse struct { func (x *ValidateVSchemaResponse) Reset() { *x = ValidateVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[178] + mi := &file_vtctldata_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10249,7 +10647,7 @@ func (x *ValidateVSchemaResponse) String() string { func (*ValidateVSchemaResponse) ProtoMessage() {} func (x *ValidateVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[178] + mi := &file_vtctldata_proto_msgTypes[182] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10262,7 +10660,7 @@ func (x *ValidateVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVSchemaResponse.ProtoReflect.Descriptor instead. func (*ValidateVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{178} + return file_vtctldata_proto_rawDescGZIP(), []int{182} } func (x *ValidateVSchemaResponse) GetResults() []string { @@ -10279,34 +10677,34 @@ func (x *ValidateVSchemaResponse) GetResultsByShard() map[string]*ValidateShardR return nil } -type WorkflowUpdateRequest struct { +type WorkflowDeleteRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - // TabletRequest gets passed on to each primary tablet involved - // in the workflow via the UpdateVRWorkflow tabletmanager RPC. - TabletRequest *tabletmanagerdata.UpdateVRWorkflowRequest `protobuf:"bytes,2,opt,name=tablet_request,json=tabletRequest,proto3" json:"tablet_request,omitempty"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` + KeepData bool `protobuf:"varint,3,opt,name=keep_data,json=keepData,proto3" json:"keep_data,omitempty"` + KeepRoutingRules bool `protobuf:"varint,4,opt,name=keep_routing_rules,json=keepRoutingRules,proto3" json:"keep_routing_rules,omitempty"` } -func (x *WorkflowUpdateRequest) Reset() { - *x = WorkflowUpdateRequest{} +func (x *WorkflowDeleteRequest) Reset() { + *x = WorkflowDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[179] + mi := &file_vtctldata_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *WorkflowUpdateRequest) String() string { +func (x *WorkflowDeleteRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*WorkflowUpdateRequest) ProtoMessage() {} +func (*WorkflowDeleteRequest) ProtoMessage() {} -func (x *WorkflowUpdateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[179] +func (x *WorkflowDeleteRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[183] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10317,51 +10715,65 @@ func (x *WorkflowUpdateRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WorkflowUpdateRequest.ProtoReflect.Descriptor instead. -func (*WorkflowUpdateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{179} +// Deprecated: Use WorkflowDeleteRequest.ProtoReflect.Descriptor instead. +func (*WorkflowDeleteRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{183} } -func (x *WorkflowUpdateRequest) GetKeyspace() string { +func (x *WorkflowDeleteRequest) GetKeyspace() string { if x != nil { return x.Keyspace } return "" } -func (x *WorkflowUpdateRequest) GetTabletRequest() *tabletmanagerdata.UpdateVRWorkflowRequest { +func (x *WorkflowDeleteRequest) GetWorkflow() string { if x != nil { - return x.TabletRequest + return x.Workflow } - return nil + return "" } -type WorkflowUpdateResponse struct { +func (x *WorkflowDeleteRequest) GetKeepData() bool { + if x != nil { + return x.KeepData + } + return false +} + +func (x *WorkflowDeleteRequest) GetKeepRoutingRules() bool { + if x != nil { + return x.KeepRoutingRules + } + return false +} + +type WorkflowDeleteResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields Summary string `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` - Details []*WorkflowUpdateResponse_TabletInfo `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty"` + Details []*WorkflowDeleteResponse_TabletInfo `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty"` } -func (x *WorkflowUpdateResponse) Reset() { - *x = WorkflowUpdateResponse{} +func (x *WorkflowDeleteResponse) Reset() { + *x = WorkflowDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[180] + mi := &file_vtctldata_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *WorkflowUpdateResponse) String() string { +func (x *WorkflowDeleteResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*WorkflowUpdateResponse) ProtoMessage() {} +func (*WorkflowDeleteResponse) ProtoMessage() {} -func (x *WorkflowUpdateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[180] +func (x *WorkflowDeleteResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[184] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10372,51 +10784,51 @@ func (x *WorkflowUpdateResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WorkflowUpdateResponse.ProtoReflect.Descriptor instead. -func (*WorkflowUpdateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{180} +// Deprecated: Use WorkflowDeleteResponse.ProtoReflect.Descriptor instead. +func (*WorkflowDeleteResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{184} } -func (x *WorkflowUpdateResponse) GetSummary() string { +func (x *WorkflowDeleteResponse) GetSummary() string { if x != nil { return x.Summary } return "" } -func (x *WorkflowUpdateResponse) GetDetails() []*WorkflowUpdateResponse_TabletInfo { +func (x *WorkflowDeleteResponse) GetDetails() []*WorkflowDeleteResponse_TabletInfo { if x != nil { return x.Details } return nil } -type Workflow_ReplicationLocation struct { +type WorkflowStatusRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - Shards []string `protobuf:"bytes,2,rep,name=shards,proto3" json:"shards,omitempty"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` } -func (x *Workflow_ReplicationLocation) Reset() { - *x = Workflow_ReplicationLocation{} +func (x *WorkflowStatusRequest) Reset() { + *x = WorkflowStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[182] + mi := &file_vtctldata_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *Workflow_ReplicationLocation) String() string { +func (x *WorkflowStatusRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Workflow_ReplicationLocation) ProtoMessage() {} +func (*WorkflowStatusRequest) ProtoMessage() {} -func (x *Workflow_ReplicationLocation) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[182] +func (x *WorkflowStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[185] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10427,52 +10839,52 @@ func (x *Workflow_ReplicationLocation) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Workflow_ReplicationLocation.ProtoReflect.Descriptor instead. -func (*Workflow_ReplicationLocation) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{6, 1} +// Deprecated: Use WorkflowStatusRequest.ProtoReflect.Descriptor instead. +func (*WorkflowStatusRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{185} } -func (x *Workflow_ReplicationLocation) GetKeyspace() string { +func (x *WorkflowStatusRequest) GetKeyspace() string { if x != nil { return x.Keyspace } return "" } -func (x *Workflow_ReplicationLocation) GetShards() []string { +func (x *WorkflowStatusRequest) GetWorkflow() string { if x != nil { - return x.Shards + return x.Workflow } - return nil + return "" } -type Workflow_ShardStream struct { +type WorkflowStatusResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Streams []*Workflow_Stream `protobuf:"bytes,1,rep,name=streams,proto3" json:"streams,omitempty"` - TabletControls []*topodata.Shard_TabletControl `protobuf:"bytes,2,rep,name=tablet_controls,json=tabletControls,proto3" json:"tablet_controls,omitempty"` - IsPrimaryServing bool `protobuf:"varint,3,opt,name=is_primary_serving,json=isPrimaryServing,proto3" json:"is_primary_serving,omitempty"` + // The key is keyspace/shard. + TableCopyState map[string]*WorkflowStatusResponse_TableCopyState `protobuf:"bytes,1,rep,name=table_copy_state,json=tableCopyState,proto3" json:"table_copy_state,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ShardStreams map[string]*WorkflowStatusResponse_ShardStreams `protobuf:"bytes,2,rep,name=shard_streams,json=shardStreams,proto3" json:"shard_streams,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (x *Workflow_ShardStream) Reset() { - *x = Workflow_ShardStream{} +func (x *WorkflowStatusResponse) Reset() { + *x = WorkflowStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[183] + mi := &file_vtctldata_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *Workflow_ShardStream) String() string { +func (x *WorkflowStatusResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Workflow_ShardStream) ProtoMessage() {} +func (*WorkflowStatusResponse) ProtoMessage() {} -func (x *Workflow_ShardStream) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[183] +func (x *WorkflowStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[186] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10483,79 +10895,58 @@ func (x *Workflow_ShardStream) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Workflow_ShardStream.ProtoReflect.Descriptor instead. -func (*Workflow_ShardStream) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{6, 2} +// Deprecated: Use WorkflowStatusResponse.ProtoReflect.Descriptor instead. +func (*WorkflowStatusResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{186} } -func (x *Workflow_ShardStream) GetStreams() []*Workflow_Stream { +func (x *WorkflowStatusResponse) GetTableCopyState() map[string]*WorkflowStatusResponse_TableCopyState { if x != nil { - return x.Streams + return x.TableCopyState } return nil } -func (x *Workflow_ShardStream) GetTabletControls() []*topodata.Shard_TabletControl { +func (x *WorkflowStatusResponse) GetShardStreams() map[string]*WorkflowStatusResponse_ShardStreams { if x != nil { - return x.TabletControls + return x.ShardStreams } return nil } -func (x *Workflow_ShardStream) GetIsPrimaryServing() bool { - if x != nil { - return x.IsPrimaryServing - } - return false -} - -type Workflow_Stream struct { +type WorkflowSwitchTrafficRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` - Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` - Tablet *topodata.TabletAlias `protobuf:"bytes,3,opt,name=tablet,proto3" json:"tablet,omitempty"` - BinlogSource *binlogdata.BinlogSource `protobuf:"bytes,4,opt,name=binlog_source,json=binlogSource,proto3" json:"binlog_source,omitempty"` - Position string `protobuf:"bytes,5,opt,name=position,proto3" json:"position,omitempty"` - StopPosition string `protobuf:"bytes,6,opt,name=stop_position,json=stopPosition,proto3" json:"stop_position,omitempty"` - State string `protobuf:"bytes,7,opt,name=state,proto3" json:"state,omitempty"` - DbName string `protobuf:"bytes,8,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - TransactionTimestamp *vttime.Time `protobuf:"bytes,9,opt,name=transaction_timestamp,json=transactionTimestamp,proto3" json:"transaction_timestamp,omitempty"` - TimeUpdated *vttime.Time `protobuf:"bytes,10,opt,name=time_updated,json=timeUpdated,proto3" json:"time_updated,omitempty"` - Message string `protobuf:"bytes,11,opt,name=message,proto3" json:"message,omitempty"` - CopyStates []*Workflow_Stream_CopyState `protobuf:"bytes,12,rep,name=copy_states,json=copyStates,proto3" json:"copy_states,omitempty"` - Logs []*Workflow_Stream_Log `protobuf:"bytes,13,rep,name=logs,proto3" json:"logs,omitempty"` - // LogFetchError is set if we fail to fetch some logs for this stream. We - // will never fail to fetch workflows because we cannot fetch the logs, but - // we will still forward log-fetch errors to the caller, should that be - // relevant to the context in which they are fetching workflows. - // - // Note that this field being set does not necessarily mean that Logs is nil; - // if there are N logs that exist for the stream, and we fail to fetch the - // ith log, we will still return logs in [0, i) + (i, N]. - LogFetchError string `protobuf:"bytes,14,opt,name=log_fetch_error,json=logFetchError,proto3" json:"log_fetch_error,omitempty"` - Tags []string `protobuf:"bytes,15,rep,name=tags,proto3" json:"tags,omitempty"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Workflow string `protobuf:"bytes,2,opt,name=workflow,proto3" json:"workflow,omitempty"` + Cells []string `protobuf:"bytes,3,rep,name=cells,proto3" json:"cells,omitempty"` + TabletTypes []topodata.TabletType `protobuf:"varint,4,rep,packed,name=tablet_types,json=tabletTypes,proto3,enum=topodata.TabletType" json:"tablet_types,omitempty"` + MaxReplicationLagAllowed *vttime.Duration `protobuf:"bytes,5,opt,name=max_replication_lag_allowed,json=maxReplicationLagAllowed,proto3" json:"max_replication_lag_allowed,omitempty"` + EnableReverseReplication bool `protobuf:"varint,6,opt,name=enable_reverse_replication,json=enableReverseReplication,proto3" json:"enable_reverse_replication,omitempty"` + Direction int32 `protobuf:"varint,7,opt,name=direction,proto3" json:"direction,omitempty"` + Timeout *vttime.Duration `protobuf:"bytes,8,opt,name=timeout,proto3" json:"timeout,omitempty"` + DryRun bool `protobuf:"varint,9,opt,name=dry_run,json=dryRun,proto3" json:"dry_run,omitempty"` } -func (x *Workflow_Stream) Reset() { - *x = Workflow_Stream{} +func (x *WorkflowSwitchTrafficRequest) Reset() { + *x = WorkflowSwitchTrafficRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[184] + mi := &file_vtctldata_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *Workflow_Stream) String() string { +func (x *WorkflowSwitchTrafficRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Workflow_Stream) ProtoMessage() {} +func (*WorkflowSwitchTrafficRequest) ProtoMessage() {} -func (x *Workflow_Stream) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[184] +func (x *WorkflowSwitchTrafficRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[187] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10566,142 +10957,889 @@ func (x *Workflow_Stream) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Workflow_Stream.ProtoReflect.Descriptor instead. -func (*Workflow_Stream) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{6, 3} +// Deprecated: Use WorkflowSwitchTrafficRequest.ProtoReflect.Descriptor instead. +func (*WorkflowSwitchTrafficRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{187} } -func (x *Workflow_Stream) GetId() int64 { +func (x *WorkflowSwitchTrafficRequest) GetKeyspace() string { if x != nil { - return x.Id + return x.Keyspace } - return 0 + return "" } -func (x *Workflow_Stream) GetShard() string { +func (x *WorkflowSwitchTrafficRequest) GetWorkflow() string { if x != nil { - return x.Shard + return x.Workflow } return "" } -func (x *Workflow_Stream) GetTablet() *topodata.TabletAlias { +func (x *WorkflowSwitchTrafficRequest) GetCells() []string { if x != nil { - return x.Tablet + return x.Cells } return nil } -func (x *Workflow_Stream) GetBinlogSource() *binlogdata.BinlogSource { +func (x *WorkflowSwitchTrafficRequest) GetTabletTypes() []topodata.TabletType { if x != nil { - return x.BinlogSource + return x.TabletTypes } return nil } -func (x *Workflow_Stream) GetPosition() string { +func (x *WorkflowSwitchTrafficRequest) GetMaxReplicationLagAllowed() *vttime.Duration { if x != nil { - return x.Position + return x.MaxReplicationLagAllowed } - return "" + return nil } -func (x *Workflow_Stream) GetStopPosition() string { +func (x *WorkflowSwitchTrafficRequest) GetEnableReverseReplication() bool { if x != nil { - return x.StopPosition + return x.EnableReverseReplication } - return "" + return false } -func (x *Workflow_Stream) GetState() string { +func (x *WorkflowSwitchTrafficRequest) GetDirection() int32 { if x != nil { - return x.State + return x.Direction } - return "" + return 0 } -func (x *Workflow_Stream) GetDbName() string { +func (x *WorkflowSwitchTrafficRequest) GetTimeout() *vttime.Duration { if x != nil { - return x.DbName + return x.Timeout } - return "" + return nil } -func (x *Workflow_Stream) GetTransactionTimestamp() *vttime.Time { +func (x *WorkflowSwitchTrafficRequest) GetDryRun() bool { if x != nil { - return x.TransactionTimestamp + return x.DryRun } - return nil + return false +} + +type WorkflowSwitchTrafficResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Summary string `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` + StartState string `protobuf:"bytes,2,opt,name=start_state,json=startState,proto3" json:"start_state,omitempty"` + CurrentState string `protobuf:"bytes,3,opt,name=current_state,json=currentState,proto3" json:"current_state,omitempty"` + DryRunResults []string `protobuf:"bytes,4,rep,name=dry_run_results,json=dryRunResults,proto3" json:"dry_run_results,omitempty"` +} + +func (x *WorkflowSwitchTrafficResponse) Reset() { + *x = WorkflowSwitchTrafficResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[188] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowSwitchTrafficResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowSwitchTrafficResponse) ProtoMessage() {} + +func (x *WorkflowSwitchTrafficResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[188] + 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 WorkflowSwitchTrafficResponse.ProtoReflect.Descriptor instead. +func (*WorkflowSwitchTrafficResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{188} +} + +func (x *WorkflowSwitchTrafficResponse) GetSummary() string { + if x != nil { + return x.Summary + } + return "" +} + +func (x *WorkflowSwitchTrafficResponse) GetStartState() string { + if x != nil { + return x.StartState + } + return "" +} + +func (x *WorkflowSwitchTrafficResponse) GetCurrentState() string { + if x != nil { + return x.CurrentState + } + return "" +} + +func (x *WorkflowSwitchTrafficResponse) GetDryRunResults() []string { + if x != nil { + return x.DryRunResults + } + return nil +} + +type WorkflowUpdateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + // TabletRequest gets passed on to each primary tablet involved + // in the workflow via the UpdateVReplicationWorkflow tabletmanager RPC. + TabletRequest *tabletmanagerdata.UpdateVReplicationWorkflowRequest `protobuf:"bytes,2,opt,name=tablet_request,json=tabletRequest,proto3" json:"tablet_request,omitempty"` +} + +func (x *WorkflowUpdateRequest) Reset() { + *x = WorkflowUpdateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[189] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowUpdateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowUpdateRequest) ProtoMessage() {} + +func (x *WorkflowUpdateRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[189] + 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 WorkflowUpdateRequest.ProtoReflect.Descriptor instead. +func (*WorkflowUpdateRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{189} +} + +func (x *WorkflowUpdateRequest) GetKeyspace() string { + if x != nil { + return x.Keyspace + } + return "" +} + +func (x *WorkflowUpdateRequest) GetTabletRequest() *tabletmanagerdata.UpdateVReplicationWorkflowRequest { + if x != nil { + return x.TabletRequest + } + return nil +} + +type WorkflowUpdateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Summary string `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` + Details []*WorkflowUpdateResponse_TabletInfo `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty"` +} + +func (x *WorkflowUpdateResponse) Reset() { + *x = WorkflowUpdateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[190] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowUpdateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowUpdateResponse) ProtoMessage() {} + +func (x *WorkflowUpdateResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[190] + 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 WorkflowUpdateResponse.ProtoReflect.Descriptor instead. +func (*WorkflowUpdateResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{190} +} + +func (x *WorkflowUpdateResponse) GetSummary() string { + if x != nil { + return x.Summary + } + return "" +} + +func (x *WorkflowUpdateResponse) GetDetails() []*WorkflowUpdateResponse_TabletInfo { + if x != nil { + return x.Details + } + return nil +} + +type Workflow_ReplicationLocation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Shards []string `protobuf:"bytes,2,rep,name=shards,proto3" json:"shards,omitempty"` +} + +func (x *Workflow_ReplicationLocation) Reset() { + *x = Workflow_ReplicationLocation{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[192] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Workflow_ReplicationLocation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Workflow_ReplicationLocation) ProtoMessage() {} + +func (x *Workflow_ReplicationLocation) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[192] + 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 Workflow_ReplicationLocation.ProtoReflect.Descriptor instead. +func (*Workflow_ReplicationLocation) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{6, 1} +} + +func (x *Workflow_ReplicationLocation) GetKeyspace() string { + if x != nil { + return x.Keyspace + } + return "" +} + +func (x *Workflow_ReplicationLocation) GetShards() []string { + if x != nil { + return x.Shards + } + return nil +} + +type Workflow_ShardStream struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Streams []*Workflow_Stream `protobuf:"bytes,1,rep,name=streams,proto3" json:"streams,omitempty"` + TabletControls []*topodata.Shard_TabletControl `protobuf:"bytes,2,rep,name=tablet_controls,json=tabletControls,proto3" json:"tablet_controls,omitempty"` + IsPrimaryServing bool `protobuf:"varint,3,opt,name=is_primary_serving,json=isPrimaryServing,proto3" json:"is_primary_serving,omitempty"` +} + +func (x *Workflow_ShardStream) Reset() { + *x = Workflow_ShardStream{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[193] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Workflow_ShardStream) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Workflow_ShardStream) ProtoMessage() {} + +func (x *Workflow_ShardStream) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[193] + 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 Workflow_ShardStream.ProtoReflect.Descriptor instead. +func (*Workflow_ShardStream) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{6, 2} +} + +func (x *Workflow_ShardStream) GetStreams() []*Workflow_Stream { + if x != nil { + return x.Streams + } + return nil +} + +func (x *Workflow_ShardStream) GetTabletControls() []*topodata.Shard_TabletControl { + if x != nil { + return x.TabletControls + } + return nil +} + +func (x *Workflow_ShardStream) GetIsPrimaryServing() bool { + if x != nil { + return x.IsPrimaryServing + } + return false +} + +type Workflow_Stream struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` + Tablet *topodata.TabletAlias `protobuf:"bytes,3,opt,name=tablet,proto3" json:"tablet,omitempty"` + BinlogSource *binlogdata.BinlogSource `protobuf:"bytes,4,opt,name=binlog_source,json=binlogSource,proto3" json:"binlog_source,omitempty"` + Position string `protobuf:"bytes,5,opt,name=position,proto3" json:"position,omitempty"` + StopPosition string `protobuf:"bytes,6,opt,name=stop_position,json=stopPosition,proto3" json:"stop_position,omitempty"` + State string `protobuf:"bytes,7,opt,name=state,proto3" json:"state,omitempty"` + DbName string `protobuf:"bytes,8,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + TransactionTimestamp *vttime.Time `protobuf:"bytes,9,opt,name=transaction_timestamp,json=transactionTimestamp,proto3" json:"transaction_timestamp,omitempty"` + TimeUpdated *vttime.Time `protobuf:"bytes,10,opt,name=time_updated,json=timeUpdated,proto3" json:"time_updated,omitempty"` + Message string `protobuf:"bytes,11,opt,name=message,proto3" json:"message,omitempty"` + CopyStates []*Workflow_Stream_CopyState `protobuf:"bytes,12,rep,name=copy_states,json=copyStates,proto3" json:"copy_states,omitempty"` + Logs []*Workflow_Stream_Log `protobuf:"bytes,13,rep,name=logs,proto3" json:"logs,omitempty"` + // LogFetchError is set if we fail to fetch some logs for this stream. We + // will never fail to fetch workflows because we cannot fetch the logs, but + // we will still forward log-fetch errors to the caller, should that be + // relevant to the context in which they are fetching workflows. + // + // Note that this field being set does not necessarily mean that Logs is nil; + // if there are N logs that exist for the stream, and we fail to fetch the + // ith log, we will still return logs in [0, i) + (i, N]. + LogFetchError string `protobuf:"bytes,14,opt,name=log_fetch_error,json=logFetchError,proto3" json:"log_fetch_error,omitempty"` + Tags []string `protobuf:"bytes,15,rep,name=tags,proto3" json:"tags,omitempty"` +} + +func (x *Workflow_Stream) Reset() { + *x = Workflow_Stream{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[194] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Workflow_Stream) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Workflow_Stream) ProtoMessage() {} + +func (x *Workflow_Stream) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[194] + 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 Workflow_Stream.ProtoReflect.Descriptor instead. +func (*Workflow_Stream) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{6, 3} +} + +func (x *Workflow_Stream) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *Workflow_Stream) GetShard() string { + if x != nil { + return x.Shard + } + return "" +} + +func (x *Workflow_Stream) GetTablet() *topodata.TabletAlias { + if x != nil { + return x.Tablet + } + return nil +} + +func (x *Workflow_Stream) GetBinlogSource() *binlogdata.BinlogSource { + if x != nil { + return x.BinlogSource + } + return nil +} + +func (x *Workflow_Stream) GetPosition() string { + if x != nil { + return x.Position + } + return "" +} + +func (x *Workflow_Stream) GetStopPosition() string { + if x != nil { + return x.StopPosition + } + return "" +} + +func (x *Workflow_Stream) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *Workflow_Stream) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *Workflow_Stream) GetTransactionTimestamp() *vttime.Time { + if x != nil { + return x.TransactionTimestamp + } + return nil +} + +func (x *Workflow_Stream) GetTimeUpdated() *vttime.Time { + if x != nil { + return x.TimeUpdated + } + return nil +} + +func (x *Workflow_Stream) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *Workflow_Stream) GetCopyStates() []*Workflow_Stream_CopyState { + if x != nil { + return x.CopyStates + } + return nil +} + +func (x *Workflow_Stream) GetLogs() []*Workflow_Stream_Log { + if x != nil { + return x.Logs + } + return nil +} + +func (x *Workflow_Stream) GetLogFetchError() string { + if x != nil { + return x.LogFetchError + } + return "" +} + +func (x *Workflow_Stream) GetTags() []string { + if x != nil { + return x.Tags + } + return nil +} + +type Workflow_Stream_CopyState struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Table string `protobuf:"bytes,1,opt,name=table,proto3" json:"table,omitempty"` + LastPk string `protobuf:"bytes,2,opt,name=last_pk,json=lastPk,proto3" json:"last_pk,omitempty"` +} + +func (x *Workflow_Stream_CopyState) Reset() { + *x = Workflow_Stream_CopyState{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[195] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Workflow_Stream_CopyState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Workflow_Stream_CopyState) ProtoMessage() {} + +func (x *Workflow_Stream_CopyState) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[195] + 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 Workflow_Stream_CopyState.ProtoReflect.Descriptor instead. +func (*Workflow_Stream_CopyState) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{6, 3, 0} +} + +func (x *Workflow_Stream_CopyState) GetTable() string { + if x != nil { + return x.Table + } + return "" +} + +func (x *Workflow_Stream_CopyState) GetLastPk() string { + if x != nil { + return x.LastPk + } + return "" +} + +type Workflow_Stream_Log struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + StreamId int64 `protobuf:"varint,2,opt,name=stream_id,json=streamId,proto3" json:"stream_id,omitempty"` + Type string `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"` + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + CreatedAt *vttime.Time `protobuf:"bytes,5,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` + UpdatedAt *vttime.Time `protobuf:"bytes,6,opt,name=updated_at,json=updatedAt,proto3" json:"updated_at,omitempty"` + Message string `protobuf:"bytes,7,opt,name=message,proto3" json:"message,omitempty"` + Count int64 `protobuf:"varint,8,opt,name=count,proto3" json:"count,omitempty"` +} + +func (x *Workflow_Stream_Log) Reset() { + *x = Workflow_Stream_Log{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[196] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Workflow_Stream_Log) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Workflow_Stream_Log) ProtoMessage() {} + +func (x *Workflow_Stream_Log) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[196] + 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 Workflow_Stream_Log.ProtoReflect.Descriptor instead. +func (*Workflow_Stream_Log) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{6, 3, 1} +} + +func (x *Workflow_Stream_Log) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *Workflow_Stream_Log) GetStreamId() int64 { + if x != nil { + return x.StreamId + } + return 0 +} + +func (x *Workflow_Stream_Log) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *Workflow_Stream_Log) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *Workflow_Stream_Log) GetCreatedAt() *vttime.Time { + if x != nil { + return x.CreatedAt + } + return nil +} + +func (x *Workflow_Stream_Log) GetUpdatedAt() *vttime.Time { + if x != nil { + return x.UpdatedAt + } + return nil +} + +func (x *Workflow_Stream_Log) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *Workflow_Stream_Log) GetCount() int64 { + if x != nil { + return x.Count + } + return 0 +} + +type GetSrvKeyspaceNamesResponse_NameList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Names []string `protobuf:"bytes,1,rep,name=names,proto3" json:"names,omitempty"` +} + +func (x *GetSrvKeyspaceNamesResponse_NameList) Reset() { + *x = GetSrvKeyspaceNamesResponse_NameList{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[200] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSrvKeyspaceNamesResponse_NameList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSrvKeyspaceNamesResponse_NameList) ProtoMessage() {} + +func (x *GetSrvKeyspaceNamesResponse_NameList) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[200] + 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 GetSrvKeyspaceNamesResponse_NameList.ProtoReflect.Descriptor instead. +func (*GetSrvKeyspaceNamesResponse_NameList) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{75, 1} +} + +func (x *GetSrvKeyspaceNamesResponse_NameList) GetNames() []string { + if x != nil { + return x.Names + } + return nil +} + +type MoveTablesCreateResponse_TabletInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Tablet *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet,proto3" json:"tablet,omitempty"` + // Created is set if the workflow was created on this tablet or not. + Created bool `protobuf:"varint,2,opt,name=created,proto3" json:"created,omitempty"` +} + +func (x *MoveTablesCreateResponse_TabletInfo) Reset() { + *x = MoveTablesCreateResponse_TabletInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[203] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MoveTablesCreateResponse_TabletInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MoveTablesCreateResponse_TabletInfo) ProtoMessage() {} + +func (x *MoveTablesCreateResponse_TabletInfo) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[203] + 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 MoveTablesCreateResponse_TabletInfo.ProtoReflect.Descriptor instead. +func (*MoveTablesCreateResponse_TabletInfo) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{100, 0} +} + +func (x *MoveTablesCreateResponse_TabletInfo) GetTablet() *topodata.TabletAlias { + if x != nil { + return x.Tablet + } + return nil +} + +func (x *MoveTablesCreateResponse_TabletInfo) GetCreated() bool { + if x != nil { + return x.Created + } + return false +} + +type WorkflowDeleteResponse_TabletInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Tablet *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet,proto3" json:"tablet,omitempty"` + // Delete is set if the workflow was deleted on this tablet. + Deleted bool `protobuf:"varint,2,opt,name=deleted,proto3" json:"deleted,omitempty"` } -func (x *Workflow_Stream) GetTimeUpdated() *vttime.Time { - if x != nil { - return x.TimeUpdated +func (x *WorkflowDeleteResponse_TabletInfo) Reset() { + *x = WorkflowDeleteResponse_TabletInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[211] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return nil } -func (x *Workflow_Stream) GetMessage() string { - if x != nil { - return x.Message - } - return "" +func (x *WorkflowDeleteResponse_TabletInfo) String() string { + return protoimpl.X.MessageStringOf(x) } -func (x *Workflow_Stream) GetCopyStates() []*Workflow_Stream_CopyState { - if x != nil { - return x.CopyStates +func (*WorkflowDeleteResponse_TabletInfo) ProtoMessage() {} + +func (x *WorkflowDeleteResponse_TabletInfo) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[211] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return nil + return mi.MessageOf(x) } -func (x *Workflow_Stream) GetLogs() []*Workflow_Stream_Log { - if x != nil { - return x.Logs - } - return nil +// Deprecated: Use WorkflowDeleteResponse_TabletInfo.ProtoReflect.Descriptor instead. +func (*WorkflowDeleteResponse_TabletInfo) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{184, 0} } -func (x *Workflow_Stream) GetLogFetchError() string { +func (x *WorkflowDeleteResponse_TabletInfo) GetTablet() *topodata.TabletAlias { if x != nil { - return x.LogFetchError + return x.Tablet } - return "" + return nil } -func (x *Workflow_Stream) GetTags() []string { +func (x *WorkflowDeleteResponse_TabletInfo) GetDeleted() bool { if x != nil { - return x.Tags + return x.Deleted } - return nil + return false } -type Workflow_Stream_CopyState struct { +type WorkflowStatusResponse_TableCopyState struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Table string `protobuf:"bytes,1,opt,name=table,proto3" json:"table,omitempty"` - LastPk string `protobuf:"bytes,2,opt,name=last_pk,json=lastPk,proto3" json:"last_pk,omitempty"` + RowsCopied int64 `protobuf:"varint,1,opt,name=rows_copied,json=rowsCopied,proto3" json:"rows_copied,omitempty"` + RowsTotal int64 `protobuf:"varint,2,opt,name=rows_total,json=rowsTotal,proto3" json:"rows_total,omitempty"` + RowsPercentage float32 `protobuf:"fixed32,3,opt,name=rows_percentage,json=rowsPercentage,proto3" json:"rows_percentage,omitempty"` + BytesCopied int64 `protobuf:"varint,4,opt,name=bytes_copied,json=bytesCopied,proto3" json:"bytes_copied,omitempty"` + BytesTotal int64 `protobuf:"varint,5,opt,name=bytes_total,json=bytesTotal,proto3" json:"bytes_total,omitempty"` + BytesPercentage float32 `protobuf:"fixed32,6,opt,name=bytes_percentage,json=bytesPercentage,proto3" json:"bytes_percentage,omitempty"` } -func (x *Workflow_Stream_CopyState) Reset() { - *x = Workflow_Stream_CopyState{} +func (x *WorkflowStatusResponse_TableCopyState) Reset() { + *x = WorkflowStatusResponse_TableCopyState{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[185] + mi := &file_vtctldata_proto_msgTypes[212] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *Workflow_Stream_CopyState) String() string { +func (x *WorkflowStatusResponse_TableCopyState) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Workflow_Stream_CopyState) ProtoMessage() {} +func (*WorkflowStatusResponse_TableCopyState) ProtoMessage() {} -func (x *Workflow_Stream_CopyState) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[185] +func (x *WorkflowStatusResponse_TableCopyState) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[212] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10712,57 +11850,83 @@ func (x *Workflow_Stream_CopyState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Workflow_Stream_CopyState.ProtoReflect.Descriptor instead. -func (*Workflow_Stream_CopyState) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{6, 3, 0} +// Deprecated: Use WorkflowStatusResponse_TableCopyState.ProtoReflect.Descriptor instead. +func (*WorkflowStatusResponse_TableCopyState) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{186, 0} } -func (x *Workflow_Stream_CopyState) GetTable() string { +func (x *WorkflowStatusResponse_TableCopyState) GetRowsCopied() int64 { if x != nil { - return x.Table + return x.RowsCopied } - return "" + return 0 } -func (x *Workflow_Stream_CopyState) GetLastPk() string { +func (x *WorkflowStatusResponse_TableCopyState) GetRowsTotal() int64 { if x != nil { - return x.LastPk + return x.RowsTotal } - return "" + return 0 } -type Workflow_Stream_Log struct { +func (x *WorkflowStatusResponse_TableCopyState) GetRowsPercentage() float32 { + if x != nil { + return x.RowsPercentage + } + return 0 +} + +func (x *WorkflowStatusResponse_TableCopyState) GetBytesCopied() int64 { + if x != nil { + return x.BytesCopied + } + return 0 +} + +func (x *WorkflowStatusResponse_TableCopyState) GetBytesTotal() int64 { + if x != nil { + return x.BytesTotal + } + return 0 +} + +func (x *WorkflowStatusResponse_TableCopyState) GetBytesPercentage() float32 { + if x != nil { + return x.BytesPercentage + } + return 0 +} + +type WorkflowStatusResponse_ShardStreamState struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` - StreamId int64 `protobuf:"varint,2,opt,name=stream_id,json=streamId,proto3" json:"stream_id,omitempty"` - Type string `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"` - State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` - CreatedAt *vttime.Time `protobuf:"bytes,5,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` - UpdatedAt *vttime.Time `protobuf:"bytes,6,opt,name=updated_at,json=updatedAt,proto3" json:"updated_at,omitempty"` - Message string `protobuf:"bytes,7,opt,name=message,proto3" json:"message,omitempty"` - Count int64 `protobuf:"varint,8,opt,name=count,proto3" json:"count,omitempty"` + Id int32 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Tablet *topodata.TabletAlias `protobuf:"bytes,2,opt,name=tablet,proto3" json:"tablet,omitempty"` + SourceShard string `protobuf:"bytes,3,opt,name=source_shard,json=sourceShard,proto3" json:"source_shard,omitempty"` + Position string `protobuf:"bytes,4,opt,name=position,proto3" json:"position,omitempty"` + Status string `protobuf:"bytes,5,opt,name=status,proto3" json:"status,omitempty"` + Info string `protobuf:"bytes,6,opt,name=info,proto3" json:"info,omitempty"` } -func (x *Workflow_Stream_Log) Reset() { - *x = Workflow_Stream_Log{} +func (x *WorkflowStatusResponse_ShardStreamState) Reset() { + *x = WorkflowStatusResponse_ShardStreamState{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[186] + mi := &file_vtctldata_proto_msgTypes[213] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *Workflow_Stream_Log) String() string { +func (x *WorkflowStatusResponse_ShardStreamState) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Workflow_Stream_Log) ProtoMessage() {} +func (*WorkflowStatusResponse_ShardStreamState) ProtoMessage() {} -func (x *Workflow_Stream_Log) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[186] +func (x *WorkflowStatusResponse_ShardStreamState) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[213] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10773,92 +11937,78 @@ func (x *Workflow_Stream_Log) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Workflow_Stream_Log.ProtoReflect.Descriptor instead. -func (*Workflow_Stream_Log) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{6, 3, 1} +// Deprecated: Use WorkflowStatusResponse_ShardStreamState.ProtoReflect.Descriptor instead. +func (*WorkflowStatusResponse_ShardStreamState) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{186, 1} } -func (x *Workflow_Stream_Log) GetId() int64 { +func (x *WorkflowStatusResponse_ShardStreamState) GetId() int32 { if x != nil { return x.Id } return 0 } -func (x *Workflow_Stream_Log) GetStreamId() int64 { +func (x *WorkflowStatusResponse_ShardStreamState) GetTablet() *topodata.TabletAlias { if x != nil { - return x.StreamId + return x.Tablet } - return 0 + return nil } -func (x *Workflow_Stream_Log) GetType() string { +func (x *WorkflowStatusResponse_ShardStreamState) GetSourceShard() string { if x != nil { - return x.Type + return x.SourceShard } return "" } -func (x *Workflow_Stream_Log) GetState() string { +func (x *WorkflowStatusResponse_ShardStreamState) GetPosition() string { if x != nil { - return x.State + return x.Position } return "" } -func (x *Workflow_Stream_Log) GetCreatedAt() *vttime.Time { - if x != nil { - return x.CreatedAt - } - return nil -} - -func (x *Workflow_Stream_Log) GetUpdatedAt() *vttime.Time { - if x != nil { - return x.UpdatedAt - } - return nil -} - -func (x *Workflow_Stream_Log) GetMessage() string { +func (x *WorkflowStatusResponse_ShardStreamState) GetStatus() string { if x != nil { - return x.Message + return x.Status } return "" } -func (x *Workflow_Stream_Log) GetCount() int64 { +func (x *WorkflowStatusResponse_ShardStreamState) GetInfo() string { if x != nil { - return x.Count + return x.Info } - return 0 + return "" } -type GetSrvKeyspaceNamesResponse_NameList struct { +type WorkflowStatusResponse_ShardStreams struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Names []string `protobuf:"bytes,1,rep,name=names,proto3" json:"names,omitempty"` + Streams []*WorkflowStatusResponse_ShardStreamState `protobuf:"bytes,2,rep,name=streams,proto3" json:"streams,omitempty"` } -func (x *GetSrvKeyspaceNamesResponse_NameList) Reset() { - *x = GetSrvKeyspaceNamesResponse_NameList{} +func (x *WorkflowStatusResponse_ShardStreams) Reset() { + *x = WorkflowStatusResponse_ShardStreams{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[190] + mi := &file_vtctldata_proto_msgTypes[214] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *GetSrvKeyspaceNamesResponse_NameList) String() string { +func (x *WorkflowStatusResponse_ShardStreams) String() string { return protoimpl.X.MessageStringOf(x) } -func (*GetSrvKeyspaceNamesResponse_NameList) ProtoMessage() {} +func (*WorkflowStatusResponse_ShardStreams) ProtoMessage() {} -func (x *GetSrvKeyspaceNamesResponse_NameList) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[190] +func (x *WorkflowStatusResponse_ShardStreams) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[214] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10869,14 +12019,14 @@ func (x *GetSrvKeyspaceNamesResponse_NameList) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use GetSrvKeyspaceNamesResponse_NameList.ProtoReflect.Descriptor instead. -func (*GetSrvKeyspaceNamesResponse_NameList) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{75, 1} +// Deprecated: Use WorkflowStatusResponse_ShardStreams.ProtoReflect.Descriptor instead. +func (*WorkflowStatusResponse_ShardStreams) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{186, 2} } -func (x *GetSrvKeyspaceNamesResponse_NameList) GetNames() []string { +func (x *WorkflowStatusResponse_ShardStreams) GetStreams() []*WorkflowStatusResponse_ShardStreamState { if x != nil { - return x.Names + return x.Streams } return nil } @@ -10886,17 +12036,16 @@ type WorkflowUpdateResponse_TabletInfo struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Tablet string `protobuf:"bytes,1,opt,name=tablet,proto3" json:"tablet,omitempty"` + Tablet *topodata.TabletAlias `protobuf:"bytes,1,opt,name=tablet,proto3" json:"tablet,omitempty"` // Changed is true if any of the provided values were different - // than what was already stored. The value is based on the query - // result's RowsAffected being 0 or not. + // than what was already stored on this tablet. Changed bool `protobuf:"varint,2,opt,name=changed,proto3" json:"changed,omitempty"` } func (x *WorkflowUpdateResponse_TabletInfo) Reset() { *x = WorkflowUpdateResponse_TabletInfo{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[200] + mi := &file_vtctldata_proto_msgTypes[217] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10909,7 +12058,7 @@ func (x *WorkflowUpdateResponse_TabletInfo) String() string { func (*WorkflowUpdateResponse_TabletInfo) ProtoMessage() {} func (x *WorkflowUpdateResponse_TabletInfo) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[200] + mi := &file_vtctldata_proto_msgTypes[217] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10922,14 +12071,14 @@ 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{180, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{190, 0} } -func (x *WorkflowUpdateResponse_TabletInfo) GetTablet() string { +func (x *WorkflowUpdateResponse_TabletInfo) GetTablet() *topodata.TabletAlias { if x != nil { return x.Tablet } - return "" + return nil } func (x *WorkflowUpdateResponse_TabletInfo) GetChanged() bool { @@ -10972,8 +12121,8 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x64, 0x6c, 0x22, 0xf4, - 0x04, 0x0a, 0x13, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x64, 0x6c, 0x22, 0xe2, + 0x05, 0x0a, 0x13, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, @@ -11012,1233 +12161,1388 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x44, 0x64, 0x6c, 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, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, - 0x79, 0x4b, 0x65, 0x79, 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, 0x5e, 0x0a, 0x05, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x25, - 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xd2, 0x0c, 0x0a, 0x08, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x3f, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x31, 0x0a, 0x15, 0x6d, 0x61, 0x78, 0x5f, - 0x76, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, - 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x12, 0x4a, 0x0a, 0x0d, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x11, - 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x1a, 0x60, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x49, 0x0a, 0x13, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, - 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0xb9, 0x01, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x34, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x46, 0x0a, 0x0f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, - 0x6f, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x10, 0x69, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, - 0x67, 0x1a, 0xf6, 0x06, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0e, 0x0a, 0x02, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x12, 0x3d, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, - 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x52, 0x0c, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, - 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x41, 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x14, 0x74, - 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, - 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x45, - 0x0a, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, - 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x63, 0x6f, 0x70, 0x79, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0d, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, - 0x4c, 0x6f, 0x67, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x67, - 0x5f, 0x66, 0x65, 0x74, 0x63, 0x68, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f, 0x67, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x04, 0x74, 0x61, 0x67, 0x73, 0x1a, 0x3a, 0x0a, 0x09, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x6c, 0x61, 0x73, 0x74, - 0x5f, 0x70, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x50, - 0x6b, 0x1a, 0xe6, 0x01, 0x0a, 0x03, 0x4c, 0x6f, 0x67, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x12, 0x2b, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, - 0x6d, 0x65, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x2b, 0x0a, - 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, - 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x59, 0x0a, 0x12, 0x41, 0x64, - 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, - 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x15, 0x0a, 0x13, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x40, 0x0a, 0x14, - 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x17, - 0x0a, 0x15, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9e, 0x01, 0x0a, 0x18, 0x41, 0x70, 0x70, 0x6c, - 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x0d, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x76, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, - 0x65, 0x73, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x1b, 0x0a, 0x19, 0x41, 0x70, 0x70, 0x6c, - 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x1d, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4a, 0x0a, 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, 0x69, 0x70, 0x52, - 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, - 0x64, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, - 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x20, 0x0a, 0x1e, 0x41, - 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, - 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd0, 0x02, - 0x0a, 0x12, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x73, - 0x71, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x64, 0x6c, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, - 0x67, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, - 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, - 0x73, 0x74, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, - 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, - 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, - 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, - 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, - 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x72, - 0x65, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, - 0x6b, 0x69, 0x70, 0x50, 0x72, 0x65, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x12, 0x2c, 0x0a, 0x09, - 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, - 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, - 0x22, 0x32, 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, 0x22, 0xc3, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x79, 0x4b, 0x65, 0x79, 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, 0x0f, 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, 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, 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, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, - 0x5f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, - 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, - 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, - 0x79, 0x52, 0x75, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, - 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x22, 0x44, 0x0a, 0x14, 0x41, 0x70, - 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x22, 0xe5, 0x01, 0x0a, 0x0d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, - 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x23, 0x0a, 0x0d, - 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, - 0x6e, 0x63, 0x79, 0x12, 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, - 0x61, 0x6c, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x12, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, - 0x6f, 0x6d, 0x50, 0x6f, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, - 0x5f, 0x73, 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, - 0x72, 0x61, 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x0e, 0x42, 0x61, 0x63, - 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, - 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xb0, 0x01, - 0x0a, 0x12, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, - 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, - 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x21, 0x0a, - 0x0c, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, - 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, 0x99, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x61, - 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x6f, 0x77, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x0c, - 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, - 0x6d, 0x52, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x73, 0x12, 0x2a, - 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x62, 0x61, - 0x73, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x62, 0x61, 0x73, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, - 0x31, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x69, - 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, - 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, - 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, - 0x26, 0x0a, 0x0f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, - 0x72, 0x44, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, - 0x05, 0x10, 0x06, 0x22, 0x49, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, - 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x8c, - 0x01, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, - 0x65, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x22, 0xa0, 0x01, - 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x30, - 0x0a, 0x14, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, - 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x22, 0x41, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, - 0x72, 0x63, 0x65, 0x22, 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d, 0x0a, - 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 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, 0xd0, 0x02, 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, 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, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x55, - 0x0a, 0x13, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x68, 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, - 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x0b, - 0x68, 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, - 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0a, 0x68, 0x6f, 0x6f, 0x6b, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x3c, 0x0a, 0x1e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x1f, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9e, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, - 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, - 0x69, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x25, - 0x0a, 0x0e, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, - 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x44, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x62, - 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, - 0x79, 0x73, 0x71, 0x6c, 0x63, 0x74, 0x6c, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x07, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x22, 0x28, 0x0a, 0x12, 0x47, - 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, - 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x19, 0x0a, - 0x17, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x30, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x43, - 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, - 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x22, 0xb6, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, - 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x49, 0x0a, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x1a, 0x50, 0x0a, 0x0c, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2a, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, - 0x14, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x71, 0x75, 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, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x22, 0xd2, 0x0c, 0x0a, 0x08, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x3f, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x31, 0x0a, 0x15, 0x6d, 0x61, 0x78, 0x5f, 0x76, 0x5f, + 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x12, 0x4a, 0x0a, 0x0d, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, + 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x1a, 0x60, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x49, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x73, 0x1a, 0xb9, 0x01, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x12, 0x34, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x46, 0x0a, 0x0f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, + 0x6c, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, + 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, + 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x1a, + 0xf6, 0x06, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, + 0x3d, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x52, 0x0c, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1a, + 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, + 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x41, + 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, + 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x14, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0b, + 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x43, 0x6f, + 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x63, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x4c, 0x6f, + 0x67, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x67, 0x5f, 0x66, + 0x65, 0x74, 0x63, 0x68, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0d, 0x6c, 0x6f, 0x67, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, + 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, + 0x61, 0x67, 0x73, 0x1a, 0x3a, 0x0a, 0x09, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, + 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x6b, 0x1a, + 0xe6, 0x01, 0x0a, 0x03, 0x4c, 0x6f, 0x67, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2b, + 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x2b, 0x0a, 0x0a, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x59, 0x0a, 0x12, 0x41, 0x64, 0x64, 0x43, + 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x22, 0x15, 0x0a, 0x13, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x40, 0x0a, 0x14, 0x41, 0x64, + 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x17, 0x0a, 0x15, + 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9e, 0x01, 0x0a, 0x18, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 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, 0x4c, - 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, - 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x3a, 0x0a, 0x10, - 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x1d, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x6a, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4a, 0x0a, 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x22, 0x32, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xf3, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, - 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4e, - 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x1a, 0x69, 0x0a, 0x0a, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x45, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, - 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x20, 0x0a, 0x08, 0x4e, - 0x61, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0x4a, 0x0a, - 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xcc, 0x01, 0x0a, 0x17, 0x47, 0x65, - 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x72, 0x76, 0x5f, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x0c, 0x73, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, - 0x1a, 0x56, 0x0a, 0x11, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2b, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf8, 0x02, 0x0a, 0x1c, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x18, 0x0a, - 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, - 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, - 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, - 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x75, 0x73, - 0x74, 0x6f, 0x6d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x63, 0x75, 0x73, 0x74, - 0x6f, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, - 0x65, 0x74, 0x12, 0x2d, 0x0a, 0x13, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x61, 0x73, 0x5f, 0x63, - 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x6c, 0x66, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x10, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x65, 0x6c, - 0x66, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x61, 0x73, 0x5f, 0x63, 0x68, - 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x11, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x12, 0x3f, 0x0a, 0x0d, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x5f, - 0x61, 0x70, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, - 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0c, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, - 0x41, 0x70, 0x70, 0x22, 0x1f, 0x0a, 0x1d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2a, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, - 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, - 0x22, 0x4e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x0c, 0x73, 0x72, 0x76, - 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0a, 0x73, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x22, 0x2d, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, - 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, - 0xc5, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x56, 0x0a, 0x0d, 0x73, 0x72, - 0x76, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x73, 0x1a, 0x53, 0x0a, 0x10, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x29, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4c, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 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, 0x6f, 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, + 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, 0xd0, 0x02, 0x0a, 0x12, + 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, + 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, + 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x64, 0x6c, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, 0x61, 0x74, + 0x65, 0x67, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, 0x73, 0x74, + 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x44, 0x0a, + 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, + 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, + 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x72, 0x65, 0x66, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x6b, 0x69, + 0x70, 0x50, 0x72, 0x65, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, + 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, + 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, + 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0x32, + 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, 0x22, 0xc3, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 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, 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, 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, 0x89, 0x02, 0x0a, 0x1b, 0x50, 0x6c, 0x61, 0x6e, 0x6e, - 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x12, 0x3a, 0x0a, 0x0d, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0c, - 0x61, 0x76, 0x6f, 0x69, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x44, 0x0a, 0x15, - 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, - 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, - 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, - 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, - 0x75, 0x74, 0x22, 0xba, 0x01, 0x0a, 0x1c, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, - 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, - 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, + 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, + 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, + 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, + 0x75, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x22, 0x44, 0x0a, 0x14, 0x41, 0x70, 0x70, 0x6c, + 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xe5, + 0x01, 0x0a, 0x0d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, + 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, + 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, + 0x79, 0x12, 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, + 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x12, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, 0x6f, 0x6d, + 0x50, 0x6f, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, + 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, + 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x0e, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, - 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, - 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, - 0x74, 0x0a, 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, - 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, - 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, - 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x61, 0x6c, 0x22, 0x1e, 0x0a, 0x1c, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x1a, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x1d, 0x0a, 0x1b, 0x52, 0x65, 0x62, - 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4f, 0x0a, 0x13, 0x52, 0x65, 0x66, 0x72, - 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x52, 0x65, 0x66, - 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x64, 0x0a, 0x1a, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x66, 0x72, - 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, - 0x66, 0x72, 0x65, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, - 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x52, - 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x4f, 0x0a, - 0x13, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, - 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x16, - 0x0a, 0x14, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x6f, 0x61, - 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, - 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, - 0x63, 0x79, 0x22, 0x46, 0x0a, 0x1c, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xbc, 0x01, 0x0a, 0x18, 0x52, - 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, - 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, - 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, - 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, - 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x43, 0x0a, 0x19, 0x52, 0x65, 0x6c, - 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, - 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x5b, - 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x52, - 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x7f, 0x0a, 0x19, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 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, 0xb0, 0x01, 0x0a, 0x12, + 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, + 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x75, + 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, 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, 0x99, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x0c, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x64, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, + 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x73, 0x12, 0x2a, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x62, 0x61, 0x73, 0x65, + 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x62, 0x61, 0x73, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x31, 0x0a, + 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x69, 0x6d, 0x65, + 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x26, 0x0a, + 0x0f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, + 0x06, 0x22, 0x49, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x8c, 0x01, 0x0a, + 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, + 0x6f, 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x6e, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x22, 0xa0, 0x01, 0x0a, 0x13, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x30, 0x0a, 0x14, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0x41, + 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, + 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, + 0x65, 0x22, 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d, 0x0a, 0x17, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 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, 0xd0, + 0x02, 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, 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, + 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, 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, 0x07, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x22, 0x82, 0x02, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, - 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, + 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, 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, + 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, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x55, 0x0a, 0x13, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x68, 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, + 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x0b, 0x68, 0x6f, + 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0a, 0x68, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x22, 0x3c, 0x0a, 0x1e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x1f, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, + 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x9e, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, + 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x25, 0x0a, 0x0e, + 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x4c, 0x69, + 0x6d, 0x69, 0x74, 0x22, 0x44, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x62, 0x61, 0x63, + 0x6b, 0x75, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x79, 0x73, + 0x71, 0x6c, 0x63, 0x74, 0x6c, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x07, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x65, 0x74, + 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, + 0x65, 0x6c, 0x6c, 0x22, 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, + 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x19, 0x0a, 0x17, 0x47, + 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x30, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, + 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x22, 0xb6, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x49, + 0x0a, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x1a, 0x50, 0x0a, 0x0c, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2a, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, 0x14, 0x47, + 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, + 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, 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, 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, + 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, 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, - 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, + 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, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, - 0x74, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x10, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, - 0x79, 0x22, 0x55, 0x0a, 0x23, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x1c, 0x53, 0x65, 0x74, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, - 0x6f, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, + 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, + 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x40, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x07, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x07, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, + 0x2c, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, + 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x22, 0x46, 0x0a, + 0x17, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x52, + 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x66, 0x0a, 0x0c, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, + 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x61, 0x74, + 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x22, 0x2f, 0x0a, + 0x11, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4d, + 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x2e, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x42, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x22, 0x8b, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, - 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, - 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, - 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, - 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x22, 0x4f, 0x0a, 0x1d, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x22, 0x5e, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, + 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x65, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6f, + 0x6e, 0x6c, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x61, 0x6d, 0x65, 0x4f, + 0x6e, 0x6c, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x22, + 0x49, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, + 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x17, 0x49, + 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 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, + 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, 0xf0, 0x05, 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, 0x22, + 0xd5, 0x01, 0x0a, 0x18, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, + 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, + 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x48, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x1a, 0x55, 0x0a, 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, + 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x22, 0xe9, 0x01, 0x0a, 0x19, 0x4d, 0x6f, 0x76, 0x65, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x6b, 0x65, + 0x65, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6b, + 0x65, 0x65, 0x70, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x12, 0x6b, 0x65, 0x65, 0x70, 0x5f, + 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, 0x70, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, + 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x72, 0x65, + 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, + 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, + 0x52, 0x75, 0x6e, 0x22, 0x5e, 0x0a, 0x1a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x0f, 0x64, + 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x22, 0x4d, 0x0a, 0x11, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x22, 0x14, 0x0a, 0x12, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x89, 0x02, 0x0a, 0x1b, 0x50, 0x6c, 0x61, + 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, + 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x0d, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, + 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0c, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x44, + 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, + 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x22, 0xba, 0x01, 0x0a, 0x1c, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, + 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x6d, 0x6f, + 0x74, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, + 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, + 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x73, 0x22, 0x74, 0x0a, 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, + 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, + 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x22, 0x1e, 0x0a, 0x1c, 0x52, 0x65, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x1a, 0x52, 0x65, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x1d, 0x0a, 0x1b, 0x52, + 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, + 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4f, 0x0a, 0x13, 0x52, 0x65, + 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x52, + 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x64, 0x0a, 0x1a, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x1b, 0x52, 0x65, + 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, + 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, + 0x6c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, + 0x4f, 0x0a, 0x13, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x22, 0x16, 0x0a, 0x14, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x6c, + 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x61, 0x69, + 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, + 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x63, 0x79, 0x22, 0x46, 0x0a, 0x1c, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xbc, 0x01, 0x0a, + 0x18, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x77, + 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, + 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, + 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x43, 0x0a, 0x19, 0x52, + 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, + 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0x5b, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 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, + 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x16, 0x0a, + 0x14, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7f, 0x0a, 0x19, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, + 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, + 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x63, 0x75, + 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, 0x65, 0x63, + 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, + 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x12, 0x14, + 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, + 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, + 0x76, 0x65, 0x22, 0x19, 0x0a, 0x17, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x46, 0x0a, + 0x15, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0x7b, 0x0a, 0x16, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x12, 0x2f, 0x0a, 0x07, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x07, 0x70, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x22, 0x82, 0x02, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, + 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, 0x0b, 0x62, 0x61, 0x63, + 0x6b, 0x75, 0x70, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, + 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0a, 0x62, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x74, + 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x6f, 0x50, 0x6f, 0x73, 0x12, 0x17, + 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x12, 0x3e, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x74, 0x6f, + 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x52, 0x12, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x6f, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0xad, 0x01, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x74, + 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, - 0x1a, 0x0a, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x15, 0x0a, 0x13, 0x53, - 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, - 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 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, + 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, 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, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xaa, 0x03, 0x0a, - 0x21, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x78, 0x0a, 0x14, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x45, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x12, 0x5a, 0x0a, 0x0a, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x3b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x1a, 0x5f, 0x0a, 0x18, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2d, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4e, 0x0a, 0x0e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x26, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8b, 0x01, 0x0a, 0x1d, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x22, 0x55, 0x0a, 0x23, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x1c, 0x53, + 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, + 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 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, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, + 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x27, 0x0a, 0x0f, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4f, 0x0a, 0x1d, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x5e, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, + 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x22, 0x51, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x72, 0x0a, 0x1f, 0x53, 0x65, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1d, + 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x22, 0x49, 0x0a, + 0x20, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x8e, 0x02, 0x0a, 0x1c, 0x53, 0x65, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x35, 0x0a, 0x0b, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x6e, 0x69, + 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0c, 0x64, 0x65, 0x6e, 0x69, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x32, 0x0a, + 0x15, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x64, 0x69, + 0x73, 0x61, 0x62, 0x6c, 0x65, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x22, 0x46, 0x0a, 0x1d, 0x53, 0x65, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x22, 0x6a, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x15, 0x0a, + 0x13, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, + 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 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, + 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, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 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, 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, + 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, 0x22, 0x19, 0x0a, 0x17, 0x53, - 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x21, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, - 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x06, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, - 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0xc6, 0x01, 0x0a, 0x22, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x52, - 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, - 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x36, 0x0a, 0x0b, 0x6f, - 0x6c, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6f, 0x6c, 0x64, 0x50, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x22, 0x5c, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, - 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, - 0x6f, 0x22, 0x5d, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, - 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, - 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, - 0x22, 0x64, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, - 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, 0x65, 0x6c, 0x6c, - 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x65, 0x0a, 0x18, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x5f, - 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, - 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x0a, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x34, 0x0a, - 0x0f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x12, 0x62, 0x0a, 0x13, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, - 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x11, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x1a, 0x69, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x58, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, - 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfc, 0x01, 0x0a, 0x18, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2c, 0x0a, 0x08, 0x64, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, + 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x53, 0x6c, 0x65, + 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0xf0, 0x01, 0x0a, 0x15, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x10, 0x0a, 0x03, + 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x27, + 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2f, 0x0a, 0x09, 0x6b, 0x65, + 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x22, 0x3f, 0x0a, 0x16, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x22, 0x5e, 0x0a, 0x18, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x03, 0x75, 0x69, 0x64, 0x22, 0x42, 0x0a, 0x19, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x53, 0x0a, 0x17, 0x53, 0x74, 0x61, 0x72, + 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x1a, 0x0a, + 0x18, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x16, 0x53, 0x74, 0x6f, + 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x19, 0x0a, + 0x17, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x21, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x52, 0x65, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, + 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0xc6, 0x01, 0x0a, + 0x22, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, + 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x36, 0x0a, + 0x0b, 0x6f, 0x6c, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6f, 0x6c, 0x64, 0x50, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x5c, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, + 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x22, 0x5d, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x22, 0x64, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, + 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, 0x65, + 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x65, 0x0a, 0x18, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, + 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, + 0x34, 0x0a, 0x0f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x12, 0x62, 0x0a, 0x13, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, + 0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x1a, 0x69, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x58, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, + 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfc, 0x01, + 0x0a, 0x18, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x12, 0x61, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, + 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd8, 0x01, 0x0a, + 0x1d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, 0x65, + 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x6e, + 0x6f, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0d, 0x73, 0x6b, 0x69, 0x70, 0x4e, 0x6f, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x27, + 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x56, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x88, 0x02, 0x0a, 0x1e, 0x56, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x12, 0x67, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, + 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, + 0x13, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x6b, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x21, 0x0a, 0x0c, + 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, + 0x31, 0x0a, 0x15, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x12, 0x61, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, - 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd8, 0x01, 0x0a, 0x1d, 0x56, - 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x74, 0x73, 0x22, 0x3c, 0x0a, 0x1e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x22, 0x8a, 0x02, 0x0a, 0x1f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x68, + 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4f, 0x0a, + 0x1b, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, - 0x69, 0x65, 0x77, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x6e, 0x6f, 0x5f, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, - 0x6b, 0x69, 0x70, 0x4e, 0x6f, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x88, 0x02, 0x0a, 0x1e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x12, 0x67, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, - 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 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, @@ -12248,100 +13552,161 @@ var file_vtctldata_proto_rawDesc = []byte{ 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, + 0x22, 0x9a, 0x01, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x38, 0x0a, 0x1c, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, - 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x98, 0x01, 0x0a, 0x16, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, - 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x65, - 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, 0x69, 0x65, 0x77, - 0x73, 0x22, 0xfa, 0x01, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, - 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, - 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x86, - 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, 0x51, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 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, 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, 0xba, 0x01, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x12, 0x1b, 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6b, 0x65, 0x65, 0x70, 0x44, 0x61, 0x74, 0x61, 0x12, + 0x2c, 0x0a, 0x12, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, + 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, + 0x70, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xd1, 0x01, + 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, + 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x46, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, 0x0a, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x22, 0x4f, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x22, 0xc1, 0x07, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, + 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x58, + 0x0a, 0x0d, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x1a, 0xe8, 0x01, 0x0a, 0x0e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, + 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x1d, 0x0a, 0x0a, + 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x72, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x27, 0x0a, 0x0f, 0x72, + 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x02, 0x52, 0x0e, 0x72, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, + 0x74, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, + 0x70, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x62, 0x79, 0x74, 0x65, + 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x79, 0x74, 0x65, 0x73, + 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x10, 0x62, 0x79, 0x74, 0x65, + 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x02, 0x52, 0x0f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, + 0x61, 0x67, 0x65, 0x1a, 0xbc, 0x01, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, + 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x69, 0x6e, + 0x66, 0x6f, 0x1a, 0x5c, 0x0a, 0x0c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x73, 0x12, 0x4c, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, + 0x1a, 0x73, 0x0a, 0x13, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x46, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x97, 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, + 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, 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, 0x3e, 0x0a, 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 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, - 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, + 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, 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 ( @@ -12357,7 +13722,7 @@ func file_vtctldata_proto_rawDescGZIP() []byte { } var file_vtctldata_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_vtctldata_proto_msgTypes = make([]protoimpl.MessageInfo, 201) +var file_vtctldata_proto_msgTypes = make([]protoimpl.MessageInfo, 218) var file_vtctldata_proto_goTypes = []interface{}{ (MaterializationIntent)(0), // 0: vtctldata.MaterializationIntent (*ExecuteVtctlCommandRequest)(nil), // 1: vtctldata.ExecuteVtctlCommandRequest @@ -12459,296 +13824,331 @@ var file_vtctldata_proto_goTypes = []interface{}{ (*GetWorkflowsResponse)(nil), // 97: vtctldata.GetWorkflowsResponse (*InitShardPrimaryRequest)(nil), // 98: vtctldata.InitShardPrimaryRequest (*InitShardPrimaryResponse)(nil), // 99: vtctldata.InitShardPrimaryResponse - (*PingTabletRequest)(nil), // 100: vtctldata.PingTabletRequest - (*PingTabletResponse)(nil), // 101: vtctldata.PingTabletResponse - (*PlannedReparentShardRequest)(nil), // 102: vtctldata.PlannedReparentShardRequest - (*PlannedReparentShardResponse)(nil), // 103: vtctldata.PlannedReparentShardResponse - (*RebuildKeyspaceGraphRequest)(nil), // 104: vtctldata.RebuildKeyspaceGraphRequest - (*RebuildKeyspaceGraphResponse)(nil), // 105: vtctldata.RebuildKeyspaceGraphResponse - (*RebuildVSchemaGraphRequest)(nil), // 106: vtctldata.RebuildVSchemaGraphRequest - (*RebuildVSchemaGraphResponse)(nil), // 107: vtctldata.RebuildVSchemaGraphResponse - (*RefreshStateRequest)(nil), // 108: vtctldata.RefreshStateRequest - (*RefreshStateResponse)(nil), // 109: vtctldata.RefreshStateResponse - (*RefreshStateByShardRequest)(nil), // 110: vtctldata.RefreshStateByShardRequest - (*RefreshStateByShardResponse)(nil), // 111: vtctldata.RefreshStateByShardResponse - (*ReloadSchemaRequest)(nil), // 112: vtctldata.ReloadSchemaRequest - (*ReloadSchemaResponse)(nil), // 113: vtctldata.ReloadSchemaResponse - (*ReloadSchemaKeyspaceRequest)(nil), // 114: vtctldata.ReloadSchemaKeyspaceRequest - (*ReloadSchemaKeyspaceResponse)(nil), // 115: vtctldata.ReloadSchemaKeyspaceResponse - (*ReloadSchemaShardRequest)(nil), // 116: vtctldata.ReloadSchemaShardRequest - (*ReloadSchemaShardResponse)(nil), // 117: vtctldata.ReloadSchemaShardResponse - (*RemoveBackupRequest)(nil), // 118: vtctldata.RemoveBackupRequest - (*RemoveBackupResponse)(nil), // 119: vtctldata.RemoveBackupResponse - (*RemoveKeyspaceCellRequest)(nil), // 120: vtctldata.RemoveKeyspaceCellRequest - (*RemoveKeyspaceCellResponse)(nil), // 121: vtctldata.RemoveKeyspaceCellResponse - (*RemoveShardCellRequest)(nil), // 122: vtctldata.RemoveShardCellRequest - (*RemoveShardCellResponse)(nil), // 123: vtctldata.RemoveShardCellResponse - (*ReparentTabletRequest)(nil), // 124: vtctldata.ReparentTabletRequest - (*ReparentTabletResponse)(nil), // 125: vtctldata.ReparentTabletResponse - (*RestoreFromBackupRequest)(nil), // 126: vtctldata.RestoreFromBackupRequest - (*RestoreFromBackupResponse)(nil), // 127: vtctldata.RestoreFromBackupResponse - (*RunHealthCheckRequest)(nil), // 128: vtctldata.RunHealthCheckRequest - (*RunHealthCheckResponse)(nil), // 129: vtctldata.RunHealthCheckResponse - (*SetKeyspaceDurabilityPolicyRequest)(nil), // 130: vtctldata.SetKeyspaceDurabilityPolicyRequest - (*SetKeyspaceDurabilityPolicyResponse)(nil), // 131: vtctldata.SetKeyspaceDurabilityPolicyResponse - (*SetKeyspaceServedFromRequest)(nil), // 132: vtctldata.SetKeyspaceServedFromRequest - (*SetKeyspaceServedFromResponse)(nil), // 133: vtctldata.SetKeyspaceServedFromResponse - (*SetKeyspaceShardingInfoRequest)(nil), // 134: vtctldata.SetKeyspaceShardingInfoRequest - (*SetKeyspaceShardingInfoResponse)(nil), // 135: vtctldata.SetKeyspaceShardingInfoResponse - (*SetShardIsPrimaryServingRequest)(nil), // 136: vtctldata.SetShardIsPrimaryServingRequest - (*SetShardIsPrimaryServingResponse)(nil), // 137: vtctldata.SetShardIsPrimaryServingResponse - (*SetShardTabletControlRequest)(nil), // 138: vtctldata.SetShardTabletControlRequest - (*SetShardTabletControlResponse)(nil), // 139: vtctldata.SetShardTabletControlResponse - (*SetWritableRequest)(nil), // 140: vtctldata.SetWritableRequest - (*SetWritableResponse)(nil), // 141: vtctldata.SetWritableResponse - (*ShardReplicationAddRequest)(nil), // 142: vtctldata.ShardReplicationAddRequest - (*ShardReplicationAddResponse)(nil), // 143: vtctldata.ShardReplicationAddResponse - (*ShardReplicationFixRequest)(nil), // 144: vtctldata.ShardReplicationFixRequest - (*ShardReplicationFixResponse)(nil), // 145: vtctldata.ShardReplicationFixResponse - (*ShardReplicationPositionsRequest)(nil), // 146: vtctldata.ShardReplicationPositionsRequest - (*ShardReplicationPositionsResponse)(nil), // 147: vtctldata.ShardReplicationPositionsResponse - (*ShardReplicationRemoveRequest)(nil), // 148: vtctldata.ShardReplicationRemoveRequest - (*ShardReplicationRemoveResponse)(nil), // 149: vtctldata.ShardReplicationRemoveResponse - (*SleepTabletRequest)(nil), // 150: vtctldata.SleepTabletRequest - (*SleepTabletResponse)(nil), // 151: vtctldata.SleepTabletResponse - (*SourceShardAddRequest)(nil), // 152: vtctldata.SourceShardAddRequest - (*SourceShardAddResponse)(nil), // 153: vtctldata.SourceShardAddResponse - (*SourceShardDeleteRequest)(nil), // 154: vtctldata.SourceShardDeleteRequest - (*SourceShardDeleteResponse)(nil), // 155: vtctldata.SourceShardDeleteResponse - (*StartReplicationRequest)(nil), // 156: vtctldata.StartReplicationRequest - (*StartReplicationResponse)(nil), // 157: vtctldata.StartReplicationResponse - (*StopReplicationRequest)(nil), // 158: vtctldata.StopReplicationRequest - (*StopReplicationResponse)(nil), // 159: vtctldata.StopReplicationResponse - (*TabletExternallyReparentedRequest)(nil), // 160: vtctldata.TabletExternallyReparentedRequest - (*TabletExternallyReparentedResponse)(nil), // 161: vtctldata.TabletExternallyReparentedResponse - (*UpdateCellInfoRequest)(nil), // 162: vtctldata.UpdateCellInfoRequest - (*UpdateCellInfoResponse)(nil), // 163: vtctldata.UpdateCellInfoResponse - (*UpdateCellsAliasRequest)(nil), // 164: vtctldata.UpdateCellsAliasRequest - (*UpdateCellsAliasResponse)(nil), // 165: vtctldata.UpdateCellsAliasResponse - (*ValidateRequest)(nil), // 166: vtctldata.ValidateRequest - (*ValidateResponse)(nil), // 167: vtctldata.ValidateResponse - (*ValidateKeyspaceRequest)(nil), // 168: vtctldata.ValidateKeyspaceRequest - (*ValidateKeyspaceResponse)(nil), // 169: vtctldata.ValidateKeyspaceResponse - (*ValidateSchemaKeyspaceRequest)(nil), // 170: vtctldata.ValidateSchemaKeyspaceRequest - (*ValidateSchemaKeyspaceResponse)(nil), // 171: vtctldata.ValidateSchemaKeyspaceResponse - (*ValidateShardRequest)(nil), // 172: vtctldata.ValidateShardRequest - (*ValidateShardResponse)(nil), // 173: vtctldata.ValidateShardResponse - (*ValidateVersionKeyspaceRequest)(nil), // 174: vtctldata.ValidateVersionKeyspaceRequest - (*ValidateVersionKeyspaceResponse)(nil), // 175: vtctldata.ValidateVersionKeyspaceResponse - (*ValidateVersionShardRequest)(nil), // 176: vtctldata.ValidateVersionShardRequest - (*ValidateVersionShardResponse)(nil), // 177: vtctldata.ValidateVersionShardResponse - (*ValidateVSchemaRequest)(nil), // 178: vtctldata.ValidateVSchemaRequest - (*ValidateVSchemaResponse)(nil), // 179: vtctldata.ValidateVSchemaResponse - (*WorkflowUpdateRequest)(nil), // 180: vtctldata.WorkflowUpdateRequest - (*WorkflowUpdateResponse)(nil), // 181: vtctldata.WorkflowUpdateResponse - nil, // 182: vtctldata.Workflow.ShardStreamsEntry - (*Workflow_ReplicationLocation)(nil), // 183: vtctldata.Workflow.ReplicationLocation - (*Workflow_ShardStream)(nil), // 184: vtctldata.Workflow.ShardStream - (*Workflow_Stream)(nil), // 185: vtctldata.Workflow.Stream - (*Workflow_Stream_CopyState)(nil), // 186: vtctldata.Workflow.Stream.CopyState - (*Workflow_Stream_Log)(nil), // 187: vtctldata.Workflow.Stream.Log - nil, // 188: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry - nil, // 189: vtctldata.GetCellsAliasesResponse.AliasesEntry - nil, // 190: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry - (*GetSrvKeyspaceNamesResponse_NameList)(nil), // 191: vtctldata.GetSrvKeyspaceNamesResponse.NameList - nil, // 192: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry - nil, // 193: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry - nil, // 194: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry - nil, // 195: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry - nil, // 196: vtctldata.ValidateResponse.ResultsByKeyspaceEntry - nil, // 197: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry - nil, // 198: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry - nil, // 199: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry - nil, // 200: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry - (*WorkflowUpdateResponse_TabletInfo)(nil), // 201: vtctldata.WorkflowUpdateResponse.TabletInfo - (*logutil.Event)(nil), // 202: logutil.Event - (*topodata.Keyspace)(nil), // 203: topodata.Keyspace - (*topodata.Shard)(nil), // 204: topodata.Shard - (*topodata.CellInfo)(nil), // 205: topodata.CellInfo - (*vschema.RoutingRules)(nil), // 206: vschema.RoutingRules - (*vschema.ShardRoutingRules)(nil), // 207: vschema.ShardRoutingRules - (*vttime.Duration)(nil), // 208: vttime.Duration - (*vtrpc.CallerID)(nil), // 209: vtrpc.CallerID - (*vschema.Keyspace)(nil), // 210: vschema.Keyspace - (*topodata.TabletAlias)(nil), // 211: topodata.TabletAlias - (topodata.TabletType)(0), // 212: topodata.TabletType - (*topodata.Tablet)(nil), // 213: topodata.Tablet - (*topodata.Keyspace_ServedFrom)(nil), // 214: topodata.Keyspace.ServedFrom - (topodata.KeyspaceType)(0), // 215: topodata.KeyspaceType - (*vttime.Time)(nil), // 216: vttime.Time - (*query.QueryResult)(nil), // 217: query.QueryResult - (*tabletmanagerdata.ExecuteHookRequest)(nil), // 218: tabletmanagerdata.ExecuteHookRequest - (*tabletmanagerdata.ExecuteHookResponse)(nil), // 219: tabletmanagerdata.ExecuteHookResponse - (*mysqlctl.BackupInfo)(nil), // 220: mysqlctl.BackupInfo - (*replicationdata.FullStatus)(nil), // 221: replicationdata.FullStatus - (*tabletmanagerdata.Permissions)(nil), // 222: tabletmanagerdata.Permissions - (*tabletmanagerdata.SchemaDefinition)(nil), // 223: tabletmanagerdata.SchemaDefinition - (*topodata.ThrottledAppRule)(nil), // 224: topodata.ThrottledAppRule - (*vschema.SrvVSchema)(nil), // 225: vschema.SrvVSchema - (*topodata.ShardReplicationError)(nil), // 226: topodata.ShardReplicationError - (*topodata.KeyRange)(nil), // 227: topodata.KeyRange - (*topodata.CellsAlias)(nil), // 228: topodata.CellsAlias - (*tabletmanagerdata.UpdateVRWorkflowRequest)(nil), // 229: tabletmanagerdata.UpdateVRWorkflowRequest - (*topodata.Shard_TabletControl)(nil), // 230: topodata.Shard.TabletControl - (*binlogdata.BinlogSource)(nil), // 231: binlogdata.BinlogSource - (*topodata.SrvKeyspace)(nil), // 232: topodata.SrvKeyspace - (*replicationdata.Status)(nil), // 233: replicationdata.Status + (*MoveTablesCreateRequest)(nil), // 100: vtctldata.MoveTablesCreateRequest + (*MoveTablesCreateResponse)(nil), // 101: vtctldata.MoveTablesCreateResponse + (*MoveTablesCompleteRequest)(nil), // 102: vtctldata.MoveTablesCompleteRequest + (*MoveTablesCompleteResponse)(nil), // 103: vtctldata.MoveTablesCompleteResponse + (*PingTabletRequest)(nil), // 104: vtctldata.PingTabletRequest + (*PingTabletResponse)(nil), // 105: vtctldata.PingTabletResponse + (*PlannedReparentShardRequest)(nil), // 106: vtctldata.PlannedReparentShardRequest + (*PlannedReparentShardResponse)(nil), // 107: vtctldata.PlannedReparentShardResponse + (*RebuildKeyspaceGraphRequest)(nil), // 108: vtctldata.RebuildKeyspaceGraphRequest + (*RebuildKeyspaceGraphResponse)(nil), // 109: vtctldata.RebuildKeyspaceGraphResponse + (*RebuildVSchemaGraphRequest)(nil), // 110: vtctldata.RebuildVSchemaGraphRequest + (*RebuildVSchemaGraphResponse)(nil), // 111: vtctldata.RebuildVSchemaGraphResponse + (*RefreshStateRequest)(nil), // 112: vtctldata.RefreshStateRequest + (*RefreshStateResponse)(nil), // 113: vtctldata.RefreshStateResponse + (*RefreshStateByShardRequest)(nil), // 114: vtctldata.RefreshStateByShardRequest + (*RefreshStateByShardResponse)(nil), // 115: vtctldata.RefreshStateByShardResponse + (*ReloadSchemaRequest)(nil), // 116: vtctldata.ReloadSchemaRequest + (*ReloadSchemaResponse)(nil), // 117: vtctldata.ReloadSchemaResponse + (*ReloadSchemaKeyspaceRequest)(nil), // 118: vtctldata.ReloadSchemaKeyspaceRequest + (*ReloadSchemaKeyspaceResponse)(nil), // 119: vtctldata.ReloadSchemaKeyspaceResponse + (*ReloadSchemaShardRequest)(nil), // 120: vtctldata.ReloadSchemaShardRequest + (*ReloadSchemaShardResponse)(nil), // 121: vtctldata.ReloadSchemaShardResponse + (*RemoveBackupRequest)(nil), // 122: vtctldata.RemoveBackupRequest + (*RemoveBackupResponse)(nil), // 123: vtctldata.RemoveBackupResponse + (*RemoveKeyspaceCellRequest)(nil), // 124: vtctldata.RemoveKeyspaceCellRequest + (*RemoveKeyspaceCellResponse)(nil), // 125: vtctldata.RemoveKeyspaceCellResponse + (*RemoveShardCellRequest)(nil), // 126: vtctldata.RemoveShardCellRequest + (*RemoveShardCellResponse)(nil), // 127: vtctldata.RemoveShardCellResponse + (*ReparentTabletRequest)(nil), // 128: vtctldata.ReparentTabletRequest + (*ReparentTabletResponse)(nil), // 129: vtctldata.ReparentTabletResponse + (*RestoreFromBackupRequest)(nil), // 130: vtctldata.RestoreFromBackupRequest + (*RestoreFromBackupResponse)(nil), // 131: vtctldata.RestoreFromBackupResponse + (*RunHealthCheckRequest)(nil), // 132: vtctldata.RunHealthCheckRequest + (*RunHealthCheckResponse)(nil), // 133: vtctldata.RunHealthCheckResponse + (*SetKeyspaceDurabilityPolicyRequest)(nil), // 134: vtctldata.SetKeyspaceDurabilityPolicyRequest + (*SetKeyspaceDurabilityPolicyResponse)(nil), // 135: vtctldata.SetKeyspaceDurabilityPolicyResponse + (*SetKeyspaceServedFromRequest)(nil), // 136: vtctldata.SetKeyspaceServedFromRequest + (*SetKeyspaceServedFromResponse)(nil), // 137: vtctldata.SetKeyspaceServedFromResponse + (*SetKeyspaceShardingInfoRequest)(nil), // 138: vtctldata.SetKeyspaceShardingInfoRequest + (*SetKeyspaceShardingInfoResponse)(nil), // 139: vtctldata.SetKeyspaceShardingInfoResponse + (*SetShardIsPrimaryServingRequest)(nil), // 140: vtctldata.SetShardIsPrimaryServingRequest + (*SetShardIsPrimaryServingResponse)(nil), // 141: vtctldata.SetShardIsPrimaryServingResponse + (*SetShardTabletControlRequest)(nil), // 142: vtctldata.SetShardTabletControlRequest + (*SetShardTabletControlResponse)(nil), // 143: vtctldata.SetShardTabletControlResponse + (*SetWritableRequest)(nil), // 144: vtctldata.SetWritableRequest + (*SetWritableResponse)(nil), // 145: vtctldata.SetWritableResponse + (*ShardReplicationAddRequest)(nil), // 146: vtctldata.ShardReplicationAddRequest + (*ShardReplicationAddResponse)(nil), // 147: vtctldata.ShardReplicationAddResponse + (*ShardReplicationFixRequest)(nil), // 148: vtctldata.ShardReplicationFixRequest + (*ShardReplicationFixResponse)(nil), // 149: vtctldata.ShardReplicationFixResponse + (*ShardReplicationPositionsRequest)(nil), // 150: vtctldata.ShardReplicationPositionsRequest + (*ShardReplicationPositionsResponse)(nil), // 151: vtctldata.ShardReplicationPositionsResponse + (*ShardReplicationRemoveRequest)(nil), // 152: vtctldata.ShardReplicationRemoveRequest + (*ShardReplicationRemoveResponse)(nil), // 153: vtctldata.ShardReplicationRemoveResponse + (*SleepTabletRequest)(nil), // 154: vtctldata.SleepTabletRequest + (*SleepTabletResponse)(nil), // 155: vtctldata.SleepTabletResponse + (*SourceShardAddRequest)(nil), // 156: vtctldata.SourceShardAddRequest + (*SourceShardAddResponse)(nil), // 157: vtctldata.SourceShardAddResponse + (*SourceShardDeleteRequest)(nil), // 158: vtctldata.SourceShardDeleteRequest + (*SourceShardDeleteResponse)(nil), // 159: vtctldata.SourceShardDeleteResponse + (*StartReplicationRequest)(nil), // 160: vtctldata.StartReplicationRequest + (*StartReplicationResponse)(nil), // 161: vtctldata.StartReplicationResponse + (*StopReplicationRequest)(nil), // 162: vtctldata.StopReplicationRequest + (*StopReplicationResponse)(nil), // 163: vtctldata.StopReplicationResponse + (*TabletExternallyReparentedRequest)(nil), // 164: vtctldata.TabletExternallyReparentedRequest + (*TabletExternallyReparentedResponse)(nil), // 165: vtctldata.TabletExternallyReparentedResponse + (*UpdateCellInfoRequest)(nil), // 166: vtctldata.UpdateCellInfoRequest + (*UpdateCellInfoResponse)(nil), // 167: vtctldata.UpdateCellInfoResponse + (*UpdateCellsAliasRequest)(nil), // 168: vtctldata.UpdateCellsAliasRequest + (*UpdateCellsAliasResponse)(nil), // 169: vtctldata.UpdateCellsAliasResponse + (*ValidateRequest)(nil), // 170: vtctldata.ValidateRequest + (*ValidateResponse)(nil), // 171: vtctldata.ValidateResponse + (*ValidateKeyspaceRequest)(nil), // 172: vtctldata.ValidateKeyspaceRequest + (*ValidateKeyspaceResponse)(nil), // 173: vtctldata.ValidateKeyspaceResponse + (*ValidateSchemaKeyspaceRequest)(nil), // 174: vtctldata.ValidateSchemaKeyspaceRequest + (*ValidateSchemaKeyspaceResponse)(nil), // 175: vtctldata.ValidateSchemaKeyspaceResponse + (*ValidateShardRequest)(nil), // 176: vtctldata.ValidateShardRequest + (*ValidateShardResponse)(nil), // 177: vtctldata.ValidateShardResponse + (*ValidateVersionKeyspaceRequest)(nil), // 178: vtctldata.ValidateVersionKeyspaceRequest + (*ValidateVersionKeyspaceResponse)(nil), // 179: vtctldata.ValidateVersionKeyspaceResponse + (*ValidateVersionShardRequest)(nil), // 180: vtctldata.ValidateVersionShardRequest + (*ValidateVersionShardResponse)(nil), // 181: vtctldata.ValidateVersionShardResponse + (*ValidateVSchemaRequest)(nil), // 182: vtctldata.ValidateVSchemaRequest + (*ValidateVSchemaResponse)(nil), // 183: vtctldata.ValidateVSchemaResponse + (*WorkflowDeleteRequest)(nil), // 184: vtctldata.WorkflowDeleteRequest + (*WorkflowDeleteResponse)(nil), // 185: vtctldata.WorkflowDeleteResponse + (*WorkflowStatusRequest)(nil), // 186: vtctldata.WorkflowStatusRequest + (*WorkflowStatusResponse)(nil), // 187: vtctldata.WorkflowStatusResponse + (*WorkflowSwitchTrafficRequest)(nil), // 188: vtctldata.WorkflowSwitchTrafficRequest + (*WorkflowSwitchTrafficResponse)(nil), // 189: vtctldata.WorkflowSwitchTrafficResponse + (*WorkflowUpdateRequest)(nil), // 190: vtctldata.WorkflowUpdateRequest + (*WorkflowUpdateResponse)(nil), // 191: vtctldata.WorkflowUpdateResponse + nil, // 192: vtctldata.Workflow.ShardStreamsEntry + (*Workflow_ReplicationLocation)(nil), // 193: vtctldata.Workflow.ReplicationLocation + (*Workflow_ShardStream)(nil), // 194: vtctldata.Workflow.ShardStream + (*Workflow_Stream)(nil), // 195: vtctldata.Workflow.Stream + (*Workflow_Stream_CopyState)(nil), // 196: vtctldata.Workflow.Stream.CopyState + (*Workflow_Stream_Log)(nil), // 197: vtctldata.Workflow.Stream.Log + nil, // 198: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry + nil, // 199: vtctldata.GetCellsAliasesResponse.AliasesEntry + nil, // 200: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry + (*GetSrvKeyspaceNamesResponse_NameList)(nil), // 201: vtctldata.GetSrvKeyspaceNamesResponse.NameList + nil, // 202: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry + nil, // 203: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry + (*MoveTablesCreateResponse_TabletInfo)(nil), // 204: vtctldata.MoveTablesCreateResponse.TabletInfo + nil, // 205: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry + nil, // 206: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry + nil, // 207: vtctldata.ValidateResponse.ResultsByKeyspaceEntry + nil, // 208: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry + nil, // 209: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry + nil, // 210: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry + nil, // 211: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry + (*WorkflowDeleteResponse_TabletInfo)(nil), // 212: vtctldata.WorkflowDeleteResponse.TabletInfo + (*WorkflowStatusResponse_TableCopyState)(nil), // 213: vtctldata.WorkflowStatusResponse.TableCopyState + (*WorkflowStatusResponse_ShardStreamState)(nil), // 214: vtctldata.WorkflowStatusResponse.ShardStreamState + (*WorkflowStatusResponse_ShardStreams)(nil), // 215: vtctldata.WorkflowStatusResponse.ShardStreams + nil, // 216: vtctldata.WorkflowStatusResponse.TableCopyStateEntry + nil, // 217: vtctldata.WorkflowStatusResponse.ShardStreamsEntry + (*WorkflowUpdateResponse_TabletInfo)(nil), // 218: vtctldata.WorkflowUpdateResponse.TabletInfo + (*logutil.Event)(nil), // 219: logutil.Event + (tabletmanagerdata.TabletSelectionPreference)(0), // 220: tabletmanagerdata.TabletSelectionPreference + (*topodata.Keyspace)(nil), // 221: topodata.Keyspace + (*topodata.Shard)(nil), // 222: topodata.Shard + (*topodata.CellInfo)(nil), // 223: topodata.CellInfo + (*vschema.RoutingRules)(nil), // 224: vschema.RoutingRules + (*vschema.ShardRoutingRules)(nil), // 225: vschema.ShardRoutingRules + (*vttime.Duration)(nil), // 226: vttime.Duration + (*vtrpc.CallerID)(nil), // 227: vtrpc.CallerID + (*vschema.Keyspace)(nil), // 228: vschema.Keyspace + (*topodata.TabletAlias)(nil), // 229: topodata.TabletAlias + (topodata.TabletType)(0), // 230: topodata.TabletType + (*topodata.Tablet)(nil), // 231: topodata.Tablet + (*topodata.Keyspace_ServedFrom)(nil), // 232: topodata.Keyspace.ServedFrom + (topodata.KeyspaceType)(0), // 233: topodata.KeyspaceType + (*vttime.Time)(nil), // 234: vttime.Time + (*query.QueryResult)(nil), // 235: query.QueryResult + (*tabletmanagerdata.ExecuteHookRequest)(nil), // 236: tabletmanagerdata.ExecuteHookRequest + (*tabletmanagerdata.ExecuteHookResponse)(nil), // 237: tabletmanagerdata.ExecuteHookResponse + (*mysqlctl.BackupInfo)(nil), // 238: mysqlctl.BackupInfo + (*replicationdata.FullStatus)(nil), // 239: replicationdata.FullStatus + (*tabletmanagerdata.Permissions)(nil), // 240: tabletmanagerdata.Permissions + (*tabletmanagerdata.SchemaDefinition)(nil), // 241: tabletmanagerdata.SchemaDefinition + (*topodata.ThrottledAppRule)(nil), // 242: topodata.ThrottledAppRule + (*vschema.SrvVSchema)(nil), // 243: vschema.SrvVSchema + (*topodata.ShardReplicationError)(nil), // 244: topodata.ShardReplicationError + (*topodata.KeyRange)(nil), // 245: topodata.KeyRange + (*topodata.CellsAlias)(nil), // 246: topodata.CellsAlias + (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 247: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*topodata.Shard_TabletControl)(nil), // 248: topodata.Shard.TabletControl + (*binlogdata.BinlogSource)(nil), // 249: binlogdata.BinlogSource + (*topodata.SrvKeyspace)(nil), // 250: topodata.SrvKeyspace + (*replicationdata.Status)(nil), // 251: replicationdata.Status } var file_vtctldata_proto_depIdxs = []int32{ - 202, // 0: vtctldata.ExecuteVtctlCommandResponse.event:type_name -> logutil.Event + 219, // 0: vtctldata.ExecuteVtctlCommandResponse.event:type_name -> logutil.Event 3, // 1: vtctldata.MaterializeSettings.table_settings:type_name -> vtctldata.TableMaterializeSettings 0, // 2: vtctldata.MaterializeSettings.materialization_intent:type_name -> vtctldata.MaterializationIntent - 203, // 3: vtctldata.Keyspace.keyspace:type_name -> topodata.Keyspace - 204, // 4: vtctldata.Shard.shard:type_name -> topodata.Shard - 183, // 5: vtctldata.Workflow.source:type_name -> vtctldata.Workflow.ReplicationLocation - 183, // 6: vtctldata.Workflow.target:type_name -> vtctldata.Workflow.ReplicationLocation - 182, // 7: vtctldata.Workflow.shard_streams:type_name -> vtctldata.Workflow.ShardStreamsEntry - 205, // 8: vtctldata.AddCellInfoRequest.cell_info:type_name -> topodata.CellInfo - 206, // 9: vtctldata.ApplyRoutingRulesRequest.routing_rules:type_name -> vschema.RoutingRules - 207, // 10: vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules:type_name -> vschema.ShardRoutingRules - 208, // 11: vtctldata.ApplySchemaRequest.wait_replicas_timeout:type_name -> vttime.Duration - 209, // 12: vtctldata.ApplySchemaRequest.caller_id:type_name -> vtrpc.CallerID - 210, // 13: vtctldata.ApplyVSchemaRequest.v_schema:type_name -> vschema.Keyspace - 210, // 14: vtctldata.ApplyVSchemaResponse.v_schema:type_name -> vschema.Keyspace - 211, // 15: vtctldata.BackupRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 16: vtctldata.BackupResponse.tablet_alias:type_name -> topodata.TabletAlias - 202, // 17: vtctldata.BackupResponse.event:type_name -> logutil.Event - 211, // 18: vtctldata.ChangeTabletTypeRequest.tablet_alias:type_name -> topodata.TabletAlias - 212, // 19: vtctldata.ChangeTabletTypeRequest.db_type:type_name -> topodata.TabletType - 213, // 20: vtctldata.ChangeTabletTypeResponse.before_tablet:type_name -> topodata.Tablet - 213, // 21: vtctldata.ChangeTabletTypeResponse.after_tablet:type_name -> topodata.Tablet - 214, // 22: vtctldata.CreateKeyspaceRequest.served_froms:type_name -> topodata.Keyspace.ServedFrom - 215, // 23: vtctldata.CreateKeyspaceRequest.type:type_name -> topodata.KeyspaceType - 216, // 24: vtctldata.CreateKeyspaceRequest.snapshot_time:type_name -> vttime.Time - 5, // 25: vtctldata.CreateKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace - 5, // 26: vtctldata.CreateShardResponse.keyspace:type_name -> vtctldata.Keyspace - 6, // 27: vtctldata.CreateShardResponse.shard:type_name -> vtctldata.Shard - 6, // 28: vtctldata.DeleteShardsRequest.shards:type_name -> vtctldata.Shard - 211, // 29: vtctldata.DeleteTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias - 211, // 30: vtctldata.EmergencyReparentShardRequest.new_primary:type_name -> topodata.TabletAlias - 211, // 31: vtctldata.EmergencyReparentShardRequest.ignore_replicas:type_name -> topodata.TabletAlias - 208, // 32: vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration - 211, // 33: vtctldata.EmergencyReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias - 202, // 34: vtctldata.EmergencyReparentShardResponse.events:type_name -> logutil.Event - 211, // 35: vtctldata.ExecuteFetchAsAppRequest.tablet_alias:type_name -> topodata.TabletAlias - 217, // 36: vtctldata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult - 211, // 37: vtctldata.ExecuteFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias - 217, // 38: vtctldata.ExecuteFetchAsDBAResponse.result:type_name -> query.QueryResult - 211, // 39: vtctldata.ExecuteHookRequest.tablet_alias:type_name -> topodata.TabletAlias - 218, // 40: vtctldata.ExecuteHookRequest.tablet_hook_request:type_name -> tabletmanagerdata.ExecuteHookRequest - 219, // 41: vtctldata.ExecuteHookResponse.hook_result:type_name -> tabletmanagerdata.ExecuteHookResponse - 188, // 42: vtctldata.FindAllShardsInKeyspaceResponse.shards:type_name -> vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry - 220, // 43: vtctldata.GetBackupsResponse.backups:type_name -> mysqlctl.BackupInfo - 205, // 44: vtctldata.GetCellInfoResponse.cell_info:type_name -> topodata.CellInfo - 189, // 45: vtctldata.GetCellsAliasesResponse.aliases:type_name -> vtctldata.GetCellsAliasesResponse.AliasesEntry - 211, // 46: vtctldata.GetFullStatusRequest.tablet_alias:type_name -> topodata.TabletAlias - 221, // 47: vtctldata.GetFullStatusResponse.status:type_name -> replicationdata.FullStatus - 5, // 48: vtctldata.GetKeyspacesResponse.keyspaces:type_name -> vtctldata.Keyspace - 5, // 49: vtctldata.GetKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace - 211, // 50: vtctldata.GetPermissionsRequest.tablet_alias:type_name -> topodata.TabletAlias - 222, // 51: vtctldata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions - 206, // 52: vtctldata.GetRoutingRulesResponse.routing_rules:type_name -> vschema.RoutingRules - 211, // 53: vtctldata.GetSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias - 223, // 54: vtctldata.GetSchemaResponse.schema:type_name -> tabletmanagerdata.SchemaDefinition - 6, // 55: vtctldata.GetShardResponse.shard:type_name -> vtctldata.Shard - 207, // 56: vtctldata.GetShardRoutingRulesResponse.shard_routing_rules:type_name -> vschema.ShardRoutingRules - 190, // 57: vtctldata.GetSrvKeyspaceNamesResponse.names:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry - 192, // 58: vtctldata.GetSrvKeyspacesResponse.srv_keyspaces:type_name -> vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry - 224, // 59: vtctldata.UpdateThrottlerConfigRequest.throttled_app:type_name -> topodata.ThrottledAppRule - 225, // 60: vtctldata.GetSrvVSchemaResponse.srv_v_schema:type_name -> vschema.SrvVSchema - 193, // 61: vtctldata.GetSrvVSchemasResponse.srv_v_schemas:type_name -> vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry - 211, // 62: vtctldata.GetTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 213, // 63: vtctldata.GetTabletResponse.tablet:type_name -> topodata.Tablet - 211, // 64: vtctldata.GetTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias - 212, // 65: vtctldata.GetTabletsRequest.tablet_type:type_name -> topodata.TabletType - 213, // 66: vtctldata.GetTabletsResponse.tablets:type_name -> topodata.Tablet - 91, // 67: vtctldata.GetTopologyPathResponse.cell:type_name -> vtctldata.TopologyCell - 211, // 68: vtctldata.GetVersionRequest.tablet_alias:type_name -> topodata.TabletAlias - 210, // 69: vtctldata.GetVSchemaResponse.v_schema:type_name -> vschema.Keyspace - 7, // 70: vtctldata.GetWorkflowsResponse.workflows:type_name -> vtctldata.Workflow - 211, // 71: vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias:type_name -> topodata.TabletAlias - 208, // 72: vtctldata.InitShardPrimaryRequest.wait_replicas_timeout:type_name -> vttime.Duration - 202, // 73: vtctldata.InitShardPrimaryResponse.events:type_name -> logutil.Event - 211, // 74: vtctldata.PingTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 75: vtctldata.PlannedReparentShardRequest.new_primary:type_name -> topodata.TabletAlias - 211, // 76: vtctldata.PlannedReparentShardRequest.avoid_primary:type_name -> topodata.TabletAlias - 208, // 77: vtctldata.PlannedReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration - 211, // 78: vtctldata.PlannedReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias - 202, // 79: vtctldata.PlannedReparentShardResponse.events:type_name -> logutil.Event - 211, // 80: vtctldata.RefreshStateRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 81: vtctldata.ReloadSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias - 202, // 82: vtctldata.ReloadSchemaKeyspaceResponse.events:type_name -> logutil.Event - 202, // 83: vtctldata.ReloadSchemaShardResponse.events:type_name -> logutil.Event - 211, // 84: vtctldata.ReparentTabletRequest.tablet:type_name -> topodata.TabletAlias - 211, // 85: vtctldata.ReparentTabletResponse.primary:type_name -> topodata.TabletAlias - 211, // 86: vtctldata.RestoreFromBackupRequest.tablet_alias:type_name -> topodata.TabletAlias - 216, // 87: vtctldata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time - 216, // 88: vtctldata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time - 211, // 89: vtctldata.RestoreFromBackupResponse.tablet_alias:type_name -> topodata.TabletAlias - 202, // 90: vtctldata.RestoreFromBackupResponse.event:type_name -> logutil.Event - 211, // 91: vtctldata.RunHealthCheckRequest.tablet_alias:type_name -> topodata.TabletAlias - 203, // 92: vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace:type_name -> topodata.Keyspace - 212, // 93: vtctldata.SetKeyspaceServedFromRequest.tablet_type:type_name -> topodata.TabletType - 203, // 94: vtctldata.SetKeyspaceServedFromResponse.keyspace:type_name -> topodata.Keyspace - 203, // 95: vtctldata.SetKeyspaceShardingInfoResponse.keyspace:type_name -> topodata.Keyspace - 204, // 96: vtctldata.SetShardIsPrimaryServingResponse.shard:type_name -> topodata.Shard - 212, // 97: vtctldata.SetShardTabletControlRequest.tablet_type:type_name -> topodata.TabletType - 204, // 98: vtctldata.SetShardTabletControlResponse.shard:type_name -> topodata.Shard - 211, // 99: vtctldata.SetWritableRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 100: vtctldata.ShardReplicationAddRequest.tablet_alias:type_name -> topodata.TabletAlias - 226, // 101: vtctldata.ShardReplicationFixResponse.error:type_name -> topodata.ShardReplicationError - 194, // 102: vtctldata.ShardReplicationPositionsResponse.replication_statuses:type_name -> vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry - 195, // 103: vtctldata.ShardReplicationPositionsResponse.tablet_map:type_name -> vtctldata.ShardReplicationPositionsResponse.TabletMapEntry - 211, // 104: vtctldata.ShardReplicationRemoveRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 105: vtctldata.SleepTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 208, // 106: vtctldata.SleepTabletRequest.duration:type_name -> vttime.Duration - 227, // 107: vtctldata.SourceShardAddRequest.key_range:type_name -> topodata.KeyRange - 204, // 108: vtctldata.SourceShardAddResponse.shard:type_name -> topodata.Shard - 204, // 109: vtctldata.SourceShardDeleteResponse.shard:type_name -> topodata.Shard - 211, // 110: vtctldata.StartReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 111: vtctldata.StopReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias - 211, // 112: vtctldata.TabletExternallyReparentedRequest.tablet:type_name -> topodata.TabletAlias - 211, // 113: vtctldata.TabletExternallyReparentedResponse.new_primary:type_name -> topodata.TabletAlias - 211, // 114: vtctldata.TabletExternallyReparentedResponse.old_primary:type_name -> topodata.TabletAlias - 205, // 115: vtctldata.UpdateCellInfoRequest.cell_info:type_name -> topodata.CellInfo - 205, // 116: vtctldata.UpdateCellInfoResponse.cell_info:type_name -> topodata.CellInfo - 228, // 117: vtctldata.UpdateCellsAliasRequest.cells_alias:type_name -> topodata.CellsAlias - 228, // 118: vtctldata.UpdateCellsAliasResponse.cells_alias:type_name -> topodata.CellsAlias - 196, // 119: vtctldata.ValidateResponse.results_by_keyspace:type_name -> vtctldata.ValidateResponse.ResultsByKeyspaceEntry - 197, // 120: vtctldata.ValidateKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry - 198, // 121: vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry - 199, // 122: vtctldata.ValidateVersionKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry - 200, // 123: vtctldata.ValidateVSchemaResponse.results_by_shard:type_name -> vtctldata.ValidateVSchemaResponse.ResultsByShardEntry - 229, // 124: vtctldata.WorkflowUpdateRequest.tablet_request:type_name -> tabletmanagerdata.UpdateVRWorkflowRequest - 201, // 125: vtctldata.WorkflowUpdateResponse.details:type_name -> vtctldata.WorkflowUpdateResponse.TabletInfo - 184, // 126: vtctldata.Workflow.ShardStreamsEntry.value:type_name -> vtctldata.Workflow.ShardStream - 185, // 127: vtctldata.Workflow.ShardStream.streams:type_name -> vtctldata.Workflow.Stream - 230, // 128: vtctldata.Workflow.ShardStream.tablet_controls:type_name -> topodata.Shard.TabletControl - 211, // 129: vtctldata.Workflow.Stream.tablet:type_name -> topodata.TabletAlias - 231, // 130: vtctldata.Workflow.Stream.binlog_source:type_name -> binlogdata.BinlogSource - 216, // 131: vtctldata.Workflow.Stream.transaction_timestamp:type_name -> vttime.Time - 216, // 132: vtctldata.Workflow.Stream.time_updated:type_name -> vttime.Time - 186, // 133: vtctldata.Workflow.Stream.copy_states:type_name -> vtctldata.Workflow.Stream.CopyState - 187, // 134: vtctldata.Workflow.Stream.logs:type_name -> vtctldata.Workflow.Stream.Log - 216, // 135: vtctldata.Workflow.Stream.Log.created_at:type_name -> vttime.Time - 216, // 136: vtctldata.Workflow.Stream.Log.updated_at:type_name -> vttime.Time - 6, // 137: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry.value:type_name -> vtctldata.Shard - 228, // 138: vtctldata.GetCellsAliasesResponse.AliasesEntry.value:type_name -> topodata.CellsAlias - 191, // 139: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry.value:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NameList - 232, // 140: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry.value:type_name -> topodata.SrvKeyspace - 225, // 141: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry.value:type_name -> vschema.SrvVSchema - 233, // 142: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry.value:type_name -> replicationdata.Status - 213, // 143: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry.value:type_name -> topodata.Tablet - 169, // 144: vtctldata.ValidateResponse.ResultsByKeyspaceEntry.value:type_name -> vtctldata.ValidateKeyspaceResponse - 173, // 145: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 173, // 146: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 173, // 147: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 173, // 148: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 149, // [149:149] is the sub-list for method output_type - 149, // [149:149] is the sub-list for method input_type - 149, // [149:149] is the sub-list for extension type_name - 149, // [149:149] is the sub-list for extension extendee - 0, // [0:149] is the sub-list for field type_name + 220, // 3: vtctldata.MaterializeSettings.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 221, // 4: vtctldata.Keyspace.keyspace:type_name -> topodata.Keyspace + 222, // 5: vtctldata.Shard.shard:type_name -> topodata.Shard + 193, // 6: vtctldata.Workflow.source:type_name -> vtctldata.Workflow.ReplicationLocation + 193, // 7: vtctldata.Workflow.target:type_name -> vtctldata.Workflow.ReplicationLocation + 192, // 8: vtctldata.Workflow.shard_streams:type_name -> vtctldata.Workflow.ShardStreamsEntry + 223, // 9: vtctldata.AddCellInfoRequest.cell_info:type_name -> topodata.CellInfo + 224, // 10: vtctldata.ApplyRoutingRulesRequest.routing_rules:type_name -> vschema.RoutingRules + 225, // 11: vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules:type_name -> vschema.ShardRoutingRules + 226, // 12: vtctldata.ApplySchemaRequest.wait_replicas_timeout:type_name -> vttime.Duration + 227, // 13: vtctldata.ApplySchemaRequest.caller_id:type_name -> vtrpc.CallerID + 228, // 14: vtctldata.ApplyVSchemaRequest.v_schema:type_name -> vschema.Keyspace + 228, // 15: vtctldata.ApplyVSchemaResponse.v_schema:type_name -> vschema.Keyspace + 229, // 16: vtctldata.BackupRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 17: vtctldata.BackupResponse.tablet_alias:type_name -> topodata.TabletAlias + 219, // 18: vtctldata.BackupResponse.event:type_name -> logutil.Event + 229, // 19: vtctldata.ChangeTabletTypeRequest.tablet_alias:type_name -> topodata.TabletAlias + 230, // 20: vtctldata.ChangeTabletTypeRequest.db_type:type_name -> topodata.TabletType + 231, // 21: vtctldata.ChangeTabletTypeResponse.before_tablet:type_name -> topodata.Tablet + 231, // 22: vtctldata.ChangeTabletTypeResponse.after_tablet:type_name -> topodata.Tablet + 232, // 23: vtctldata.CreateKeyspaceRequest.served_froms:type_name -> topodata.Keyspace.ServedFrom + 233, // 24: vtctldata.CreateKeyspaceRequest.type:type_name -> topodata.KeyspaceType + 234, // 25: vtctldata.CreateKeyspaceRequest.snapshot_time:type_name -> vttime.Time + 5, // 26: vtctldata.CreateKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace + 5, // 27: vtctldata.CreateShardResponse.keyspace:type_name -> vtctldata.Keyspace + 6, // 28: vtctldata.CreateShardResponse.shard:type_name -> vtctldata.Shard + 6, // 29: vtctldata.DeleteShardsRequest.shards:type_name -> vtctldata.Shard + 229, // 30: vtctldata.DeleteTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias + 229, // 31: vtctldata.EmergencyReparentShardRequest.new_primary:type_name -> topodata.TabletAlias + 229, // 32: vtctldata.EmergencyReparentShardRequest.ignore_replicas:type_name -> topodata.TabletAlias + 226, // 33: vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration + 229, // 34: vtctldata.EmergencyReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias + 219, // 35: vtctldata.EmergencyReparentShardResponse.events:type_name -> logutil.Event + 229, // 36: vtctldata.ExecuteFetchAsAppRequest.tablet_alias:type_name -> topodata.TabletAlias + 235, // 37: vtctldata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult + 229, // 38: vtctldata.ExecuteFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias + 235, // 39: vtctldata.ExecuteFetchAsDBAResponse.result:type_name -> query.QueryResult + 229, // 40: vtctldata.ExecuteHookRequest.tablet_alias:type_name -> topodata.TabletAlias + 236, // 41: vtctldata.ExecuteHookRequest.tablet_hook_request:type_name -> tabletmanagerdata.ExecuteHookRequest + 237, // 42: vtctldata.ExecuteHookResponse.hook_result:type_name -> tabletmanagerdata.ExecuteHookResponse + 198, // 43: vtctldata.FindAllShardsInKeyspaceResponse.shards:type_name -> vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry + 238, // 44: vtctldata.GetBackupsResponse.backups:type_name -> mysqlctl.BackupInfo + 223, // 45: vtctldata.GetCellInfoResponse.cell_info:type_name -> topodata.CellInfo + 199, // 46: vtctldata.GetCellsAliasesResponse.aliases:type_name -> vtctldata.GetCellsAliasesResponse.AliasesEntry + 229, // 47: vtctldata.GetFullStatusRequest.tablet_alias:type_name -> topodata.TabletAlias + 239, // 48: vtctldata.GetFullStatusResponse.status:type_name -> replicationdata.FullStatus + 5, // 49: vtctldata.GetKeyspacesResponse.keyspaces:type_name -> vtctldata.Keyspace + 5, // 50: vtctldata.GetKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace + 229, // 51: vtctldata.GetPermissionsRequest.tablet_alias:type_name -> topodata.TabletAlias + 240, // 52: vtctldata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions + 224, // 53: vtctldata.GetRoutingRulesResponse.routing_rules:type_name -> vschema.RoutingRules + 229, // 54: vtctldata.GetSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias + 241, // 55: vtctldata.GetSchemaResponse.schema:type_name -> tabletmanagerdata.SchemaDefinition + 6, // 56: vtctldata.GetShardResponse.shard:type_name -> vtctldata.Shard + 225, // 57: vtctldata.GetShardRoutingRulesResponse.shard_routing_rules:type_name -> vschema.ShardRoutingRules + 200, // 58: vtctldata.GetSrvKeyspaceNamesResponse.names:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry + 202, // 59: vtctldata.GetSrvKeyspacesResponse.srv_keyspaces:type_name -> vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry + 242, // 60: vtctldata.UpdateThrottlerConfigRequest.throttled_app:type_name -> topodata.ThrottledAppRule + 243, // 61: vtctldata.GetSrvVSchemaResponse.srv_v_schema:type_name -> vschema.SrvVSchema + 203, // 62: vtctldata.GetSrvVSchemasResponse.srv_v_schemas:type_name -> vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry + 229, // 63: vtctldata.GetTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 231, // 64: vtctldata.GetTabletResponse.tablet:type_name -> topodata.Tablet + 229, // 65: vtctldata.GetTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias + 230, // 66: vtctldata.GetTabletsRequest.tablet_type:type_name -> topodata.TabletType + 231, // 67: vtctldata.GetTabletsResponse.tablets:type_name -> topodata.Tablet + 91, // 68: vtctldata.GetTopologyPathResponse.cell:type_name -> vtctldata.TopologyCell + 229, // 69: vtctldata.GetVersionRequest.tablet_alias:type_name -> topodata.TabletAlias + 228, // 70: vtctldata.GetVSchemaResponse.v_schema:type_name -> vschema.Keyspace + 7, // 71: vtctldata.GetWorkflowsResponse.workflows:type_name -> vtctldata.Workflow + 229, // 72: vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias:type_name -> topodata.TabletAlias + 226, // 73: vtctldata.InitShardPrimaryRequest.wait_replicas_timeout:type_name -> vttime.Duration + 219, // 74: vtctldata.InitShardPrimaryResponse.events:type_name -> logutil.Event + 230, // 75: vtctldata.MoveTablesCreateRequest.tablet_types:type_name -> topodata.TabletType + 220, // 76: vtctldata.MoveTablesCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 204, // 77: vtctldata.MoveTablesCreateResponse.details:type_name -> vtctldata.MoveTablesCreateResponse.TabletInfo + 229, // 78: vtctldata.PingTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 79: vtctldata.PlannedReparentShardRequest.new_primary:type_name -> topodata.TabletAlias + 229, // 80: vtctldata.PlannedReparentShardRequest.avoid_primary:type_name -> topodata.TabletAlias + 226, // 81: vtctldata.PlannedReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration + 229, // 82: vtctldata.PlannedReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias + 219, // 83: vtctldata.PlannedReparentShardResponse.events:type_name -> logutil.Event + 229, // 84: vtctldata.RefreshStateRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 85: vtctldata.ReloadSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias + 219, // 86: vtctldata.ReloadSchemaKeyspaceResponse.events:type_name -> logutil.Event + 219, // 87: vtctldata.ReloadSchemaShardResponse.events:type_name -> logutil.Event + 229, // 88: vtctldata.ReparentTabletRequest.tablet:type_name -> topodata.TabletAlias + 229, // 89: vtctldata.ReparentTabletResponse.primary:type_name -> topodata.TabletAlias + 229, // 90: vtctldata.RestoreFromBackupRequest.tablet_alias:type_name -> topodata.TabletAlias + 234, // 91: vtctldata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time + 234, // 92: vtctldata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time + 229, // 93: vtctldata.RestoreFromBackupResponse.tablet_alias:type_name -> topodata.TabletAlias + 219, // 94: vtctldata.RestoreFromBackupResponse.event:type_name -> logutil.Event + 229, // 95: vtctldata.RunHealthCheckRequest.tablet_alias:type_name -> topodata.TabletAlias + 221, // 96: vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace:type_name -> topodata.Keyspace + 230, // 97: vtctldata.SetKeyspaceServedFromRequest.tablet_type:type_name -> topodata.TabletType + 221, // 98: vtctldata.SetKeyspaceServedFromResponse.keyspace:type_name -> topodata.Keyspace + 221, // 99: vtctldata.SetKeyspaceShardingInfoResponse.keyspace:type_name -> topodata.Keyspace + 222, // 100: vtctldata.SetShardIsPrimaryServingResponse.shard:type_name -> topodata.Shard + 230, // 101: vtctldata.SetShardTabletControlRequest.tablet_type:type_name -> topodata.TabletType + 222, // 102: vtctldata.SetShardTabletControlResponse.shard:type_name -> topodata.Shard + 229, // 103: vtctldata.SetWritableRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 104: vtctldata.ShardReplicationAddRequest.tablet_alias:type_name -> topodata.TabletAlias + 244, // 105: vtctldata.ShardReplicationFixResponse.error:type_name -> topodata.ShardReplicationError + 205, // 106: vtctldata.ShardReplicationPositionsResponse.replication_statuses:type_name -> vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry + 206, // 107: vtctldata.ShardReplicationPositionsResponse.tablet_map:type_name -> vtctldata.ShardReplicationPositionsResponse.TabletMapEntry + 229, // 108: vtctldata.ShardReplicationRemoveRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 109: vtctldata.SleepTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 226, // 110: vtctldata.SleepTabletRequest.duration:type_name -> vttime.Duration + 245, // 111: vtctldata.SourceShardAddRequest.key_range:type_name -> topodata.KeyRange + 222, // 112: vtctldata.SourceShardAddResponse.shard:type_name -> topodata.Shard + 222, // 113: vtctldata.SourceShardDeleteResponse.shard:type_name -> topodata.Shard + 229, // 114: vtctldata.StartReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 115: vtctldata.StopReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias + 229, // 116: vtctldata.TabletExternallyReparentedRequest.tablet:type_name -> topodata.TabletAlias + 229, // 117: vtctldata.TabletExternallyReparentedResponse.new_primary:type_name -> topodata.TabletAlias + 229, // 118: vtctldata.TabletExternallyReparentedResponse.old_primary:type_name -> topodata.TabletAlias + 223, // 119: vtctldata.UpdateCellInfoRequest.cell_info:type_name -> topodata.CellInfo + 223, // 120: vtctldata.UpdateCellInfoResponse.cell_info:type_name -> topodata.CellInfo + 246, // 121: vtctldata.UpdateCellsAliasRequest.cells_alias:type_name -> topodata.CellsAlias + 246, // 122: vtctldata.UpdateCellsAliasResponse.cells_alias:type_name -> topodata.CellsAlias + 207, // 123: vtctldata.ValidateResponse.results_by_keyspace:type_name -> vtctldata.ValidateResponse.ResultsByKeyspaceEntry + 208, // 124: vtctldata.ValidateKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry + 209, // 125: vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry + 210, // 126: vtctldata.ValidateVersionKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry + 211, // 127: vtctldata.ValidateVSchemaResponse.results_by_shard:type_name -> vtctldata.ValidateVSchemaResponse.ResultsByShardEntry + 212, // 128: vtctldata.WorkflowDeleteResponse.details:type_name -> vtctldata.WorkflowDeleteResponse.TabletInfo + 216, // 129: vtctldata.WorkflowStatusResponse.table_copy_state:type_name -> vtctldata.WorkflowStatusResponse.TableCopyStateEntry + 217, // 130: vtctldata.WorkflowStatusResponse.shard_streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamsEntry + 230, // 131: vtctldata.WorkflowSwitchTrafficRequest.tablet_types:type_name -> topodata.TabletType + 226, // 132: vtctldata.WorkflowSwitchTrafficRequest.max_replication_lag_allowed:type_name -> vttime.Duration + 226, // 133: vtctldata.WorkflowSwitchTrafficRequest.timeout:type_name -> vttime.Duration + 247, // 134: vtctldata.WorkflowUpdateRequest.tablet_request:type_name -> tabletmanagerdata.UpdateVReplicationWorkflowRequest + 218, // 135: vtctldata.WorkflowUpdateResponse.details:type_name -> vtctldata.WorkflowUpdateResponse.TabletInfo + 194, // 136: vtctldata.Workflow.ShardStreamsEntry.value:type_name -> vtctldata.Workflow.ShardStream + 195, // 137: vtctldata.Workflow.ShardStream.streams:type_name -> vtctldata.Workflow.Stream + 248, // 138: vtctldata.Workflow.ShardStream.tablet_controls:type_name -> topodata.Shard.TabletControl + 229, // 139: vtctldata.Workflow.Stream.tablet:type_name -> topodata.TabletAlias + 249, // 140: vtctldata.Workflow.Stream.binlog_source:type_name -> binlogdata.BinlogSource + 234, // 141: vtctldata.Workflow.Stream.transaction_timestamp:type_name -> vttime.Time + 234, // 142: vtctldata.Workflow.Stream.time_updated:type_name -> vttime.Time + 196, // 143: vtctldata.Workflow.Stream.copy_states:type_name -> vtctldata.Workflow.Stream.CopyState + 197, // 144: vtctldata.Workflow.Stream.logs:type_name -> vtctldata.Workflow.Stream.Log + 234, // 145: vtctldata.Workflow.Stream.Log.created_at:type_name -> vttime.Time + 234, // 146: vtctldata.Workflow.Stream.Log.updated_at:type_name -> vttime.Time + 6, // 147: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry.value:type_name -> vtctldata.Shard + 246, // 148: vtctldata.GetCellsAliasesResponse.AliasesEntry.value:type_name -> topodata.CellsAlias + 201, // 149: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry.value:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NameList + 250, // 150: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry.value:type_name -> topodata.SrvKeyspace + 243, // 151: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry.value:type_name -> vschema.SrvVSchema + 229, // 152: vtctldata.MoveTablesCreateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 251, // 153: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry.value:type_name -> replicationdata.Status + 231, // 154: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry.value:type_name -> topodata.Tablet + 173, // 155: vtctldata.ValidateResponse.ResultsByKeyspaceEntry.value:type_name -> vtctldata.ValidateKeyspaceResponse + 177, // 156: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 177, // 157: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 177, // 158: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 177, // 159: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 229, // 160: vtctldata.WorkflowDeleteResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 229, // 161: vtctldata.WorkflowStatusResponse.ShardStreamState.tablet:type_name -> topodata.TabletAlias + 214, // 162: vtctldata.WorkflowStatusResponse.ShardStreams.streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamState + 213, // 163: vtctldata.WorkflowStatusResponse.TableCopyStateEntry.value:type_name -> vtctldata.WorkflowStatusResponse.TableCopyState + 215, // 164: vtctldata.WorkflowStatusResponse.ShardStreamsEntry.value:type_name -> vtctldata.WorkflowStatusResponse.ShardStreams + 229, // 165: vtctldata.WorkflowUpdateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 166, // [166:166] is the sub-list for method output_type + 166, // [166:166] is the sub-list for method input_type + 166, // [166:166] is the sub-list for extension type_name + 166, // [166:166] is the sub-list for extension extendee + 0, // [0:166] is the sub-list for field type_name } func init() { file_vtctldata_proto_init() } @@ -13946,7 +15346,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PingTabletRequest); i { + switch v := v.(*MoveTablesCreateRequest); i { case 0: return &v.state case 1: @@ -13958,7 +15358,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PingTabletResponse); i { + switch v := v.(*MoveTablesCreateResponse); i { case 0: return &v.state case 1: @@ -13970,7 +15370,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PlannedReparentShardRequest); i { + switch v := v.(*MoveTablesCompleteRequest); i { case 0: return &v.state case 1: @@ -13982,7 +15382,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PlannedReparentShardResponse); i { + switch v := v.(*MoveTablesCompleteResponse); i { case 0: return &v.state case 1: @@ -13994,7 +15394,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RebuildKeyspaceGraphRequest); i { + switch v := v.(*PingTabletRequest); i { case 0: return &v.state case 1: @@ -14006,7 +15406,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RebuildKeyspaceGraphResponse); i { + switch v := v.(*PingTabletResponse); i { case 0: return &v.state case 1: @@ -14018,7 +15418,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RebuildVSchemaGraphRequest); i { + switch v := v.(*PlannedReparentShardRequest); i { case 0: return &v.state case 1: @@ -14030,7 +15430,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RebuildVSchemaGraphResponse); i { + switch v := v.(*PlannedReparentShardResponse); i { case 0: return &v.state case 1: @@ -14042,7 +15442,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RefreshStateRequest); i { + switch v := v.(*RebuildKeyspaceGraphRequest); i { case 0: return &v.state case 1: @@ -14054,7 +15454,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RefreshStateResponse); i { + switch v := v.(*RebuildKeyspaceGraphResponse); i { case 0: return &v.state case 1: @@ -14066,7 +15466,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RefreshStateByShardRequest); i { + switch v := v.(*RebuildVSchemaGraphRequest); i { case 0: return &v.state case 1: @@ -14078,7 +15478,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RefreshStateByShardResponse); i { + switch v := v.(*RebuildVSchemaGraphResponse); i { case 0: return &v.state case 1: @@ -14090,7 +15490,7 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReloadSchemaRequest); i { + switch v := v.(*RefreshStateRequest); i { case 0: return &v.state case 1: @@ -14102,6 +15502,54 @@ func file_vtctldata_proto_init() { } } file_vtctldata_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RefreshStateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RefreshStateByShardRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RefreshStateByShardResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReloadSchemaRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaResponse); i { case 0: return &v.state @@ -14113,7 +15561,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaKeyspaceRequest); i { case 0: return &v.state @@ -14125,7 +15573,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaKeyspaceResponse); i { case 0: return &v.state @@ -14137,7 +15585,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaShardRequest); i { case 0: return &v.state @@ -14149,7 +15597,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaShardResponse); i { case 0: return &v.state @@ -14161,7 +15609,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveBackupRequest); i { case 0: return &v.state @@ -14173,7 +15621,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveBackupResponse); i { case 0: return &v.state @@ -14185,7 +15633,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveKeyspaceCellRequest); i { case 0: return &v.state @@ -14197,7 +15645,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveKeyspaceCellResponse); i { case 0: return &v.state @@ -14209,7 +15657,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveShardCellRequest); i { case 0: return &v.state @@ -14221,7 +15669,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveShardCellResponse); i { case 0: return &v.state @@ -14233,7 +15681,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[127].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReparentTabletRequest); i { case 0: return &v.state @@ -14245,7 +15693,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[128].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReparentTabletResponse); i { case 0: return &v.state @@ -14257,7 +15705,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[129].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RestoreFromBackupRequest); i { case 0: return &v.state @@ -14269,7 +15717,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[130].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RestoreFromBackupResponse); i { case 0: return &v.state @@ -14281,7 +15729,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[127].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[131].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RunHealthCheckRequest); i { case 0: return &v.state @@ -14293,7 +15741,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[128].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[132].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RunHealthCheckResponse); i { case 0: return &v.state @@ -14305,7 +15753,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[129].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[133].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceDurabilityPolicyRequest); i { case 0: return &v.state @@ -14317,7 +15765,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[130].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[134].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceDurabilityPolicyResponse); i { case 0: return &v.state @@ -14329,7 +15777,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[131].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[135].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceServedFromRequest); i { case 0: return &v.state @@ -14341,7 +15789,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[132].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[136].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceServedFromResponse); i { case 0: return &v.state @@ -14353,7 +15801,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[133].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[137].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceShardingInfoRequest); i { case 0: return &v.state @@ -14365,7 +15813,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[134].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[138].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceShardingInfoResponse); i { case 0: return &v.state @@ -14377,7 +15825,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[135].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[139].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardIsPrimaryServingRequest); i { case 0: return &v.state @@ -14389,7 +15837,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[136].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[140].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardIsPrimaryServingResponse); i { case 0: return &v.state @@ -14401,7 +15849,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[137].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[141].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardTabletControlRequest); i { case 0: return &v.state @@ -14413,7 +15861,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[138].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[142].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardTabletControlResponse); i { case 0: return &v.state @@ -14425,7 +15873,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[139].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[143].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetWritableRequest); i { case 0: return &v.state @@ -14437,7 +15885,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[140].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[144].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetWritableResponse); i { case 0: return &v.state @@ -14449,7 +15897,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[141].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[145].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationAddRequest); i { case 0: return &v.state @@ -14461,7 +15909,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[142].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[146].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationAddResponse); i { case 0: return &v.state @@ -14473,7 +15921,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[143].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[147].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationFixRequest); i { case 0: return &v.state @@ -14485,7 +15933,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[144].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[148].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationFixResponse); i { case 0: return &v.state @@ -14497,7 +15945,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[145].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[149].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationPositionsRequest); i { case 0: return &v.state @@ -14509,7 +15957,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[146].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[150].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationPositionsResponse); i { case 0: return &v.state @@ -14521,7 +15969,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[147].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[151].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationRemoveRequest); i { case 0: return &v.state @@ -14533,7 +15981,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[148].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[152].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationRemoveResponse); i { case 0: return &v.state @@ -14545,7 +15993,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[149].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[153].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SleepTabletRequest); i { case 0: return &v.state @@ -14557,7 +16005,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[150].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[154].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SleepTabletResponse); i { case 0: return &v.state @@ -14569,7 +16017,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[151].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[155].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardAddRequest); i { case 0: return &v.state @@ -14581,7 +16029,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[152].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[156].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardAddResponse); i { case 0: return &v.state @@ -14593,7 +16041,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[153].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[157].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardDeleteRequest); i { case 0: return &v.state @@ -14605,7 +16053,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[154].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[158].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardDeleteResponse); i { case 0: return &v.state @@ -14617,7 +16065,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[155].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[159].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StartReplicationRequest); i { case 0: return &v.state @@ -14629,7 +16077,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[156].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[160].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StartReplicationResponse); i { case 0: return &v.state @@ -14641,7 +16089,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[157].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[161].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StopReplicationRequest); i { case 0: return &v.state @@ -14653,7 +16101,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[158].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[162].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StopReplicationResponse); i { case 0: return &v.state @@ -14665,7 +16113,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[159].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[163].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletExternallyReparentedRequest); i { case 0: return &v.state @@ -14677,7 +16125,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[160].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[164].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletExternallyReparentedResponse); i { case 0: return &v.state @@ -14689,7 +16137,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[161].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[165].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellInfoRequest); i { case 0: return &v.state @@ -14701,7 +16149,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[162].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[166].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellInfoResponse); i { case 0: return &v.state @@ -14713,7 +16161,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[163].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[167].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellsAliasRequest); i { case 0: return &v.state @@ -14725,7 +16173,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[164].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[168].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellsAliasResponse); i { case 0: return &v.state @@ -14737,7 +16185,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[165].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[169].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateRequest); i { case 0: return &v.state @@ -14749,7 +16197,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[166].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[170].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateResponse); i { case 0: return &v.state @@ -14761,7 +16209,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[167].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[171].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateKeyspaceRequest); i { case 0: return &v.state @@ -14773,7 +16221,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[168].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[172].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateKeyspaceResponse); i { case 0: return &v.state @@ -14785,7 +16233,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[169].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[173].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateSchemaKeyspaceRequest); i { case 0: return &v.state @@ -14797,7 +16245,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[170].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[174].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateSchemaKeyspaceResponse); i { case 0: return &v.state @@ -14809,7 +16257,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[171].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[175].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateShardRequest); i { case 0: return &v.state @@ -14821,7 +16269,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[172].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[176].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateShardResponse); i { case 0: return &v.state @@ -14833,7 +16281,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[173].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[177].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionKeyspaceRequest); i { case 0: return &v.state @@ -14845,7 +16293,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[174].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[178].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionKeyspaceResponse); i { case 0: return &v.state @@ -14857,7 +16305,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[175].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[179].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionShardRequest); i { case 0: return &v.state @@ -14869,7 +16317,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[176].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[180].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionShardResponse); i { case 0: return &v.state @@ -14881,7 +16329,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[177].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[181].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVSchemaRequest); i { case 0: return &v.state @@ -14893,7 +16341,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[178].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[182].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVSchemaResponse); i { case 0: return &v.state @@ -14905,7 +16353,79 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[179].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[183].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowDeleteRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[184].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowDeleteResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[185].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowStatusRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[186].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowStatusResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[187].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowSwitchTrafficRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[188].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowSwitchTrafficResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[189].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateRequest); i { case 0: return &v.state @@ -14917,7 +16437,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[180].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[190].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateResponse); i { case 0: return &v.state @@ -14929,7 +16449,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[182].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[192].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_ReplicationLocation); i { case 0: return &v.state @@ -14941,7 +16461,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[183].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[193].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_ShardStream); i { case 0: return &v.state @@ -14953,7 +16473,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[184].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[194].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream); i { case 0: return &v.state @@ -14965,7 +16485,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[185].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[195].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream_CopyState); i { case 0: return &v.state @@ -14977,7 +16497,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[186].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[196].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream_Log); i { case 0: return &v.state @@ -14989,7 +16509,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[190].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[200].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspaceNamesResponse_NameList); i { case 0: return &v.state @@ -15001,7 +16521,67 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[200].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[203].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MoveTablesCreateResponse_TabletInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[211].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowDeleteResponse_TabletInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[212].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowStatusResponse_TableCopyState); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[213].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowStatusResponse_ShardStreamState); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[214].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowStatusResponse_ShardStreams); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[217].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateResponse_TabletInfo); i { case 0: return &v.state @@ -15020,7 +16600,7 @@ func file_vtctldata_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_vtctldata_proto_rawDesc, NumEnums: 1, - NumMessages: 201, + NumMessages: 218, 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 85fba4d8019..aba656454d6 100644 --- a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go +++ b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go @@ -204,6 +204,11 @@ func (m *MaterializeSettings) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) + i-- + dAtA[i] = 0x78 + } if m.DeferSecondaryKeys { i-- if m.DeferSecondaryKeys { @@ -5475,6 +5480,13 @@ func (m *GetWorkflowsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x22 + } if m.NameOnly { i-- if m.NameOnly { @@ -5672,7 +5684,7 @@ func (m *InitShardPrimaryResponse) MarshalToSizedBufferVT(dAtA []byte) (int, err return len(dAtA) - i, nil } -func (m *PingTabletRequest) MarshalVT() (dAtA []byte, err error) { +func (m *MoveTablesCreateRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -5685,12 +5697,12 @@ func (m *PingTabletRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *PingTabletRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *PingTabletRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -5702,130 +5714,168 @@ func (m *PingTabletRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.TabletAlias != nil { - size, err := m.TabletAlias.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if m.AutoStart { + i-- + if m.AutoStart { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0xa + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x88 } - return len(dAtA) - i, nil -} - -func (m *PingTabletResponse) MarshalVT() (dAtA []byte, err error) { - if m == nil { - return nil, nil + if m.DeferSecondaryKeys { + i-- + if m.DeferSecondaryKeys { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 } - size := m.SizeVT() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBufferVT(dAtA[:size]) - if err != nil { - return nil, err + if m.DropForeignKeys { + i-- + if m.DropForeignKeys { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x78 } - return dAtA[:n], nil -} - -func (m *PingTabletResponse) MarshalToVT(dAtA []byte) (int, error) { - size := m.SizeVT() - return m.MarshalToSizedBufferVT(dAtA[:size]) -} - -func (m *PingTabletResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { - if m == nil { - return 0, nil + if m.StopAfterCopy { + i-- + if m.StopAfterCopy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x70 } - i := len(dAtA) - _ = i - var l int - _ = l - if m.unknownFields != nil { - i -= len(m.unknownFields) - copy(dAtA[i:], m.unknownFields) + if len(m.OnDdl) > 0 { + i -= len(m.OnDdl) + copy(dAtA[i:], m.OnDdl) + i = encodeVarint(dAtA, i, uint64(len(m.OnDdl))) + i-- + dAtA[i] = 0x6a } - return len(dAtA) - i, nil -} - -func (m *PlannedReparentShardRequest) MarshalVT() (dAtA []byte, err error) { - if m == nil { - return nil, nil + if len(m.SourceTimeZone) > 0 { + i -= len(m.SourceTimeZone) + copy(dAtA[i:], m.SourceTimeZone) + i = encodeVarint(dAtA, i, uint64(len(m.SourceTimeZone))) + i-- + dAtA[i] = 0x62 } - size := m.SizeVT() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBufferVT(dAtA[:size]) - if err != nil { - return nil, err + if len(m.ExternalClusterName) > 0 { + i -= len(m.ExternalClusterName) + copy(dAtA[i:], m.ExternalClusterName) + i = encodeVarint(dAtA, i, uint64(len(m.ExternalClusterName))) + i-- + dAtA[i] = 0x5a } - return dAtA[:n], nil -} - -func (m *PlannedReparentShardRequest) MarshalToVT(dAtA []byte) (int, error) { - size := m.SizeVT() - return m.MarshalToSizedBufferVT(dAtA[:size]) -} - -func (m *PlannedReparentShardRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { - if m == nil { - return 0, nil + if len(m.ExcludeTables) > 0 { + for iNdEx := len(m.ExcludeTables) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ExcludeTables[iNdEx]) + copy(dAtA[i:], m.ExcludeTables[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.ExcludeTables[iNdEx]))) + i-- + dAtA[i] = 0x52 + } } - i := len(dAtA) - _ = i - var l int - _ = l - if m.unknownFields != nil { - i -= len(m.unknownFields) - copy(dAtA[i:], m.unknownFields) + if len(m.IncludeTables) > 0 { + for iNdEx := len(m.IncludeTables) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.IncludeTables[iNdEx]) + copy(dAtA[i:], m.IncludeTables[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.IncludeTables[iNdEx]))) + i-- + dAtA[i] = 0x4a + } } - if m.WaitReplicasTimeout != nil { - size, err := m.WaitReplicasTimeout.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if m.AllTables { + i-- + if m.AllTables { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x2a + dAtA[i] = 0x40 } - if m.AvoidPrimary != nil { - size, err := m.AvoidPrimary.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if len(m.SourceShards) > 0 { + for iNdEx := len(m.SourceShards) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.SourceShards[iNdEx]) + copy(dAtA[i:], m.SourceShards[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.SourceShards[iNdEx]))) + i-- + dAtA[i] = 0x3a } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) + } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) i-- - dAtA[i] = 0x22 + dAtA[i] = 0x30 } - if m.NewPrimary != nil { - size, err := m.NewPrimary.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if len(m.TabletTypes) > 0 { + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x2a + } + if len(m.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.TargetKeyspace) > 0 { + i -= len(m.TargetKeyspace) + copy(dAtA[i:], m.TargetKeyspace) + i = encodeVarint(dAtA, i, uint64(len(m.TargetKeyspace))) i-- dAtA[i] = 0x1a } - if len(m.Shard) > 0 { - i -= len(m.Shard) - copy(dAtA[i:], m.Shard) - i = encodeVarint(dAtA, i, uint64(len(m.Shard))) + if len(m.SourceKeyspace) > 0 { + i -= len(m.SourceKeyspace) + copy(dAtA[i:], m.SourceKeyspace) + i = encodeVarint(dAtA, i, uint64(len(m.SourceKeyspace))) i-- dAtA[i] = 0x12 } - if len(m.Keyspace) > 0 { - i -= len(m.Keyspace) - copy(dAtA[i:], m.Keyspace) - i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *PlannedReparentShardResponse) MarshalVT() (dAtA []byte, err error) { +func (m *MoveTablesCreateResponse_TabletInfo) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -5838,12 +5888,12 @@ func (m *PlannedReparentShardResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *PlannedReparentShardResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateResponse_TabletInfo) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *PlannedReparentShardResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateResponse_TabletInfo) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -5855,46 +5905,30 @@ func (m *PlannedReparentShardResponse) MarshalToSizedBufferVT(dAtA []byte) (int, i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Events) > 0 { - for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { - size, err := m.Events[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x22 + if m.Created { + i-- + if m.Created { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x10 } - if m.PromotedPrimary != nil { - size, err := m.PromotedPrimary.MarshalToSizedBufferVT(dAtA[:i]) + if m.Tablet != nil { + size, err := m.Tablet.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } i -= size i = encodeVarint(dAtA, i, uint64(size)) i-- - dAtA[i] = 0x1a - } - if len(m.Shard) > 0 { - i -= len(m.Shard) - copy(dAtA[i:], m.Shard) - i = encodeVarint(dAtA, i, uint64(len(m.Shard))) - i-- - dAtA[i] = 0x12 - } - if len(m.Keyspace) > 0 { - i -= len(m.Keyspace) - copy(dAtA[i:], m.Keyspace) - i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) - i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *RebuildKeyspaceGraphRequest) MarshalVT() (dAtA []byte, err error) { +func (m *MoveTablesCreateResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -5907,12 +5941,12 @@ func (m *RebuildKeyspaceGraphRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RebuildKeyspaceGraphRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *RebuildKeyspaceGraphRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *MoveTablesCreateResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -5924,36 +5958,29 @@ func (m *RebuildKeyspaceGraphRequest) MarshalToSizedBufferVT(dAtA []byte) (int, i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.AllowPartial { - i-- - if m.AllowPartial { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x18 - } - if len(m.Cells) > 0 { - for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Cells[iNdEx]) - copy(dAtA[i:], m.Cells[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + if len(m.Details) > 0 { + for iNdEx := len(m.Details) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Details[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) i-- dAtA[i] = 0x12 } } - if len(m.Keyspace) > 0 { - i -= len(m.Keyspace) - copy(dAtA[i:], m.Keyspace) - i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + if len(m.Summary) > 0 { + i -= len(m.Summary) + copy(dAtA[i:], m.Summary) + i = encodeVarint(dAtA, i, uint64(len(m.Summary))) i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *RebuildKeyspaceGraphResponse) MarshalVT() (dAtA []byte, err error) { +func (m *MoveTablesCompleteRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -5966,12 +5993,12 @@ func (m *RebuildKeyspaceGraphResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RebuildKeyspaceGraphResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *MoveTablesCompleteRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *RebuildKeyspaceGraphResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *MoveTablesCompleteRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -5983,52 +6010,64 @@ func (m *RebuildKeyspaceGraphResponse) MarshalToSizedBufferVT(dAtA []byte) (int, i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - return len(dAtA) - i, nil -} - -func (m *RebuildVSchemaGraphRequest) 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 + if m.DryRun { + i-- + if m.DryRun { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x38 } - return dAtA[:n], nil -} - -func (m *RebuildVSchemaGraphRequest) MarshalToVT(dAtA []byte) (int, error) { - size := m.SizeVT() - return m.MarshalToSizedBufferVT(dAtA[:size]) -} - -func (m *RebuildVSchemaGraphRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { - if m == nil { - return 0, nil + if m.RenameTables { + i-- + if m.RenameTables { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 } - i := len(dAtA) - _ = i - var l int - _ = l - if m.unknownFields != nil { - i -= len(m.unknownFields) - copy(dAtA[i:], m.unknownFields) + if m.KeepRoutingRules { + i-- + if m.KeepRoutingRules { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 } - if len(m.Cells) > 0 { - for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Cells[iNdEx]) - copy(dAtA[i:], m.Cells[iNdEx]) - i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) - i-- - dAtA[i] = 0xa + if m.KeepData { + i-- + if m.KeepData { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x20 + } + if len(m.TargetKeyspace) > 0 { + i -= len(m.TargetKeyspace) + copy(dAtA[i:], m.TargetKeyspace) + i = encodeVarint(dAtA, i, uint64(len(m.TargetKeyspace))) + i-- + dAtA[i] = 0x1a + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *RebuildVSchemaGraphResponse) MarshalVT() (dAtA []byte, err error) { +func (m *MoveTablesCompleteResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -6041,12 +6080,12 @@ func (m *RebuildVSchemaGraphResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RebuildVSchemaGraphResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *MoveTablesCompleteResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *RebuildVSchemaGraphResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *MoveTablesCompleteResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -6058,10 +6097,26 @@ func (m *RebuildVSchemaGraphResponse) MarshalToSizedBufferVT(dAtA []byte) (int, i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.DryRunResults) > 0 { + for iNdEx := len(m.DryRunResults) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DryRunResults[iNdEx]) + copy(dAtA[i:], m.DryRunResults[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.DryRunResults[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Summary) > 0 { + i -= len(m.Summary) + copy(dAtA[i:], m.Summary) + i = encodeVarint(dAtA, i, uint64(len(m.Summary))) + i-- + dAtA[i] = 0xa + } return len(dAtA) - i, nil } -func (m *RefreshStateRequest) MarshalVT() (dAtA []byte, err error) { +func (m *PingTabletRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -6074,12 +6129,401 @@ func (m *RefreshStateRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RefreshStateRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *PingTabletRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *RefreshStateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *PingTabletRequest) 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.TabletAlias != nil { + size, err := m.TabletAlias.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 *PingTabletResponse) 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 *PingTabletResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *PingTabletResponse) 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 *PlannedReparentShardRequest) 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 *PlannedReparentShardRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *PlannedReparentShardRequest) 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.WaitReplicasTimeout != nil { + size, err := m.WaitReplicasTimeout.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + if m.AvoidPrimary != nil { + size, err := m.AvoidPrimary.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + if m.NewPrimary != nil { + size, err := m.NewPrimary.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Shard) > 0 { + i -= len(m.Shard) + copy(dAtA[i:], m.Shard) + i = encodeVarint(dAtA, i, uint64(len(m.Shard))) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *PlannedReparentShardResponse) 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 *PlannedReparentShardResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *PlannedReparentShardResponse) 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.Events) > 0 { + for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Events[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } + } + if m.PromotedPrimary != nil { + size, err := m.PromotedPrimary.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Shard) > 0 { + i -= len(m.Shard) + copy(dAtA[i:], m.Shard) + i = encodeVarint(dAtA, i, uint64(len(m.Shard))) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RebuildKeyspaceGraphRequest) 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 *RebuildKeyspaceGraphRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RebuildKeyspaceGraphRequest) 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.AllowPartial { + i-- + if m.AllowPartial { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RebuildKeyspaceGraphResponse) 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 *RebuildKeyspaceGraphResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RebuildKeyspaceGraphResponse) 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 *RebuildVSchemaGraphRequest) 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 *RebuildVSchemaGraphRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RebuildVSchemaGraphRequest) 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.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *RebuildVSchemaGraphResponse) 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 *RebuildVSchemaGraphResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RebuildVSchemaGraphResponse) 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 *RefreshStateRequest) 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 *RefreshStateRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RefreshStateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -9733,7 +10177,7 @@ func (m *ValidateVSchemaResponse) MarshalToSizedBufferVT(dAtA []byte) (int, erro return len(dAtA) - i, nil } -func (m *WorkflowUpdateRequest) MarshalVT() (dAtA []byte, err error) { +func (m *WorkflowDeleteRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -9746,12 +10190,12 @@ func (m *WorkflowUpdateRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *WorkflowUpdateRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *WorkflowUpdateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -9763,13 +10207,30 @@ func (m *WorkflowUpdateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.TabletRequest != nil { - size, err := m.TabletRequest.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if m.KeepRoutingRules { + i-- + if m.KeepRoutingRules { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x20 + } + if m.KeepData { + i-- + if m.KeepData { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- dAtA[i] = 0x12 } @@ -9783,7 +10244,7 @@ func (m *WorkflowUpdateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) return len(dAtA) - i, nil } -func (m *WorkflowUpdateResponse_TabletInfo) MarshalVT() (dAtA []byte, err error) { +func (m *WorkflowDeleteResponse_TabletInfo) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -9796,12 +10257,12 @@ func (m *WorkflowUpdateResponse_TabletInfo) MarshalVT() (dAtA []byte, err error) return dAtA[:n], nil } -func (m *WorkflowUpdateResponse_TabletInfo) MarshalToVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteResponse_TabletInfo) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *WorkflowUpdateResponse_TabletInfo) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteResponse_TabletInfo) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -9813,9 +10274,9 @@ func (m *WorkflowUpdateResponse_TabletInfo) MarshalToSizedBufferVT(dAtA []byte) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.Changed { + if m.Deleted { i-- - if m.Changed { + if m.Deleted { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -9823,17 +10284,20 @@ func (m *WorkflowUpdateResponse_TabletInfo) MarshalToSizedBufferVT(dAtA []byte) i-- dAtA[i] = 0x10 } - if len(m.Tablet) > 0 { - i -= len(m.Tablet) - copy(dAtA[i:], m.Tablet) - i = encodeVarint(dAtA, i, uint64(len(m.Tablet))) + if m.Tablet != nil { + size, err := m.Tablet.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 *WorkflowUpdateResponse) MarshalVT() (dAtA []byte, err error) { +func (m *WorkflowDeleteResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -9846,12 +10310,12 @@ func (m *WorkflowUpdateResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *WorkflowUpdateResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *WorkflowUpdateResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *WorkflowDeleteResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -9885,282 +10349,935 @@ func (m *WorkflowUpdateResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error return len(dAtA) - i, nil } -func encodeVarint(dAtA []byte, offset int, v uint64) int { - offset -= sov(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *WorkflowStatusRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil } - dAtA[offset] = uint8(v) - return base + 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 *ExecuteVtctlCommandRequest) SizeVT() (n int) { + +func (m *WorkflowStatusRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowStatusRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if len(m.Args) > 0 { - for _, s := range m.Args { - l = len(s) - n += 1 + l + sov(uint64(l)) - } + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if m.ActionTimeout != 0 { - n += 1 + sov(uint64(m.ActionTimeout)) + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x12 } - n += len(m.unknownFields) - return n + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } -func (m *ExecuteVtctlCommandResponse) SizeVT() (n int) { +func (m *WorkflowStatusResponse_TableCopyState) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 + return nil, nil } - var l int - _ = l - if m.Event != nil { - l = m.Event.SizeVT() - n += 1 + l + sov(uint64(l)) + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *TableMaterializeSettings) SizeVT() (n int) { +func (m *WorkflowStatusResponse_TableCopyState) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowStatusResponse_TableCopyState) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - l = len(m.TargetTable) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - l = len(m.SourceExpression) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.BytesPercentage != 0 { + i -= 4 + binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.BytesPercentage)))) + i-- + dAtA[i] = 0x35 } - l = len(m.CreateDdl) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.BytesTotal != 0 { + i = encodeVarint(dAtA, i, uint64(m.BytesTotal)) + i-- + dAtA[i] = 0x28 } - n += len(m.unknownFields) - return n -} - -func (m *MaterializeSettings) SizeVT() (n int) { - if m == nil { - return 0 + if m.BytesCopied != 0 { + i = encodeVarint(dAtA, i, uint64(m.BytesCopied)) + i-- + dAtA[i] = 0x20 } - var l int - _ = l - l = len(m.Workflow) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.RowsPercentage != 0 { + i -= 4 + binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.RowsPercentage)))) + i-- + dAtA[i] = 0x1d } - l = len(m.SourceKeyspace) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.RowsTotal != 0 { + i = encodeVarint(dAtA, i, uint64(m.RowsTotal)) + i-- + dAtA[i] = 0x10 } - l = len(m.TargetKeyspace) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.RowsCopied != 0 { + i = encodeVarint(dAtA, i, uint64(m.RowsCopied)) + i-- + dAtA[i] = 0x8 } - if m.StopAfterCopy { - n += 2 + return len(dAtA) - i, nil +} + +func (m *WorkflowStatusResponse_ShardStreamState) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil } - if len(m.TableSettings) > 0 { - for _, e := range m.TableSettings { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) - } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - l = len(m.Cell) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return dAtA[:n], nil +} + +func (m *WorkflowStatusResponse_ShardStreamState) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowStatusResponse_ShardStreamState) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil } - l = len(m.TabletTypes) - if l > 0 { - n += 1 + l + sov(uint64(l)) + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - l = len(m.ExternalCluster) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.Info) > 0 { + i -= len(m.Info) + copy(dAtA[i:], m.Info) + i = encodeVarint(dAtA, i, uint64(len(m.Info))) + i-- + dAtA[i] = 0x32 } - if m.MaterializationIntent != 0 { - n += 1 + sov(uint64(m.MaterializationIntent)) + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = encodeVarint(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0x2a } - l = len(m.SourceTimeZone) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.Position) > 0 { + i -= len(m.Position) + copy(dAtA[i:], m.Position) + i = encodeVarint(dAtA, i, uint64(len(m.Position))) + i-- + dAtA[i] = 0x22 } - l = len(m.TargetTimeZone) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.SourceShard) > 0 { + i -= len(m.SourceShard) + copy(dAtA[i:], m.SourceShard) + i = encodeVarint(dAtA, i, uint64(len(m.SourceShard))) + i-- + dAtA[i] = 0x1a } - if len(m.SourceShards) > 0 { - for _, s := range m.SourceShards { - l = len(s) - n += 1 + l + sov(uint64(l)) + if m.Tablet != nil { + size, err := m.Tablet.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 } - l = len(m.OnDdl) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - if m.DeferSecondaryKeys { - n += 2 + if m.Id != 0 { + i = encodeVarint(dAtA, i, uint64(m.Id)) + i-- + dAtA[i] = 0x8 } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *Keyspace) SizeVT() (n int) { +func (m *WorkflowStatusResponse_ShardStreams) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return nil, nil } - if m.Keyspace != nil { - l = m.Keyspace.SizeVT() - n += 1 + l + sov(uint64(l)) + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *Shard) SizeVT() (n int) { +func (m *WorkflowStatusResponse_ShardStreams) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowStatusResponse_ShardStreams) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Keyspace) - if l > 0 { - n += 1 + l + sov(uint64(l)) - } - l = len(m.Name) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if m.Shard != nil { - l = m.Shard.SizeVT() - n += 1 + l + sov(uint64(l)) + if len(m.Streams) > 0 { + for iNdEx := len(m.Streams) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Streams[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *Workflow_ReplicationLocation) SizeVT() (n int) { +func (m *WorkflowStatusResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Keyspace) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return nil, nil } - if len(m.Shards) > 0 { - for _, s := range m.Shards { - l = len(s) - n += 1 + l + sov(uint64(l)) - } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *Workflow_ShardStream) SizeVT() (n int) { +func (m *WorkflowStatusResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowStatusResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if len(m.Streams) > 0 { - for _, e := range m.Streams { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) - } + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if len(m.TabletControls) > 0 { - for _, e := range m.TabletControls { - l = e.SizeVT() - n += 1 + l + sov(uint64(l)) + if len(m.ShardStreams) > 0 { + for k := range m.ShardStreams { + v := m.ShardStreams[k] + baseI := i + size, err := v.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + i -= len(k) + copy(dAtA[i:], k) + i = encodeVarint(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = encodeVarint(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x12 } } - if m.IsPrimaryServing { - n += 2 + if len(m.TableCopyState) > 0 { + for k := range m.TableCopyState { + v := m.TableCopyState[k] + baseI := i + size, err := v.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + i -= len(k) + copy(dAtA[i:], k) + i = encodeVarint(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = encodeVarint(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0xa + } } - n += len(m.unknownFields) - return n + return len(dAtA) - i, nil } -func (m *Workflow_Stream_CopyState) SizeVT() (n int) { +func (m *WorkflowSwitchTrafficRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Table) - if l > 0 { - n += 1 + l + sov(uint64(l)) + return nil, nil } - l = len(m.LastPk) - if l > 0 { - n += 1 + l + sov(uint64(l)) + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - n += len(m.unknownFields) - return n + return dAtA[:n], nil } -func (m *Workflow_Stream_Log) SizeVT() (n int) { +func (m *WorkflowSwitchTrafficRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowSwitchTrafficRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { - return 0 + return 0, nil } + i := len(dAtA) + _ = i var l int _ = l - if m.Id != 0 { - n += 1 + sov(uint64(m.Id)) + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } - if m.StreamId != 0 { - n += 1 + sov(uint64(m.StreamId)) + if m.DryRun { + i-- + if m.DryRun { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x48 } - l = len(m.Type) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.Timeout != nil { + size, err := m.Timeout.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 } - l = len(m.State) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if m.Direction != 0 { + i = encodeVarint(dAtA, i, uint64(m.Direction)) + i-- + dAtA[i] = 0x38 } - if m.CreatedAt != nil { - l = m.CreatedAt.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.EnableReverseReplication { + i-- + if m.EnableReverseReplication { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 } - if m.UpdatedAt != nil { - l = m.UpdatedAt.SizeVT() - n += 1 + l + sov(uint64(l)) + if m.MaxReplicationLagAllowed != nil { + size, err := m.MaxReplicationLagAllowed.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a } - l = len(m.Message) - if l > 0 { - n += 1 + l + sov(uint64(l)) + if len(m.TabletTypes) > 0 { + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x22 } - if m.Count != 0 { - n += 1 + sov(uint64(m.Count)) + if len(m.Cells) > 0 { + for iNdEx := len(m.Cells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Cells[iNdEx]) + copy(dAtA[i:], m.Cells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.Cells[iNdEx]))) + i-- + dAtA[i] = 0x1a + } } - n += len(m.unknownFields) - return n + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } -func (m *Workflow_Stream) SizeVT() (n int) { +func (m *WorkflowSwitchTrafficResponse) 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 *WorkflowSwitchTrafficResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowSwitchTrafficResponse) 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.DryRunResults) > 0 { + for iNdEx := len(m.DryRunResults) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DryRunResults[iNdEx]) + copy(dAtA[i:], m.DryRunResults[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.DryRunResults[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.CurrentState) > 0 { + i -= len(m.CurrentState) + copy(dAtA[i:], m.CurrentState) + i = encodeVarint(dAtA, i, uint64(len(m.CurrentState))) + i-- + dAtA[i] = 0x1a + } + if len(m.StartState) > 0 { + i -= len(m.StartState) + copy(dAtA[i:], m.StartState) + i = encodeVarint(dAtA, i, uint64(len(m.StartState))) + i-- + dAtA[i] = 0x12 + } + if len(m.Summary) > 0 { + i -= len(m.Summary) + copy(dAtA[i:], m.Summary) + i = encodeVarint(dAtA, i, uint64(len(m.Summary))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *WorkflowUpdateRequest) 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 *WorkflowUpdateRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowUpdateRequest) 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.TabletRequest != nil { + size, err := m.TabletRequest.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *WorkflowUpdateResponse_TabletInfo) 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 *WorkflowUpdateResponse_TabletInfo) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowUpdateResponse_TabletInfo) 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.Changed { + i-- + if m.Changed { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if m.Tablet != nil { + size, err := m.Tablet.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 *WorkflowUpdateResponse) 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 *WorkflowUpdateResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowUpdateResponse) 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.Details) > 0 { + for iNdEx := len(m.Details) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Details[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Summary) > 0 { + i -= len(m.Summary) + copy(dAtA[i:], m.Summary) + i = encodeVarint(dAtA, i, uint64(len(m.Summary))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarint(dAtA []byte, offset int, v uint64) int { + offset -= sov(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *ExecuteVtctlCommandRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Args) > 0 { + for _, s := range m.Args { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if m.ActionTimeout != 0 { + n += 1 + sov(uint64(m.ActionTimeout)) + } + n += len(m.unknownFields) + return n +} + +func (m *ExecuteVtctlCommandResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Event != nil { + l = m.Event.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *TableMaterializeSettings) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TargetTable) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceExpression) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.CreateDdl) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *MaterializeSettings) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.TargetKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.StopAfterCopy { + n += 2 + } + if len(m.TableSettings) > 0 { + for _, e := range m.TableSettings { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + l = len(m.Cell) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.TabletTypes) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.ExternalCluster) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.MaterializationIntent != 0 { + n += 1 + sov(uint64(m.MaterializationIntent)) + } + l = len(m.SourceTimeZone) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.TargetTimeZone) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.SourceShards) > 0 { + for _, s := range m.SourceShards { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + l = len(m.OnDdl) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.DeferSecondaryKeys { + n += 2 + } + if m.TabletSelectionPreference != 0 { + n += 1 + sov(uint64(m.TabletSelectionPreference)) + } + n += len(m.unknownFields) + return n +} + +func (m *Keyspace) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.Keyspace != nil { + l = m.Keyspace.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Shard) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.Shard != nil { + l = m.Shard.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Workflow_ReplicationLocation) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Shards) > 0 { + for _, s := range m.Shards { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *Workflow_ShardStream) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Streams) > 0 { + for _, e := range m.Streams { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + if len(m.TabletControls) > 0 { + for _, e := range m.TabletControls { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + if m.IsPrimaryServing { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *Workflow_Stream_CopyState) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Table) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.LastPk) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *Workflow_Stream_Log) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Id != 0 { + n += 1 + sov(uint64(m.Id)) + } + if m.StreamId != 0 { + n += 1 + sov(uint64(m.StreamId)) + } + l = len(m.Type) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.State) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.CreatedAt != nil { + l = m.CreatedAt.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.UpdatedAt != nil { + l = m.UpdatedAt.SizeVT() + n += 1 + l + sov(uint64(l)) + } + l = len(m.Message) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.Count != 0 { + n += 1 + sov(uint64(m.Count)) + } + n += len(m.unknownFields) + return n +} + +func (m *Workflow_Stream) SizeVT() (n int) { if m == nil { return 0 } @@ -11914,6 +13031,10 @@ func (m *GetWorkflowsRequest) SizeVT() (n int) { if m.NameOnly { n += 2 } + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -11979,6 +13100,176 @@ func (m *InitShardPrimaryResponse) SizeVT() (n int) { return n } +func (m *MoveTablesCreateRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.TargetKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Cells) > 0 { + for _, s := range m.Cells { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.TabletTypes) > 0 { + l = 0 + for _, e := range m.TabletTypes { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if m.TabletSelectionPreference != 0 { + n += 1 + sov(uint64(m.TabletSelectionPreference)) + } + if len(m.SourceShards) > 0 { + for _, s := range m.SourceShards { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if m.AllTables { + n += 2 + } + if len(m.IncludeTables) > 0 { + for _, s := range m.IncludeTables { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.ExcludeTables) > 0 { + for _, s := range m.ExcludeTables { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + l = len(m.ExternalClusterName) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceTimeZone) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.OnDdl) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.StopAfterCopy { + n += 2 + } + if m.DropForeignKeys { + n += 2 + } + if m.DeferSecondaryKeys { + n += 3 + } + if m.AutoStart { + n += 3 + } + n += len(m.unknownFields) + return n +} + +func (m *MoveTablesCreateResponse_TabletInfo) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Tablet != nil { + l = m.Tablet.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.Created { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *MoveTablesCreateResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Summary) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Details) > 0 { + for _, e := range m.Details { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *MoveTablesCompleteRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.TargetKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.KeepData { + n += 2 + } + if m.KeepRoutingRules { + n += 2 + } + if m.RenameTables { + n += 2 + } + if m.DryRun { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *MoveTablesCompleteResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Summary) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.DryRunResults) > 0 { + for _, s := range m.DryRunResults { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + func (m *PingTabletRequest) SizeVT() (n int) { if m == nil { return 0 @@ -13482,6 +14773,274 @@ func (m *ValidateVSchemaResponse) SizeVT() (n int) { return n } +func (m *WorkflowDeleteRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.KeepData { + n += 2 + } + if m.KeepRoutingRules { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowDeleteResponse_TabletInfo) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Tablet != nil { + l = m.Tablet.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.Deleted { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowDeleteResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Summary) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Details) > 0 { + for _, e := range m.Details { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowStatusRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowStatusResponse_TableCopyState) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RowsCopied != 0 { + n += 1 + sov(uint64(m.RowsCopied)) + } + if m.RowsTotal != 0 { + n += 1 + sov(uint64(m.RowsTotal)) + } + if m.RowsPercentage != 0 { + n += 5 + } + if m.BytesCopied != 0 { + n += 1 + sov(uint64(m.BytesCopied)) + } + if m.BytesTotal != 0 { + n += 1 + sov(uint64(m.BytesTotal)) + } + if m.BytesPercentage != 0 { + n += 5 + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowStatusResponse_ShardStreamState) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Id != 0 { + n += 1 + sov(uint64(m.Id)) + } + if m.Tablet != nil { + l = m.Tablet.SizeVT() + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceShard) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Position) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Status) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Info) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowStatusResponse_ShardStreams) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Streams) > 0 { + for _, e := range m.Streams { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowStatusResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.TableCopyState) > 0 { + for k, v := range m.TableCopyState { + _ = k + _ = v + l = 0 + if v != nil { + l = v.SizeVT() + } + l += 1 + sov(uint64(l)) + mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + l + n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + } + } + if len(m.ShardStreams) > 0 { + for k, v := range m.ShardStreams { + _ = k + _ = v + l = 0 + if v != nil { + l = v.SizeVT() + } + l += 1 + sov(uint64(l)) + mapEntrySize := 1 + len(k) + sov(uint64(len(k))) + l + n += mapEntrySize + 1 + sov(uint64(mapEntrySize)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowSwitchTrafficRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Keyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.Workflow) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Cells) > 0 { + for _, s := range m.Cells { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.TabletTypes) > 0 { + l = 0 + for _, e := range m.TabletTypes { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if m.MaxReplicationLagAllowed != nil { + l = m.MaxReplicationLagAllowed.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.EnableReverseReplication { + n += 2 + } + if m.Direction != 0 { + n += 1 + sov(uint64(m.Direction)) + } + if m.Timeout != nil { + l = m.Timeout.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.DryRun { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *WorkflowSwitchTrafficResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Summary) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.StartState) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.CurrentState) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.DryRunResults) > 0 { + for _, s := range m.DryRunResults { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + func (m *WorkflowUpdateRequest) SizeVT() (n int) { if m == nil { return 0 @@ -13506,8 +15065,8 @@ func (m *WorkflowUpdateResponse_TabletInfo) SizeVT() (n int) { } var l int _ = l - l = len(m.Tablet) - if l > 0 { + if m.Tablet != nil { + l = m.Tablet.SizeVT() n += 1 + l + sov(uint64(l)) } if m.Changed { @@ -13732,7 +15291,3173 @@ func (m *ExecuteVtctlCommandResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *TableMaterializeSettings) UnmarshalVT(dAtA []byte) error { +func (m *TableMaterializeSettings) 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: TableMaterializeSettings: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TableMaterializeSettings: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetTable", 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.TargetTable = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceExpression", 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.SourceExpression = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CreateDdl", 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.CreateDdl = 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 *MaterializeSettings) 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: MaterializeSettings: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MaterializeSettings: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", 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.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", 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.SourceKeyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetKeyspace", 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.TargetKeyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StopAfterCopy", 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.StopAfterCopy = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TableSettings", 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.TableSettings = append(m.TableSettings, &TableMaterializeSettings{}) + if err := m.TableSettings[len(m.TableSettings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cell", 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.Cell = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", 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.TabletTypes = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExternalCluster", 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.ExternalCluster = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaterializationIntent", wireType) + } + m.MaterializationIntent = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaterializationIntent |= MaterializationIntent(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceTimeZone", 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.SourceTimeZone = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetTimeZone", 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.TargetTimeZone = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceShards", 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.SourceShards = append(m.SourceShards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", 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.OnDdl = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", 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.DeferSecondaryKeys = bool(v != 0) + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) + } + m.TabletSelectionPreference = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletSelectionPreference |= tabletmanagerdata.TabletSelectionPreference(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 { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Keyspace) 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: Keyspace: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Keyspace: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", 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.Keyspace == nil { + m.Keyspace = &topodata.Keyspace{} + } + if err := m.Keyspace.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 *Shard) 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: Shard: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Shard: 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard == nil { + m.Shard = &topodata.Shard{} + } + if err := m.Shard.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 *Workflow_ReplicationLocation) 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: Workflow_ReplicationLocation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow_ReplicationLocation: 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", 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.Shards = append(m.Shards, 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_ShardStream) 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: Workflow_ShardStream: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow_ShardStream: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Streams", 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.Streams = append(m.Streams, &Workflow_Stream{}) + if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletControls", 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.TabletControls = append(m.TabletControls, &topodata.Shard_TabletControl{}) + if err := m.TabletControls[len(m.TabletControls)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsPrimaryServing", 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.IsPrimaryServing = bool(v != 0) + 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_Stream_CopyState) 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: Workflow_Stream_CopyState: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow_Stream_CopyState: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Table", 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.Table = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LastPk", 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.LastPk = 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_Stream_Log) 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: Workflow_Stream_Log: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow_Stream_Log: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + m.Id = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Id |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamId", wireType) + } + m.StreamId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StreamId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", 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.Type = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", 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.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CreatedAt", 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.CreatedAt == nil { + m.CreatedAt = &vttime.Time{} + } + if err := m.CreatedAt.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UpdatedAt", 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.UpdatedAt == nil { + m.UpdatedAt = &vttime.Time{} + } + if err := m.UpdatedAt.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", 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.Message = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + } + m.Count = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Count |= int64(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 { + 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_Stream) 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: Workflow_Stream: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow_Stream: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + m.Id = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Id |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", 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.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} + } + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BinlogSource", 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.BinlogSource == nil { + m.BinlogSource = &binlogdata.BinlogSource{} + } + if err := m.BinlogSource.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Position", 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.Position = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StopPosition", 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.StopPosition = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", 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.State = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DbName", 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.DbName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionTimestamp", 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.TransactionTimestamp == nil { + m.TransactionTimestamp = &vttime.Time{} + } + if err := m.TransactionTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeUpdated", 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.TimeUpdated == nil { + m.TimeUpdated = &vttime.Time{} + } + if err := m.TimeUpdated.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", 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.Message = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CopyStates", 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.CopyStates = append(m.CopyStates, &Workflow_Stream_CopyState{}) + if err := m.CopyStates[len(m.CopyStates)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Logs", 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.Logs = append(m.Logs, &Workflow_Stream_Log{}) + if err := m.Logs[len(m.Logs)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogFetchError", 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.LogFetchError = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tags", 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.Tags = append(m.Tags, 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) 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: Workflow: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Workflow: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", 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.Source == nil { + m.Source = &Workflow_ReplicationLocation{} + } + if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &Workflow_ReplicationLocation{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxVReplicationLag", wireType) + } + m.MaxVReplicationLag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxVReplicationLag |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardStreams", 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.ShardStreams == nil { + m.ShardStreams = make(map[string]*Workflow_ShardStream) + } + var mapkey string + var mapvalue *Workflow_ShardStream + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &Workflow_ShardStream{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.ShardStreams[mapkey] = mapvalue + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", 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.WorkflowType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", 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.WorkflowSubType = 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 *AddCellInfoRequest) 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: AddCellInfoRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", 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.CellInfo == nil { + m.CellInfo = &topodata.CellInfo{} + } + if err := m.CellInfo.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 *AddCellInfoResponse) 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: AddCellInfoResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddCellInfoResponse: 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 *AddCellsAliasRequest) 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: AddCellsAliasRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddCellsAliasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", 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.Cells = append(m.Cells, 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 *AddCellsAliasResponse) 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: AddCellsAliasResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AddCellsAliasResponse: 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 { + 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.RoutingRules == nil { + m.RoutingRules = &vschema.RoutingRules{} + } + if err := m.RoutingRules.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 *ApplyRoutingRulesResponse) 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: ApplyRoutingRulesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyRoutingRulesResponse: 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 *ApplyShardRoutingRulesRequest) 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: ApplyShardRoutingRulesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyShardRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardRoutingRules", 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.ShardRoutingRules == nil { + m.ShardRoutingRules = &vschema.ShardRoutingRules{} + } + if err := m.ShardRoutingRules.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 *ApplyShardRoutingRulesResponse) 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: ApplyShardRoutingRulesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyShardRoutingRulesResponse: 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 *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -13755,15 +18480,15 @@ func (m *TableMaterializeSettings) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TableMaterializeSettings: wiretype end group for non-group") + return fmt.Errorf("proto: ApplySchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TableMaterializeSettings: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplySchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TargetTable", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -13791,11 +18516,11 @@ func (m *TableMaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TargetTable = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceExpression", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sql", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -13823,11 +18548,11 @@ func (m *TableMaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SourceExpression = string(dAtA[iNdEx:postIndex]) + m.Sql = append(m.Sql, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 3: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CreateDdl", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DdlStrategy", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -13855,62 +18580,11 @@ func (m *TableMaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CreateDdl = string(dAtA[iNdEx:postIndex]) + m.DdlStrategy = 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 *MaterializeSettings) 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: MaterializeSettings: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: MaterializeSettings: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UuidList", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -13938,11 +18612,11 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Workflow = string(dAtA[iNdEx:postIndex]) + m.UuidList = append(m.UuidList, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 2: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MigrationContext", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -13970,13 +18644,13 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SourceKeyspace = string(dAtA[iNdEx:postIndex]) + m.MigrationContext = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TargetKeyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -13986,27 +18660,31 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.TargetKeyspace = string(dAtA[iNdEx:postIndex]) + if m.WaitReplicasTimeout == nil { + m.WaitReplicasTimeout = &vttime.Duration{} + } + if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 4: + case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StopAfterCopy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SkipPreflight", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -14023,10 +18701,10 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { break } } - m.StopAfterCopy = bool(v != 0) - case 5: + m.SkipPreflight = bool(v != 0) + case 9: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TableSettings", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CallerId", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -14053,46 +18731,67 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TableSettings = append(m.TableSettings, &TableMaterializeSettings{}) - if err := m.TableSettings[len(m.TableSettings)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.CallerId == nil { + m.CallerId = &vtrpc.CallerID{} + } + if err := m.CallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", 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 - } + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - if postIndex > l { + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ApplySchemaResponse) 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 } - m.Cell = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 7: + 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: ApplySchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplySchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UuidList", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14120,11 +18819,62 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TabletTypes = string(dAtA[iNdEx:postIndex]) + m.UuidList = append(m.UuidList, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 8: + 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 *ApplyVSchemaRequest) 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: ApplyVSchemaRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExternalCluster", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14152,13 +18902,13 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ExternalCluster = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 9: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaterializationIntent", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SkipRebuild", wireType) } - m.MaterializationIntent = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14168,16 +18918,17 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaterializationIntent |= MaterializationIntent(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - case 10: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceTimeZone", wireType) + m.SkipRebuild = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14187,27 +18938,15 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.SourceTimeZone = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 11: + m.DryRun = bool(v != 0) + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TargetTimeZone", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14235,13 +18974,13 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TargetTimeZone = string(dAtA[iNdEx:postIndex]) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 12: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceShards", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VSchema", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14251,27 +18990,31 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.SourceShards = append(m.SourceShards, string(dAtA[iNdEx:postIndex])) + if m.VSchema == nil { + m.VSchema = &vschema.Keyspace{} + } + if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 13: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sql", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14299,28 +19042,8 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.OnDdl = string(dAtA[iNdEx:postIndex]) + m.Sql = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 14: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", 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.DeferSecondaryKeys = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14343,7 +19066,7 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Keyspace) UnmarshalVT(dAtA []byte) error { +func (m *ApplyVSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14366,47 +19089,15 @@ func (m *Keyspace) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Keyspace: wiretype end group for non-group") + return fmt.Errorf("proto: ApplyVSchemaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Keyspace: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplyVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VSchema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -14433,10 +19124,10 @@ func (m *Keyspace) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Keyspace == nil { - m.Keyspace = &topodata.Keyspace{} + if m.VSchema == nil { + m.VSchema = &vschema.Keyspace{} } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -14462,7 +19153,7 @@ func (m *Keyspace) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Shard) UnmarshalVT(dAtA []byte) error { +func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14485,17 +19176,17 @@ func (m *Shard) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Shard: wiretype end group for non-group") + return fmt.Errorf("proto: BackupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Shard: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BackupRequest: 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 TabletAlias", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14505,29 +19196,33 @@ func (m *Shard) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} + } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14537,29 +19232,17 @@ func (m *Shard) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.AllowPrimary = bool(v != 0) case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) } - var msglen int + m.Concurrency = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14569,82 +19252,14 @@ func (m *Shard) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Concurrency |= uint64(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.Shard == nil { - m.Shard = &topodata.Shard{} - } - if err := m.Shard.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 *Workflow_ReplicationLocation) 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: Workflow_ReplicationLocation: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow_ReplicationLocation: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncrementalFromPos", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14672,13 +19287,13 @@ func (m *Workflow_ReplicationLocation) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.IncrementalFromPos = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14688,24 +19303,12 @@ func (m *Workflow_ReplicationLocation) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex + m.UpgradeSafe = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -14728,7 +19331,7 @@ func (m *Workflow_ReplicationLocation) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Workflow_ShardStream) UnmarshalVT(dAtA []byte) error { +func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -14751,15 +19354,15 @@ func (m *Workflow_ShardStream) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Workflow_ShardStream: wiretype end group for non-group") + return fmt.Errorf("proto: BackupResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow_ShardStream: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -14786,16 +19389,18 @@ func (m *Workflow_ShardStream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Streams = append(m.Streams, &Workflow_Stream{}) - if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} + } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletControls", 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 @@ -14805,100 +19410,27 @@ func (m *Workflow_ShardStream) 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 } - m.TabletControls = append(m.TabletControls, &topodata.Shard_TabletControl{}) - if err := m.TabletControls[len(m.TabletControls)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsPrimaryServing", 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.IsPrimaryServing = bool(v != 0) - 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_Stream_CopyState) 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: Workflow_Stream_CopyState: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow_Stream_CopyState: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Table", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -14926,13 +19458,13 @@ func (m *Workflow_Stream_CopyState) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Table = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LastPk", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -14942,23 +19474,27 @@ func (m *Workflow_Stream_CopyState) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.LastPk = string(dAtA[iNdEx:postIndex]) + if m.Event == nil { + m.Event = &logutil.Event{} + } + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -14982,7 +19518,7 @@ func (m *Workflow_Stream_CopyState) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { +func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -15005,17 +19541,17 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Workflow_Stream_Log: wiretype end group for non-group") + return fmt.Errorf("proto: BackupShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow_Stream_Log: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BackupShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } - m.Id = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15025,33 +19561,27 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Id |= int64(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StreamId", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.StreamId = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.StreamId |= int64(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - case 3: + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Keyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -15079,13 +19609,13 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Type = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15095,29 +19625,36 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + m.AllowPrimary = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) } - if postIndex > l { - return io.ErrUnexpectedEOF + m.Concurrency = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Concurrency |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } } - m.State = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CreatedAt", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15127,31 +19664,66 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLength + m.UpgradeSafe = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.CreatedAt == nil { - m.CreatedAt = &vttime.Time{} + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChangeTabletTypeRequest) 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 err := m.CreatedAt.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + if iNdEx >= l { + return io.ErrUnexpectedEOF } - iNdEx = postIndex - case 6: + 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: ChangeTabletTypeRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChangeTabletTypeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UpdatedAt", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15178,18 +19750,18 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.UpdatedAt == nil { - m.UpdatedAt = &vttime.Time{} + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - if err := m.UpdatedAt.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DbType", wireType) } - var stringLen uint64 + m.DbType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15199,29 +19771,16 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.DbType |= topodata.TabletType(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.Message = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 8: + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) } - m.Count = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15231,11 +19790,12 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Count |= int64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } + m.DryRun = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -15258,7 +19818,7 @@ func (m *Workflow_Stream_Log) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { +func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -15281,66 +19841,15 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Workflow_Stream: wiretype end group for non-group") + return fmt.Errorf("proto: ChangeTabletTypeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow_Stream: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ChangeTabletTypeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) - } - m.Id = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Id |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field BeforeTablet", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15367,16 +19876,16 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Tablet == nil { - m.Tablet = &topodata.TabletAlias{} + if m.BeforeTablet == nil { + m.BeforeTablet = &topodata.Tablet{} } - if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.BeforeTablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 4: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BinlogSource", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AfterTablet", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15403,18 +19912,18 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.BinlogSource == nil { - m.BinlogSource = &binlogdata.BinlogSource{} + if m.AfterTablet == nil { + m.AfterTablet = &topodata.Tablet{} } - if err := m.BinlogSource.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.AfterTablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WasDryRun", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15424,27 +19933,66 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + m.WasDryRun = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Position = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 6: + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CreateKeyspaceRequest) 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: CreateKeyspaceRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StopPosition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -15472,13 +20020,13 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.StopPosition = string(dAtA[iNdEx:postIndex]) + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15488,29 +20036,17 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.State = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) + m.Force = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowEmptyVSchema", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15520,27 +20056,15 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.DbName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 9: + m.AllowEmptyVSchema = bool(v != 0) + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TransactionTimestamp", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ServedFroms", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15567,18 +20091,16 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TransactionTimestamp == nil { - m.TransactionTimestamp = &vttime.Time{} - } - if err := m.TransactionTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.ServedFroms = append(m.ServedFroms, &topodata.Keyspace_ServedFrom{}) + if err := m.ServedFroms[len(m.ServedFroms)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 10: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeUpdated", wireType) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) } - var msglen int + m.Type = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15588,31 +20110,14 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Type |= topodata.KeyspaceType(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.TimeUpdated == nil { - m.TimeUpdated = &vttime.Time{} - } - if err := m.TimeUpdated.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 11: + case 8: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field BaseKeyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -15640,11 +20145,11 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Message = string(dAtA[iNdEx:postIndex]) + m.BaseKeyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 12: + case 9: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CopyStates", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SnapshotTime", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15671,16 +20176,18 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CopyStates = append(m.CopyStates, &Workflow_Stream_CopyState{}) - if err := m.CopyStates[len(m.CopyStates)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.SnapshotTime == nil { + m.SnapshotTime = &vttime.Time{} + } + if err := m.SnapshotTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 13: + case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Logs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DurabilityPolicy", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15690,29 +20197,27 @@ func (m *Workflow_Stream) 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 } - m.Logs = append(m.Logs, &Workflow_Stream_Log{}) - if err := m.Logs[len(m.Logs)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.DurabilityPolicy = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 14: + case 11: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LogFetchError", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SidecarDbName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -15740,13 +20245,64 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.LogFetchError = string(dAtA[iNdEx:postIndex]) + m.SidecarDbName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 15: + 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 *CreateKeyspaceResponse) 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: CreateKeyspaceResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tags", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15756,23 +20312,27 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Tags = append(m.Tags, string(dAtA[iNdEx:postIndex])) + if m.Keyspace == nil { + m.Keyspace = &Keyspace{} + } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -15796,7 +20356,7 @@ func (m *Workflow_Stream) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *Workflow) UnmarshalVT(dAtA []byte) error { +func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -15819,15 +20379,15 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Workflow: wiretype end group for non-group") + return fmt.Errorf("proto: CreateShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Workflow: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CreateShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -15855,13 +20415,13 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15871,33 +20431,29 @@ func (m *Workflow) 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.Source == nil { - m.Source = &Workflow_ReplicationLocation{} - } - if err := m.Source.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.ShardName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15907,33 +20463,17 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Target == nil { - m.Target = &Workflow_ReplicationLocation{} - } - if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.Force = bool(v != 0) case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxVReplicationLag", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludeParent", wireType) } - m.MaxVReplicationLag = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -15943,14 +20483,66 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxVReplicationLag |= int64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - case 5: + m.IncludeParent = bool(v != 0) + 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 *CreateShardResponse) 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: CreateShardResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardStreams", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -15977,111 +20569,18 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ShardStreams == nil { - m.ShardStreams = make(map[string]*Workflow_ShardStream) + if m.Keyspace == nil { + m.Keyspace = &Keyspace{} } - var mapkey string - var mapvalue *Workflow_ShardStream - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &Workflow_ShardStream{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.ShardStreams[mapkey] = mapvalue iNdEx = postIndex - case 6: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16091,29 +20590,33 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.WorkflowType = string(dAtA[iNdEx:postIndex]) + if m.Shard == nil { + m.Shard = &Shard{} + } + if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", wireType) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardAlreadyExists", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16123,24 +20626,12 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.WorkflowSubType = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.ShardAlreadyExists = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16163,7 +20654,7 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *AddCellInfoRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteCellInfoRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16186,10 +20677,10 @@ func (m *AddCellInfoRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AddCellInfoRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteCellInfoRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AddCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -16225,10 +20716,10 @@ func (m *AddCellInfoRequest) UnmarshalVT(dAtA []byte) error { m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16238,28 +20729,12 @@ func (m *AddCellInfoRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.CellInfo == nil { - m.CellInfo = &topodata.CellInfo{} - } - if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.Force = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16282,7 +20757,7 @@ func (m *AddCellInfoRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *AddCellInfoResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteCellInfoResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16305,10 +20780,10 @@ func (m *AddCellInfoResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AddCellInfoResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteCellInfoResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AddCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -16333,7 +20808,7 @@ func (m *AddCellInfoResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *AddCellsAliasRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteCellsAliasRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16356,10 +20831,10 @@ func (m *AddCellsAliasRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AddCellsAliasRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteCellsAliasRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AddCellsAliasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteCellsAliasRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -16394,38 +20869,6 @@ func (m *AddCellsAliasRequest) UnmarshalVT(dAtA []byte) error { } m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", 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.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16448,7 +20891,7 @@ func (m *AddCellsAliasRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *AddCellsAliasResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteCellsAliasResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16471,10 +20914,10 @@ func (m *AddCellsAliasResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: AddCellsAliasResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteCellsAliasResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: AddCellsAliasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteCellsAliasResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -16499,7 +20942,7 @@ func (m *AddCellsAliasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16522,17 +20965,17 @@ 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: DeleteKeyspaceRequest: 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: DeleteKeyspaceRequest: 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 Keyspace", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16542,31 +20985,27 @@ func (m *ApplyRoutingRulesRequest) 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.RoutingRules == nil { - m.RoutingRules = &vschema.RoutingRules{} - } - if err := m.RoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SkipRebuild", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -16583,12 +21022,12 @@ func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { break } } - m.SkipRebuild = bool(v != 0) + m.Recursive = bool(v != 0) case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RebuildCells", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16598,24 +21037,12 @@ func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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 + m.Force = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16638,7 +21065,7 @@ func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16661,10 +21088,10 @@ func (m *ApplyRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplyRoutingRulesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -16689,7 +21116,7 @@ func (m *ApplyRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteShardsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16712,15 +21139,15 @@ func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplyShardRoutingRulesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteShardsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyShardRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteShardsRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardRoutingRules", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -16747,16 +21174,14 @@ func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ShardRoutingRules == nil { - m.ShardRoutingRules = &vschema.ShardRoutingRules{} - } - if err := m.ShardRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Shards = append(m.Shards, &Shard{}) + if err := m.Shards[len(m.Shards)-1].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) + return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -16773,12 +21198,12 @@ func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { break } } - m.SkipRebuild = bool(v != 0) - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RebuildCells", wireType) + m.Recursive = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EvenIfServing", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -16788,24 +21213,32 @@ func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + m.EvenIfServing = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - if postIndex > l { - return io.ErrUnexpectedEOF + 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.RebuildCells = append(m.RebuildCells, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex + m.Force = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -16828,7 +21261,7 @@ func (m *ApplyShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteShardsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16851,10 +21284,10 @@ func (m *ApplyShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplyShardRoutingRulesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteShardsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyShardRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteShardsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -16879,7 +21312,7 @@ func (m *ApplyShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *DeleteSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16902,15 +21335,15 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplySchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteSrvVSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplySchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteSrvVSchemaRequest: 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 Cell", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -16938,139 +21371,113 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Cell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sql", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Sql = append(m.Sql, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DdlStrategy", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeleteSrvVSchemaResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - m.DdlStrategy = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UuidList", 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 - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteSrvVSchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteSrvVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.UuidList = append(m.UuidList, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MigrationContext", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeleteTabletsRequest) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - m.MigrationContext = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 7: + 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: DeleteTabletsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteTabletsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAliases", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -17097,16 +21504,14 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.WaitReplicasTimeout == nil { - m.WaitReplicasTimeout = &vttime.Duration{} - } - if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.TabletAliases = append(m.TabletAliases, &topodata.TabletAlias{}) + if err := m.TabletAliases[len(m.TabletAliases)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 8: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SkipPreflight", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -17123,43 +21528,7 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { break } } - m.SkipPreflight = bool(v != 0) - case 9: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CallerId", 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.CallerId == nil { - m.CallerId = &vtrpc.CallerID{} - } - if err := m.CallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.AllowPrimary = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -17182,7 +21551,7 @@ func (m *ApplySchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *DeleteTabletsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -17205,44 +21574,12 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplySchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteTabletsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplySchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteTabletsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UuidList", 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.UuidList = append(m.UuidList, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -17265,7 +21602,7 @@ func (m *ApplySchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -17288,10 +21625,10 @@ func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplyVSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: EmergencyReparentShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: EmergencyReparentShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -17327,48 +21664,8 @@ func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { m.Keyspace = string(dAtA[iNdEx:postIndex]) 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 != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DryRun", 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.DryRun = bool(v != 0) - case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -17396,11 +21693,11 @@ func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 5: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -17427,18 +21724,18 @@ func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.VSchema == nil { - m.VSchema = &vschema.Keyspace{} + if m.NewPrimary == nil { + m.NewPrimary = &topodata.TabletAlias{} } - if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 6: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sql", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IgnoreReplicas", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17448,78 +21745,29 @@ func (m *ApplyVSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Sql = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + m.IgnoreReplicas = append(m.IgnoreReplicas, &topodata.TabletAlias{}) + if err := m.IgnoreReplicas[len(m.IgnoreReplicas)-1].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 *ApplyVSchemaResponse) 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: ApplyVSchemaResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -17546,13 +21794,33 @@ func (m *ApplyVSchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.VSchema == nil { - m.VSchema = &vschema.Keyspace{} + if m.WaitReplicasTimeout == nil { + m.WaitReplicasTimeout = &vttime.Duration{} } - if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PreventCrossCellPromotion", 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.PreventCrossCellPromotion = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -17575,7 +21843,7 @@ func (m *ApplyVSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { +func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -17598,17 +21866,17 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BackupRequest: wiretype end group for non-group") + return fmt.Errorf("proto: EmergencyReparentShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: EmergencyReparentShardResponse: 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 @@ -17618,33 +21886,29 @@ func (m *BackupRequest) 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 case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17654,36 +21918,29 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.AllowPrimary = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.Concurrency = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Concurrency |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - case 4: + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IncrementalFromPos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PromotedPrimary", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17693,29 +21950,33 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.IncrementalFromPos = string(dAtA[iNdEx:postIndex]) + if m.PromotedPrimary == nil { + m.PromotedPrimary = &topodata.TabletAlias{} + } + if err := m.PromotedPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17725,12 +21986,26 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.UpgradeSafe = bool(v != 0) + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Events = append(m.Events, &logutil.Event{}) + if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -17753,7 +22028,7 @@ func (m *BackupRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -17776,10 +22051,10 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BackupResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsAppRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsAppRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -17820,7 +22095,7 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -17848,13 +22123,13 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Query = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) } - var stringLen uint64 + m.MaxRows = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17864,27 +22139,85 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.MaxRows |= int64(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UsePool", wireType) } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + 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.UsePool = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecuteFetchAsAppResponse) 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: ExecuteFetchAsAppResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecuteFetchAsAppResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -17911,10 +22244,10 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Event == nil { - m.Event = &logutil.Event{} + if m.Result == nil { + m.Result = &query.QueryResult{} } - if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -17940,7 +22273,7 @@ func (m *BackupResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsDBARequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -17963,17 +22296,17 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BackupShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsDBARequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BackupShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsDBARequest: 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 TabletAlias", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -17983,27 +22316,31 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} + } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -18031,13 +22368,13 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.Query = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) } - var v int + m.MaxRows = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -18047,17 +22384,16 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.MaxRows |= int64(b&0x7F) << shift if b < 0x80 { break } } - m.AllowPrimary = bool(v != 0) case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DisableBinlogs", wireType) } - m.Concurrency = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -18067,14 +22403,15 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Concurrency |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } + m.DisableBinlogs = bool(v != 0) case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UpgradeSafe", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -18091,7 +22428,7 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { break } } - m.UpgradeSafe = bool(v != 0) + m.ReloadSchema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -18114,7 +22451,7 @@ func (m *BackupShardRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ChangeTabletTypeRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteFetchAsDBAResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18137,15 +22474,15 @@ func (m *ChangeTabletTypeRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ChangeTabletTypeRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteFetchAsDBAResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ChangeTabletTypeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteFetchAsDBAResponse: 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 Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18172,52 +22509,13 @@ func (m *ChangeTabletTypeRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + if m.Result == nil { + m.Result = &query.QueryResult{} } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DbType", wireType) - } - m.DbType = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.DbType |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DryRun", 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.DryRun = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -18240,7 +22538,7 @@ func (m *ChangeTabletTypeRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18263,15 +22561,15 @@ func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ChangeTabletTypeResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteHookRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ChangeTabletTypeResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteHookRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BeforeTablet", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18298,16 +22596,16 @@ func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.BeforeTablet == nil { - m.BeforeTablet = &topodata.Tablet{} + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - if err := m.BeforeTablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AfterTablet", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletHookRequest", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18334,33 +22632,13 @@ func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.AfterTablet == nil { - m.AfterTablet = &topodata.Tablet{} + if m.TabletHookRequest == nil { + m.TabletHookRequest = &tabletmanagerdata.ExecuteHookRequest{} } - if err := m.AfterTablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletHookRequest.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WasDryRun", 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.WasDryRun = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -18383,7 +22661,7 @@ func (m *ChangeTabletTypeResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *CreateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18406,87 +22684,15 @@ func (m *CreateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CreateKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ExecuteHookResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CreateKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ExecuteHookResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", 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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", 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.Force = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowEmptyVSchema", 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.AllowEmptyVSchema = bool(v != 0) - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ServedFroms", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field HookResult", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18513,133 +22719,67 @@ func (m *CreateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ServedFroms = append(m.ServedFroms, &topodata.Keyspace_ServedFrom{}) - if err := m.ServedFroms[len(m.ServedFroms)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.HookResult == nil { + m.HookResult = &tabletmanagerdata.ExecuteHookResponse{} + } + if err := m.HookResult.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) - } - m.Type = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Type |= topodata.KeyspaceType(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BaseKeyspace", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.BaseKeyspace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 9: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SnapshotTime", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FindAllShardsInKeyspaceRequest) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - if m.SnapshotTime == nil { - m.SnapshotTime = &vttime.Time{} - } - if err := m.SnapshotTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 10: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DurabilityPolicy", 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 + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - m.DurabilityPolicy = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 11: + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: FindAllShardsInKeyspaceRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FindAllShardsInKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SidecarDbName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -18667,7 +22807,7 @@ func (m *CreateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SidecarDbName = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -18691,7 +22831,7 @@ func (m *CreateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *CreateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18714,15 +22854,15 @@ func (m *CreateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CreateKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: FindAllShardsInKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CreateKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: FindAllShardsInKeyspaceResponse: 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 Shards", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18739,22 +22879,115 @@ func (m *CreateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { break } } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Keyspace == nil { - m.Keyspace = &Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Shards == nil { + m.Shards = make(map[string]*Shard) + } + var mapkey string + var mapvalue *Shard + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &Shard{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Shards[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -18778,7 +23011,7 @@ func (m *CreateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18801,10 +23034,10 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CreateShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetBackupsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CreateShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetBackupsRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -18841,7 +23074,7 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -18869,13 +23102,13 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ShardName = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) } - var v int + m.Limit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -18885,15 +23118,14 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.Limit |= uint32(b&0x7F) << shift if b < 0x80 { break } } - m.Force = bool(v != 0) case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeParent", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Detailed", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -18910,7 +23142,26 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { break } } - m.IncludeParent = bool(v != 0) + m.Detailed = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DetailedLimit", wireType) + } + m.DetailedLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DetailedLimit |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -18933,7 +23184,7 @@ func (m *CreateShardRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *CreateShardResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetBackupsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18956,15 +23207,15 @@ func (m *CreateShardResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CreateShardResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetBackupsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CreateShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetBackupsResponse: 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 Backups", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18991,18 +23242,67 @@ func (m *CreateShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Keyspace == nil { - m.Keyspace = &Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Backups = append(m.Backups, &mysqlctl.BackupInfo{}) + if err := m.Backups[len(m.Backups)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + 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 *GetCellInfoRequest) 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: GetCellInfoRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19012,48 +23312,24 @@ func (m *CreateShardResponse) 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.Shard == nil { - m.Shard = &Shard{} - } - if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Cell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardAlreadyExists", 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.ShardAlreadyExists = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -19076,7 +23352,7 @@ func (m *CreateShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteCellInfoRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetCellInfoResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19099,17 +23375,17 @@ func (m *DeleteCellInfoRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteCellInfoRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetCellInfoResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19119,44 +23395,28 @@ func (m *DeleteCellInfoRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + if m.CellInfo == nil { + m.CellInfo = &topodata.CellInfo{} } - 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 - } + if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.Force = bool(v != 0) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -19179,7 +23439,7 @@ func (m *DeleteCellInfoRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteCellInfoResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetCellInfoNamesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19202,10 +23462,10 @@ func (m *DeleteCellInfoResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteCellInfoResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetCellInfoNamesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetCellInfoNamesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -19230,7 +23490,7 @@ func (m *DeleteCellInfoResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteCellsAliasRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetCellInfoNamesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19253,15 +23513,15 @@ func (m *DeleteCellsAliasRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteCellsAliasRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetCellInfoNamesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteCellsAliasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetCellInfoNamesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -19289,7 +23549,7 @@ func (m *DeleteCellsAliasRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Names = append(m.Names, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -19313,7 +23573,7 @@ func (m *DeleteCellsAliasRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteCellsAliasResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetCellsAliasesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19336,10 +23596,10 @@ func (m *DeleteCellsAliasResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteCellsAliasResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetCellsAliasesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteCellsAliasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetCellsAliasesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -19364,7 +23624,7 @@ func (m *DeleteCellsAliasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19387,17 +23647,17 @@ func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetCellsAliasesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetCellsAliasesResponse: 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 Aliases", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19407,29 +23667,177 @@ func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + if m.Aliases == nil { + m.Aliases = make(map[string]*topodata.CellsAlias) + } + var mapkey string + var mapvalue *topodata.CellsAlias + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &topodata.CellsAlias{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Aliases[mapkey] = mapvalue iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - var v int + 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 *GetFullStatusRequest) 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: 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 @@ -19439,17 +23847,84 @@ func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.Recursive = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + if msglen < 0 { + return ErrInvalidLength } - var v int + 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 @@ -19459,12 +23934,28 @@ func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.Force = bool(v != 0) + 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:]) @@ -19487,7 +23978,7 @@ func (m *DeleteKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspacesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19510,10 +24001,10 @@ func (m *DeleteKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspacesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetKeyspacesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -19538,7 +24029,7 @@ func (m *DeleteKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteShardsRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspacesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19561,15 +24052,15 @@ func (m *DeleteShardsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteShardsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspacesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteShardsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + 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 Shards", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspaces", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -19596,71 +24087,11 @@ func (m *DeleteShardsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shards = append(m.Shards, &Shard{}) - if err := m.Shards[len(m.Shards)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Keyspaces = append(m.Keyspaces, &Keyspace{}) + if err := m.Keyspaces[len(m.Keyspaces)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Recursive", 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.Recursive = bool(v != 0) - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field EvenIfServing", 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.EvenIfServing = bool(v != 0) - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", 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.Force = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -19683,7 +24114,7 @@ func (m *DeleteShardsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteShardsResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspaceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19706,12 +24137,44 @@ func (m *DeleteShardsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteShardsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspaceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteShardsResponse: 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 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 skippy, err := skip(dAtA[iNdEx:]) @@ -19734,7 +24197,7 @@ func (m *DeleteShardsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19757,17 +24220,17 @@ func (m *DeleteSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteSrvVSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteSrvVSchemaRequest: 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 Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19777,23 +24240,27 @@ func (m *DeleteSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cell = string(dAtA[iNdEx:postIndex]) + if m.Keyspace == nil { + m.Keyspace = &Keyspace{} + } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -19817,7 +24284,7 @@ func (m *DeleteSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19840,12 +24307,48 @@ func (m *DeleteSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteSrvVSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetPermissionsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteSrvVSchemaResponse: 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 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:]) @@ -19868,7 +24371,7 @@ func (m *DeleteSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteTabletsRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19891,15 +24394,15 @@ func (m *DeleteTabletsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteTabletsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetPermissionsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteTabletsRequest: 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 TabletAliases", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Permissions", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -19926,31 +24429,13 @@ func (m *DeleteTabletsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TabletAliases = append(m.TabletAliases, &topodata.TabletAlias{}) - if err := m.TabletAliases[len(m.TabletAliases)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Permissions == nil { + m.Permissions = &tabletmanagerdata.Permissions{} + } + if err := m.Permissions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowPrimary", 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.AllowPrimary = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -19973,7 +24458,7 @@ func (m *DeleteTabletsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteTabletsResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19996,10 +24481,10 @@ func (m *DeleteTabletsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteTabletsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetRoutingRulesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteTabletsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -20024,7 +24509,7 @@ func (m *DeleteTabletsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20047,17 +24532,17 @@ func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: EmergencyReparentShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetRoutingRulesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: EmergencyReparentShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetRoutingRulesResponse: 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 RoutingRules", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20067,59 +24552,82 @@ func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + if m.RoutingRules == nil { + m.RoutingRules = &vschema.RoutingRules{} } - 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 - } + if err := m.RoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetSchemaRequest) 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: GetSchemaRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -20146,18 +24654,18 @@ func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.NewPrimary == nil { - m.NewPrimary = &topodata.TabletAlias{} + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 4: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IgnoreReplicas", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20167,31 +24675,29 @@ func (m *EmergencyReparentShardRequest) 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 } - m.IgnoreReplicas = append(m.IgnoreReplicas, &topodata.TabletAlias{}) - if err := m.IgnoreReplicas[len(m.IgnoreReplicas)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 5: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20201,31 +24707,27 @@ func (m *EmergencyReparentShardRequest) 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.WaitReplicasTimeout == nil { - m.WaitReplicasTimeout = &vttime.Duration{} - } - if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 6: + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PreventCrossCellPromotion", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -20242,63 +24744,12 @@ func (m *EmergencyReparentShardRequest) UnmarshalVT(dAtA []byte) error { break } } - m.PreventCrossCellPromotion = bool(v != 0) - 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 *EmergencyReparentShardResponse) 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: EmergencyReparentShardResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: EmergencyReparentShardResponse: 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) + m.IncludeViews = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableNamesOnly", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20308,29 +24759,17 @@ func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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 - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + m.TableNamesOnly = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableSizesOnly", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20340,29 +24779,17 @@ func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PromotedPrimary", wireType) + m.TableSizesOnly = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableSchemaOnly", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20372,31 +24799,66 @@ func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLength + m.TableSchemaOnly = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.PromotedPrimary == nil { - m.PromotedPrimary = &topodata.TabletAlias{} + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetSchemaResponse) 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 } - if err := m.PromotedPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - iNdEx = postIndex - case 4: + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetSchemaResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -20423,8 +24885,10 @@ func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Events = append(m.Events, &logutil.Event{}) - if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Schema == nil { + m.Schema = &tabletmanagerdata.SchemaDefinition{} + } + if err := m.Schema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -20450,7 +24914,7 @@ func (m *EmergencyReparentShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20473,17 +24937,17 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAppRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAppRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetShardRequest: 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 @@ -20493,31 +24957,27 @@ func (m *ExecuteFetchAsAppRequest) 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 case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20545,47 +25005,8 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Query = string(dAtA[iNdEx:postIndex]) + m.ShardName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) - } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UsePool", 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.UsePool = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20608,7 +25029,7 @@ func (m *ExecuteFetchAsAppRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20631,15 +25052,15 @@ func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsAppResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsAppResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -20666,10 +25087,10 @@ func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Result == nil { - m.Result = &query.QueryResult{} + if m.Shard == nil { + m.Shard = &Shard{} } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -20695,7 +25116,7 @@ func (m *ExecuteFetchAsAppResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsDBARequest) UnmarshalVT(dAtA []byte) error { +func (m *GetShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20718,139 +25139,12 @@ func (m *ExecuteFetchAsDBARequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsDBARequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetShardRoutingRulesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsDBARequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetShardRoutingRulesRequest: 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 - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", 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.Query = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) - } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DisableBinlogs", 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.DisableBinlogs = bool(v != 0) - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReloadSchema", 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.ReloadSchema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20873,7 +25167,7 @@ func (m *ExecuteFetchAsDBARequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteFetchAsDBAResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20896,15 +25190,15 @@ func (m *ExecuteFetchAsDBAResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteFetchAsDBAResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetShardRoutingRulesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteFetchAsDBAResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetShardRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ShardRoutingRules", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -20931,10 +25225,10 @@ func (m *ExecuteFetchAsDBAResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Result == nil { - m.Result = &query.QueryResult{} + if m.ShardRoutingRules == nil { + m.ShardRoutingRules = &vschema.ShardRoutingRules{} } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ShardRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -20960,7 +25254,7 @@ func (m *ExecuteFetchAsDBAResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvKeyspaceNamesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20983,17 +25277,17 @@ func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteHookRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvKeyspaceNamesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteHookRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvKeyspaceNamesRequest: 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 Cells", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21003,33 +25297,80 @@ func (m *ExecuteHookRequest) 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 { + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { return err } - iNdEx = postIndex - case 2: + 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 *GetSrvKeyspaceNamesResponse_NameList) 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: GetSrvKeyspaceNamesResponse_NameList: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse_NameList: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletHookRequest", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21039,27 +25380,23 @@ func (m *ExecuteHookRequest) 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.TabletHookRequest == nil { - m.TabletHookRequest = &tabletmanagerdata.ExecuteHookRequest{} - } - if err := m.TabletHookRequest.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Names = append(m.Names, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -21083,7 +25420,7 @@ func (m *ExecuteHookRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvKeyspaceNamesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21106,15 +25443,15 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecuteHookResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecuteHookResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field HookResult", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -21141,12 +25478,105 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.HookResult == nil { - m.HookResult = &tabletmanagerdata.ExecuteHookResponse{} + if m.Names == nil { + m.Names = make(map[string]*GetSrvKeyspaceNamesResponse_NameList) } - if err := m.HookResult.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey string + var mapvalue *GetSrvKeyspaceNamesResponse_NameList + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &GetSrvKeyspaceNamesResponse_NameList{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.Names[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -21170,7 +25600,7 @@ func (m *ExecuteHookResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *FindAllShardsInKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvKeyspacesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21193,10 +25623,10 @@ func (m *FindAllShardsInKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: FindAllShardsInKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvKeyspacesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: FindAllShardsInKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvKeyspacesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -21231,6 +25661,38 @@ func (m *FindAllShardsInKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", 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.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -21253,7 +25715,7 @@ func (m *FindAllShardsInKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvKeyspacesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21276,15 +25738,15 @@ func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: FindAllShardsInKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvKeyspacesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: FindAllShardsInKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvKeyspacesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SrvKeyspaces", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -21311,11 +25773,11 @@ func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Shards == nil { - m.Shards = make(map[string]*Shard) + if m.SrvKeyspaces == nil { + m.SrvKeyspaces = make(map[string]*topodata.SrvKeyspace) } var mapkey string - var mapvalue *Shard + var mapvalue *topodata.SrvKeyspace for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -21389,7 +25851,7 @@ func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &Shard{} + mapvalue = &topodata.SrvKeyspace{} if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -21409,7 +25871,7 @@ func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { iNdEx += skippy } } - m.Shards[mapkey] = mapvalue + m.SrvKeyspaces[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -21433,7 +25895,7 @@ func (m *FindAllShardsInKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21456,10 +25918,10 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetBackupsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateThrottlerConfigRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetBackupsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateThrottlerConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -21495,8 +25957,59 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Enable", 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.Enable = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Disable", 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.Disable = bool(v != 0) + case 4: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Threshold", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Threshold = float64(math.Float64frombits(v)) + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CustomQuery", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -21524,13 +26037,13 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.CustomQuery = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 6: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CustomQuerySet", wireType) } - m.Limit = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21540,14 +26053,15 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Limit |= uint32(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - case 4: + m.CustomQuerySet = bool(v != 0) + case 7: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Detailed", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CheckAsCheckSelf", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -21564,12 +26078,12 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { break } } - m.Detailed = bool(v != 0) - case 5: + m.CheckAsCheckSelf = bool(v != 0) + case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DetailedLimit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CheckAsCheckShard", wireType) } - m.DetailedLimit = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21579,11 +26093,48 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.DetailedLimit |= uint32(b&0x7F) << shift + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.CheckAsCheckShard = bool(v != 0) + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ThrottledApp", 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.ThrottledApp == nil { + m.ThrottledApp = &topodata.ThrottledAppRule{} + } + if err := m.ThrottledApp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -21606,7 +26157,7 @@ func (m *GetBackupsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetBackupsResponse) UnmarshalVT(dAtA []byte) error { +func (m *UpdateThrottlerConfigResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21629,46 +26180,12 @@ func (m *GetBackupsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetBackupsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateThrottlerConfigResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetBackupsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateThrottlerConfigResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Backups", 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.Backups = append(m.Backups, &mysqlctl.BackupInfo{}) - if err := m.Backups[len(m.Backups)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -21691,7 +26208,7 @@ func (m *GetBackupsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetCellInfoRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21714,10 +26231,10 @@ func (m *GetCellInfoRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetCellInfoRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvVSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -21774,7 +26291,7 @@ func (m *GetCellInfoRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetCellInfoResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21797,15 +26314,15 @@ func (m *GetCellInfoResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetCellInfoResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvVSchemaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SrvVSchema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -21832,10 +26349,10 @@ func (m *GetCellInfoResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.CellInfo == nil { - m.CellInfo = &topodata.CellInfo{} + if m.SrvVSchema == nil { + m.SrvVSchema = &vschema.SrvVSchema{} } - if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.SrvVSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -21861,58 +26378,7 @@ func (m *GetCellInfoResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetCellInfoNamesRequest) 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: GetCellInfoNamesRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellInfoNamesRequest: 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 *GetCellInfoNamesResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvVSchemasRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21935,15 +26401,15 @@ func (m *GetCellInfoNamesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetCellInfoNamesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvVSchemasRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellInfoNamesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvVSchemasRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -21971,7 +26437,7 @@ func (m *GetCellInfoNamesResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Names = append(m.Names, string(dAtA[iNdEx:postIndex])) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -21995,58 +26461,7 @@ func (m *GetCellInfoNamesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetCellsAliasesRequest) 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: GetCellsAliasesRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellsAliasesRequest: 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 *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetSrvVSchemasResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22069,15 +26484,15 @@ func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetCellsAliasesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetSrvVSchemasResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetCellsAliasesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetSrvVSchemasResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Aliases", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SrvVSchemas", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -22104,11 +26519,11 @@ func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Aliases == nil { - m.Aliases = make(map[string]*topodata.CellsAlias) + if m.SrvVSchemas == nil { + m.SrvVSchemas = make(map[string]*vschema.SrvVSchema) } var mapkey string - var mapvalue *topodata.CellsAlias + var mapvalue *vschema.SrvVSchema for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -22182,7 +26597,7 @@ func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &topodata.CellsAlias{} + mapvalue = &vschema.SrvVSchema{} if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -22202,7 +26617,7 @@ func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { iNdEx += skippy } } - m.Aliases[mapkey] = mapvalue + m.SrvVSchemas[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -22226,7 +26641,7 @@ func (m *GetCellsAliasesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetFullStatusRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetTabletRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22249,10 +26664,10 @@ 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: GetTabletRequest: 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: GetTabletRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -22313,7 +26728,7 @@ func (m *GetFullStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetTabletResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22336,15 +26751,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: GetTabletResponse: 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: GetTabletResponse: 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 Tablet", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -22371,10 +26786,10 @@ func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Status == nil { - m.Status = &replicationdata.FullStatus{} + if m.Tablet == nil { + m.Tablet = &topodata.Tablet{} } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -22400,58 +26815,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 *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22474,17 +26838,17 @@ 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: GetTabletsRequest: 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: GetTabletsRequest: 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 Keyspace", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -22494,80 +26858,59 @@ func (m *GetKeyspacesResponse) 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 } - m.Keyspaces = append(m.Keyspaces, &Keyspace{}) - if err := m.Keyspaces[len(m.Keyspaces)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + 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 + } } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - 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 + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > 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: + m.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -22595,62 +26938,31 @@ func (m *GetKeyspaceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Cells = append(m.Cells, 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 *GetKeyspaceResponse) 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 + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Strict", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + 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 + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: GetKeyspaceResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.Strict = bool(v != 0) + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAliases", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -22677,13 +26989,30 @@ func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Keyspace == nil { - m.Keyspace = &Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.TabletAliases = append(m.TabletAliases, &topodata.TabletAlias{}) + if err := m.TabletAliases[len(m.TabletAliases)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) + } + m.TabletType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletType |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -22706,7 +27035,7 @@ func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetTabletsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22729,15 +27058,15 @@ 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: GetTabletsResponse: 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: GetTabletsResponse: 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 Tablets", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -22764,10 +27093,8 @@ func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} - } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Tablets = append(m.Tablets, &topodata.Tablet{}) + if err := m.Tablets[len(m.Tablets)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -22793,7 +27120,7 @@ func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetTopologyPathRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22816,17 +27143,17 @@ 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: GetTopologyPathRequest: 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: GetTopologyPathRequest: 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 Path", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -22836,79 +27163,24 @@ func (m *GetPermissionsResponse) 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 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Permissions == nil { - m.Permissions = &tabletmanagerdata.Permissions{} - } - if err := m.Permissions.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 *GetRoutingRulesRequest) 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 + break + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: GetRoutingRulesRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Path = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -22931,7 +27203,7 @@ func (m *GetRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetTopologyPathResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22954,15 +27226,15 @@ func (m *GetRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetRoutingRulesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetTopologyPathResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetTopologyPathResponse: 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 Cell", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -22989,10 +27261,10 @@ func (m *GetRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RoutingRules == nil { - m.RoutingRules = &vschema.RoutingRules{} + if m.Cell == nil { + m.Cell = &TopologyCell{} } - if err := m.RoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Cell.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -23018,7 +27290,7 @@ func (m *GetRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23041,17 +27313,17 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: TopologyCell: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TopologyCell: 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 Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23061,31 +27333,27 @@ func (m *GetSchemaRequest) 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.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -23113,11 +27381,11 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) + m.Path = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -23145,33 +27413,13 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) + m.Data = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", 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.IncludeViews = bool(v != 0) - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableNamesOnly", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Children", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23181,52 +27429,24 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.TableNamesOnly = bool(v != 0) - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableSizesOnly", 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 - } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.TableSizesOnly = bool(v != 0) - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableSchemaOnly", wireType) + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - 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 - } + if postIndex > l { + return io.ErrUnexpectedEOF } - m.TableSchemaOnly = bool(v != 0) + m.Children = append(m.Children, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -23249,7 +27469,7 @@ func (m *GetSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetVSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23272,17 +27492,17 @@ func (m *GetSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetVSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Schema", 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 @@ -23292,27 +27512,23 @@ func (m *GetSchemaResponse) 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.Schema == nil { - m.Schema = &tabletmanagerdata.SchemaDefinition{} - } - if err := m.Schema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -23336,7 +27552,7 @@ func (m *GetSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetVersionRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23359,17 +27575,17 @@ func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetVersionRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetVersionRequest: 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 TabletAlias", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23379,27 +27595,82 @@ func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} + } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 2: + 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 *GetVersionResponse) 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: GetVersionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetVersionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -23427,7 +27698,7 @@ func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ShardName = string(dAtA[iNdEx:postIndex]) + m.Version = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -23451,7 +27722,7 @@ func (m *GetShardRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetShardResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetVSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23474,15 +27745,15 @@ func (m *GetShardResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetShardResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetVSchemaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VSchema", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -23509,10 +27780,10 @@ func (m *GetShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Shard == nil { - m.Shard = &Shard{} + if m.VSchema == nil { + m.VSchema = &vschema.Keyspace{} } - if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -23538,7 +27809,7 @@ func (m *GetShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetWorkflowsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23561,12 +27832,116 @@ func (m *GetShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetShardRoutingRulesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetWorkflowsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetShardRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetWorkflowsRequest: 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 + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ActiveOnly", 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.ActiveOnly = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NameOnly", 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.NameOnly = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", 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.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -23589,7 +27964,7 @@ func (m *GetShardRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetWorkflowsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23612,15 +27987,15 @@ func (m *GetShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetShardRoutingRulesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetWorkflowsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetShardRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetWorkflowsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardRoutingRules", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflows", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -23647,10 +28022,8 @@ func (m *GetShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ShardRoutingRules == nil { - m.ShardRoutingRules = &vschema.ShardRoutingRules{} - } - if err := m.ShardRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Workflows = append(m.Workflows, &Workflow{}) + if err := m.Workflows[len(m.Workflows)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -23676,7 +28049,7 @@ func (m *GetShardRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvKeyspaceNamesRequest) UnmarshalVT(dAtA []byte) error { +func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23699,17 +28072,137 @@ func (m *GetSrvKeyspaceNamesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: InitShardPrimaryRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitShardPrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryElectTabletAlias", 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.PrimaryElectTabletAlias == nil { + m.PrimaryElectTabletAlias = &topodata.TabletAlias{} + } + if err := m.PrimaryElectTabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", 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.Force = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23719,23 +28212,27 @@ func (m *GetSrvKeyspaceNamesRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + if m.WaitReplicasTimeout == nil { + m.WaitReplicasTimeout = &vttime.Duration{} + } + if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -23759,7 +28256,7 @@ func (m *GetSrvKeyspaceNamesRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvKeyspaceNamesResponse_NameList) UnmarshalVT(dAtA []byte) error { +func (m *InitShardPrimaryResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23782,17 +28279,17 @@ func (m *GetSrvKeyspaceNamesResponse_NameList) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse_NameList: wiretype end group for non-group") + return fmt.Errorf("proto: InitShardPrimaryResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse_NameList: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InitShardPrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23802,23 +28299,25 @@ func (m *GetSrvKeyspaceNamesResponse_NameList) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Names = append(m.Names, string(dAtA[iNdEx:postIndex])) + m.Events = append(m.Events, &logutil.Event{}) + if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -23842,7 +28341,7 @@ func (m *GetSrvKeyspaceNamesResponse_NameList) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvKeyspaceNamesResponse) UnmarshalVT(dAtA []byte) error { +func (m *MoveTablesCreateRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -23865,17 +28364,17 @@ func (m *GetSrvKeyspaceNamesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: MoveTablesCreateRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvKeyspaceNamesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MoveTablesCreateRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -23885,175 +28384,27 @@ func (m *GetSrvKeyspaceNamesResponse) 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.Names == nil { - m.Names = make(map[string]*GetSrvKeyspaceNamesResponse_NameList) - } - var mapkey string - var mapvalue *GetSrvKeyspaceNamesResponse_NameList - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &GetSrvKeyspaceNamesResponse_NameList{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.Names[mapkey] = mapvalue + m.Workflow = 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 *GetSrvKeyspacesRequest) 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: GetSrvKeyspacesRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvKeyspacesRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24081,11 +28432,11 @@ func (m *GetSrvKeyspacesRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.SourceKeyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TargetKeyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24113,64 +28464,13 @@ func (m *GetSrvKeyspacesRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + m.TargetKeyspace = 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 *GetSrvKeyspacesResponse) 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: GetSrvKeyspacesResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvKeyspacesResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SrvKeyspaces", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24180,29 +28480,27 @@ func (m *GetSrvKeyspacesResponse) 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.SrvKeyspaces == nil { - m.SrvKeyspaces = make(map[string]*topodata.SrvKeyspace) - } - var mapkey string - var mapvalue *topodata.SrvKeyspace - for iNdEx < postIndex { - entryPreIndex := iNdEx - var wire uint64 + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType == 0 { + var v topodata.TabletType for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24212,43 +28510,44 @@ func (m *GetSrvKeyspacesResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= uint64(b&0x7F) << shift + v |= topodata.TabletType(b&0x7F) << shift if b < 0x80 { break } } - fieldNum := int32(wire >> 3) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow } - if postStringIndexmapkey > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24258,97 +28557,122 @@ func (m *GetSrvKeyspacesResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - mapmsglen |= int(b&0x7F) << shift + v |= topodata.TabletType(b&0x7F) << shift if b < 0x80 { break } } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &topodata.SrvKeyspace{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy + m.TabletTypes = append(m.TabletTypes, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } - m.SrvKeyspaces[mapkey] = mapvalue - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { + m.TabletSelectionPreference = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletSelectionPreference |= tabletmanagerdata.TabletSelectionPreference(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceShards", 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 } - if (iNdEx + skippy) > l { + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > 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 *UpdateThrottlerConfigRequest) 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 + m.SourceShards = append(m.SourceShards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllTables", wireType) } - if iNdEx >= l { - return io.ErrUnexpectedEOF + 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 + } } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.AllTables = bool(v != 0) + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeTables", wireType) } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: UpdateThrottlerConfigRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateThrottlerConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + 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.IncludeTables = append(m.IncludeTables, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24376,13 +28700,13 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Enable", wireType) + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExternalClusterName", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24392,17 +28716,29 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.Enable = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Disable", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - var v int + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExternalClusterName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceTimeZone", wireType) + } + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24412,26 +28748,27 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.Disable = bool(v != 0) - case 4: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Threshold", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - var v uint64 - if (iNdEx + 8) > l { + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Threshold = float64(math.Float64frombits(v)) - case 5: + m.SourceTimeZone = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CustomQuery", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24459,11 +28796,11 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CustomQuery = string(dAtA[iNdEx:postIndex]) + m.OnDdl = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 6: + case 14: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CustomQuerySet", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StopAfterCopy", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -24480,10 +28817,10 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { break } } - m.CustomQuerySet = bool(v != 0) - case 7: + m.StopAfterCopy = bool(v != 0) + case 15: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CheckAsCheckSelf", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DropForeignKeys", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -24500,10 +28837,10 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { break } } - m.CheckAsCheckSelf = bool(v != 0) - case 8: + m.DropForeignKeys = bool(v != 0) + case 16: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CheckAsCheckShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -24520,12 +28857,12 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { break } } - m.CheckAsCheckShard = bool(v != 0) - case 9: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ThrottledApp", wireType) + m.DeferSecondaryKeys = bool(v != 0) + case 17: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AutoStart", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24535,28 +28872,12 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.ThrottledApp == nil { - m.ThrottledApp = &topodata.ThrottledAppRule{} - } - if err := m.ThrottledApp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.AutoStart = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -24579,7 +28900,7 @@ func (m *UpdateThrottlerConfigRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateThrottlerConfigResponse) UnmarshalVT(dAtA []byte) error { +func (m *MoveTablesCreateResponse_TabletInfo) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -24602,12 +28923,68 @@ func (m *UpdateThrottlerConfigResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateThrottlerConfigResponse: wiretype end group for non-group") + return fmt.Errorf("proto: MoveTablesCreateResponse_TabletInfo: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateThrottlerConfigResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MoveTablesCreateResponse_TabletInfo: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", 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.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} + } + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Created", 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.Created = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -24630,7 +29007,7 @@ func (m *UpdateThrottlerConfigResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *MoveTablesCreateResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -24653,15 +29030,15 @@ func (m *GetSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvVSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: MoveTablesCreateResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MoveTablesCreateResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Summary", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24689,7 +29066,41 @@ func (m *GetSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cell = string(dAtA[iNdEx:postIndex]) + m.Summary = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Details", 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.Details = append(m.Details, &MoveTablesCreateResponse_TabletInfo{}) + if err := m.Details[len(m.Details)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -24713,7 +29124,7 @@ func (m *GetSrvVSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *MoveTablesCompleteRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -24736,17 +29147,17 @@ func (m *GetSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvVSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: MoveTablesCompleteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MoveTablesCompleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SrvVSchema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -24756,28 +29167,136 @@ func (m *GetSrvVSchemaResponse) 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.SrvVSchema == nil { - m.SrvVSchema = &vschema.SrvVSchema{} + m.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetKeyspace", wireType) } - if err := m.SrvVSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + 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.TargetKeyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeepData", 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.KeepData = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeepRoutingRules", 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.KeepRoutingRules = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RenameTables", 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.RenameTables = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", 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.DryRun = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -24800,7 +29319,7 @@ func (m *GetSrvVSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvVSchemasRequest) UnmarshalVT(dAtA []byte) error { +func (m *MoveTablesCompleteResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -24823,15 +29342,47 @@ func (m *GetSrvVSchemasRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvVSchemasRequest: wiretype end group for non-group") + return fmt.Errorf("proto: MoveTablesCompleteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvVSchemasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MoveTablesCompleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Summary", 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.Summary = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DryRunResults", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -24859,7 +29410,7 @@ func (m *GetSrvVSchemasRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + m.DryRunResults = append(m.DryRunResults, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -24883,7 +29434,7 @@ func (m *GetSrvVSchemasRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetSrvVSchemasResponse) UnmarshalVT(dAtA []byte) error { +func (m *PingTabletRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -24906,15 +29457,15 @@ func (m *GetSrvVSchemasResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetSrvVSchemasResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PingTabletRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetSrvVSchemasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PingTabletRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SrvVSchemas", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -24941,105 +29492,12 @@ func (m *GetSrvVSchemasResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.SrvVSchemas == nil { - m.SrvVSchemas = make(map[string]*vschema.SrvVSchema) + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - var mapkey string - var mapvalue *vschema.SrvVSchema - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &vschema.SrvVSchema{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.SrvVSchemas[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -25063,7 +29521,7 @@ func (m *GetSrvVSchemasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTabletRequest) UnmarshalVT(dAtA []byte) error { +func (m *PingTabletResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25084,50 +29542,14 @@ func (m *GetTabletRequest) UnmarshalVT(dAtA []byte) error { } } fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: GetTabletRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetTabletRequest: 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 + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PingTabletResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PingTabletResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -25150,7 +29572,7 @@ func (m *GetTabletRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTabletResponse) UnmarshalVT(dAtA []byte) error { +func (m *PlannedReparentShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25173,15 +29595,79 @@ func (m *GetTabletResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetTabletResponse: wiretype end group for non-group") + return fmt.Errorf("proto: PlannedReparentShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetTabletResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PlannedReparentShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) + 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -25208,10 +29694,82 @@ func (m *GetTabletResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Tablet == nil { - m.Tablet = &topodata.Tablet{} + if m.NewPrimary == nil { + m.NewPrimary = &topodata.TabletAlias{} } - if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AvoidPrimary", 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.AvoidPrimary == nil { + m.AvoidPrimary = &topodata.TabletAlias{} + } + if err := m.AvoidPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", 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.WaitReplicasTimeout == nil { + m.WaitReplicasTimeout = &vttime.Duration{} + } + if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -25237,7 +29795,7 @@ func (m *GetTabletResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { +func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25260,10 +29818,10 @@ func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetTabletsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: PlannedReparentShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetTabletsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PlannedReparentShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -25332,9 +29890,9 @@ func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PromotedPrimary", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -25344,47 +29902,31 @@ func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Strict", wireType) + if m.PromotedPrimary == nil { + m.PromotedPrimary = &topodata.TabletAlias{} } - 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 - } + if err := m.PromotedPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.Strict = bool(v != 0) - case 5: + iNdEx = postIndex + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletAliases", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -25411,30 +29953,11 @@ func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TabletAliases = append(m.TabletAliases, &topodata.TabletAlias{}) - if err := m.TabletAliases[len(m.TabletAliases)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Events = append(m.Events, &logutil.Event{}) + if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) - } - m.TabletType = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TabletType |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -25457,7 +29980,7 @@ func (m *GetTabletsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTabletsResponse) UnmarshalVT(dAtA []byte) error { +func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25480,17 +30003,17 @@ func (m *GetTabletsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetTabletsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RebuildKeyspaceGraphRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetTabletsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RebuildKeyspaceGraphRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tablets", 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 @@ -25500,26 +30023,76 @@ func (m *GetTabletsResponse) 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 } - m.Tablets = append(m.Tablets, &topodata.Tablet{}) - if err := m.Tablets[len(m.Tablets)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Keyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", 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.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllowPartial", 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.AllowPartial = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -25542,7 +30115,7 @@ func (m *GetTabletsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTopologyPathRequest) UnmarshalVT(dAtA []byte) error { +func (m *RebuildKeyspaceGraphResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25565,15 +30138,66 @@ func (m *GetTopologyPathRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetTopologyPathRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RebuildKeyspaceGraphResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetTopologyPathRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RebuildKeyspaceGraphResponse: 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 *RebuildVSchemaGraphRequest) 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: RebuildVSchemaGraphRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RebuildVSchemaGraphRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -25601,7 +30225,7 @@ func (m *GetTopologyPathRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Path = string(dAtA[iNdEx:postIndex]) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -25625,7 +30249,7 @@ func (m *GetTopologyPathRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetTopologyPathResponse) UnmarshalVT(dAtA []byte) error { +func (m *RebuildVSchemaGraphResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25648,15 +30272,66 @@ func (m *GetTopologyPathResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetTopologyPathResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RebuildVSchemaGraphResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetTopologyPathResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RebuildVSchemaGraphResponse: 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 *RefreshStateRequest) 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: RefreshStateRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RefreshStateRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -25683,10 +30358,10 @@ func (m *GetTopologyPathResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Cell == nil { - m.Cell = &TopologyCell{} + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - if err := m.Cell.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -25712,7 +30387,7 @@ func (m *GetTopologyPathResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { +func (m *RefreshStateResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25735,47 +30410,66 @@ func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TopologyCell: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshStateResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TopologyCell: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshStateResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", 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 - } + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - if postIndex > l { + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RefreshStateByShardRequest) 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 } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: + 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: RefreshStateByShardRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RefreshStateByShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -25803,11 +30497,11 @@ func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Path = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -25835,11 +30529,11 @@ func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Data = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Children", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -25867,7 +30561,7 @@ func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Children = append(m.Children, string(dAtA[iNdEx:postIndex])) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -25891,7 +30585,7 @@ func (m *TopologyCell) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetVSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *RefreshStateByShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25914,15 +30608,35 @@ func (m *GetVSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetVSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RefreshStateByShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RefreshStateByShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsPartialRefresh", 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.IsPartialRefresh = bool(v != 0) + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PartialRefreshDetails", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -25950,7 +30664,7 @@ func (m *GetVSchemaRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.PartialRefreshDetails = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -25974,7 +30688,7 @@ func (m *GetVSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetVersionRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -25997,10 +30711,10 @@ func (m *GetVersionRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetVersionRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetVersionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -26061,7 +30775,7 @@ func (m *GetVersionRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetVersionResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26084,44 +30798,12 @@ func (m *GetVersionResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetVersionResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetVersionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", 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.Version = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -26144,7 +30826,7 @@ func (m *GetVersionResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetVSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26167,17 +30849,17 @@ func (m *GetVSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetVSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaKeyspaceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VSchema", 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 @@ -26187,82 +30869,27 @@ func (m *GetVSchemaResponse) 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.VSchema == nil { - m.VSchema = &vschema.Keyspace{} - } - if err := m.VSchema.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = 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 *GetWorkflowsRequest) 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: GetWorkflowsRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetWorkflowsRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -26290,11 +30917,11 @@ func (m *GetWorkflowsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.WaitPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ActiveOnly", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludePrimary", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -26311,12 +30938,12 @@ func (m *GetWorkflowsRequest) UnmarshalVT(dAtA []byte) error { break } } - m.ActiveOnly = bool(v != 0) - case 3: + m.IncludePrimary = bool(v != 0) + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NameOnly", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) } - var v int + m.Concurrency = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -26326,12 +30953,11 @@ func (m *GetWorkflowsRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.Concurrency |= uint32(b&0x7F) << shift if b < 0x80 { break } } - m.NameOnly = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -26354,7 +30980,7 @@ func (m *GetWorkflowsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetWorkflowsResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26377,15 +31003,15 @@ func (m *GetWorkflowsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetWorkflowsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetWorkflowsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workflows", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -26412,8 +31038,8 @@ func (m *GetWorkflowsResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Workflows = append(m.Workflows, &Workflow{}) - if err := m.Workflows[len(m.Workflows)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Events = append(m.Events, &logutil.Event{}) + if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -26439,7 +31065,7 @@ func (m *GetWorkflowsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26462,10 +31088,10 @@ func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitShardPrimaryRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitShardPrimaryRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -26534,9 +31160,9 @@ func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryElectTabletAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -26546,31 +31172,27 @@ func (m *InitShardPrimaryRequest) 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.PrimaryElectTabletAlias == nil { - m.PrimaryElectTabletAlias = &topodata.TabletAlias{} - } - if err := m.PrimaryElectTabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.WaitPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludePrimary", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -26587,12 +31209,12 @@ func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { break } } - m.Force = bool(v != 0) + m.IncludePrimary = bool(v != 0) case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) } - var msglen int + m.Concurrency = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -26602,28 +31224,11 @@ func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Concurrency |= uint32(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.WaitReplicasTimeout == nil { - m.WaitReplicasTimeout = &vttime.Duration{} - } - if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -26646,7 +31251,7 @@ func (m *InitShardPrimaryRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *InitShardPrimaryResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReloadSchemaShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26669,13 +31274,13 @@ func (m *InitShardPrimaryResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InitShardPrimaryResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReloadSchemaShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InitShardPrimaryResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReloadSchemaShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) } @@ -26731,145 +31336,7 @@ func (m *InitShardPrimaryResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PingTabletRequest) 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: PingTabletRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PingTabletRequest: 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 *PingTabletResponse) 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: PingTabletResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PingTabletResponse: 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 *PlannedReparentShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -26892,10 +31359,10 @@ func (m *PlannedReparentShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PlannedReparentShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RemoveBackupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PlannedReparentShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RemoveBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -26964,9 +31431,9 @@ func (m *PlannedReparentShardRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -26976,100 +31443,75 @@ func (m *PlannedReparentShardRequest) 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.NewPrimary == nil { - m.NewPrimary = &topodata.TabletAlias{} - } - if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AvoidPrimary", 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.AvoidPrimary == nil { - m.AvoidPrimary = &topodata.TabletAlias{} - } - if err := m.AvoidPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitReplicasTimeout", 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 + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RemoveBackupResponse) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - if m.WaitReplicasTimeout == nil { - m.WaitReplicasTimeout = &vttime.Duration{} - } - if err := m.WaitReplicasTimeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - iNdEx = postIndex + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RemoveBackupResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RemoveBackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -27092,7 +31534,7 @@ func (m *PlannedReparentShardRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { +func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27115,10 +31557,10 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PlannedReparentShardResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RemoveKeyspaceCellRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PlannedReparentShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RemoveKeyspaceCellRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -27155,7 +31597,7 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -27183,13 +31625,13 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.Cell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PromotedPrimary", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -27199,33 +31641,17 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.PromotedPrimary == nil { - m.PromotedPrimary = &topodata.TabletAlias{} - } - if err := m.PromotedPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.Force = bool(v != 0) case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -27235,26 +31661,63 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLength + m.Recursive = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Events = append(m.Events, &logutil.Event{}) - if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RemoveKeyspaceCellResponse) 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 } - iNdEx = postIndex + 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: RemoveKeyspaceCellResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RemoveKeyspaceCellResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -27277,7 +31740,7 @@ func (m *PlannedReparentShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { +func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27300,10 +31763,10 @@ func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RebuildKeyspaceGraphRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RemoveShardCellRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RebuildKeyspaceGraphRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RemoveShardCellRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -27340,7 +31803,7 @@ func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -27368,11 +31831,43 @@ func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + m.ShardName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cell", 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.Cell = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AllowPartial", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -27389,7 +31884,27 @@ func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { break } } - m.AllowPartial = bool(v != 0) + m.Force = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Recursive", 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.Recursive = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -27412,7 +31927,7 @@ func (m *RebuildKeyspaceGraphRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RebuildKeyspaceGraphResponse) UnmarshalVT(dAtA []byte) error { +func (m *RemoveShardCellResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27435,10 +31950,10 @@ func (m *RebuildKeyspaceGraphResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RebuildKeyspaceGraphResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RemoveShardCellResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RebuildKeyspaceGraphResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RemoveShardCellResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -27463,7 +31978,7 @@ func (m *RebuildKeyspaceGraphResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RebuildVSchemaGraphRequest) UnmarshalVT(dAtA []byte) error { +func (m *ReparentTabletRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27486,17 +32001,17 @@ func (m *RebuildVSchemaGraphRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RebuildVSchemaGraphRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ReparentTabletRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RebuildVSchemaGraphRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReparentTabletRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -27506,23 +32021,27 @@ func (m *RebuildVSchemaGraphRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + if m.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} + } + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -27546,7 +32065,7 @@ func (m *RebuildVSchemaGraphRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RebuildVSchemaGraphResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27569,12 +32088,112 @@ func (m *RebuildVSchemaGraphResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RebuildVSchemaGraphResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReparentTabletResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RebuildVSchemaGraphResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReparentTabletResponse: 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Primary", 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.Primary == nil { + m.Primary = &topodata.TabletAlias{} + } + if err := m.Primary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -27597,7 +32216,7 @@ func (m *RebuildVSchemaGraphResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RefreshStateRequest) UnmarshalVT(dAtA []byte) error { +func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27620,10 +32239,10 @@ func (m *RefreshStateRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshStateRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RestoreFromBackupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RestoreFromBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -27662,57 +32281,130 @@ func (m *RefreshStateRequest) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BackupTime", 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.BackupTime == nil { + m.BackupTime = &vttime.Time{} + } + if err := m.BackupTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestoreToPos", 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 } - if (iNdEx + skippy) > l { + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > 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 *RefreshStateResponse) 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 + m.RestoreToPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) } - if iNdEx >= l { + 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.DryRun = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RestoreToTimestamp", 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 } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.RestoreToTimestamp == nil { + m.RestoreToTimestamp = &vttime.Time{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: RefreshStateResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if err := m.RestoreToTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -27735,7 +32427,7 @@ func (m *RefreshStateResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RefreshStateByShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -27758,13 +32450,49 @@ func (m *RefreshStateByShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshStateByShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RestoreFromBackupResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateByShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RestoreFromBackupResponse: 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 + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } @@ -27796,7 +32524,7 @@ func (m *RefreshStateByShardRequest) UnmarshalVT(dAtA []byte) error { } m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } @@ -27828,11 +32556,11 @@ func (m *RefreshStateByShardRequest) UnmarshalVT(dAtA []byte) error { } m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Event", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -27842,126 +32570,27 @@ func (m *RefreshStateByShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, 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 *RefreshStateByShardResponse) 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: RefreshStateByShardResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshStateByShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsPartialRefresh", 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.IsPartialRefresh = bool(v != 0) - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PartialRefreshDetails", 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 m.Event == nil { + m.Event = &logutil.Event{} } - if postIndex > l { - return io.ErrUnexpectedEOF + if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.PartialRefreshDetails = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -27985,7 +32614,7 @@ func (m *RefreshStateByShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { +func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28008,10 +32637,10 @@ func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RunHealthCheckRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RunHealthCheckRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -28072,7 +32701,7 @@ func (m *ReloadSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *RunHealthCheckResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28095,10 +32724,10 @@ func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RunHealthCheckResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RunHealthCheckResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -28123,7 +32752,7 @@ func (m *ReloadSchemaResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28146,10 +32775,10 @@ func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -28186,7 +32815,7 @@ func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DurabilityPolicy", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -28214,47 +32843,8 @@ func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.WaitPosition = string(dAtA[iNdEx:postIndex]) + m.DurabilityPolicy = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludePrimary", 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.IncludePrimary = bool(v != 0) - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) - } - m.Concurrency = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Concurrency |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28277,7 +32867,7 @@ func (m *ReloadSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceDurabilityPolicyResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28300,15 +32890,15 @@ func (m *ReloadSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -28335,8 +32925,10 @@ func (m *ReloadSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Events = append(m.Events, &logutil.Event{}) - if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Keyspace == nil { + m.Keyspace = &topodata.Keyspace{} + } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -28362,7 +32954,7 @@ func (m *ReloadSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28385,10 +32977,10 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceServedFromRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceServedFromRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -28424,10 +33016,10 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) } - var stringLen uint64 + m.TabletType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -28437,27 +33029,14 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.TabletType |= topodata.TabletType(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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitPosition", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -28485,11 +33064,11 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.WaitPosition = string(dAtA[iNdEx:postIndex]) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludePrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Remove", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -28506,12 +33085,12 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { break } } - m.IncludePrimary = bool(v != 0) + m.Remove = bool(v != 0) case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrency", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", wireType) } - m.Concurrency = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -28521,11 +33100,24 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Concurrency |= uint32(b&0x7F) << shift + 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.SourceKeyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28548,7 +33140,7 @@ func (m *ReloadSchemaShardRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReloadSchemaShardResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceServedFromResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28571,15 +33163,15 @@ func (m *ReloadSchemaShardResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReloadSchemaShardResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceServedFromResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReloadSchemaShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceServedFromResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 2: + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -28606,8 +33198,10 @@ func (m *ReloadSchemaShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Events = append(m.Events, &logutil.Event{}) - if err := m.Events[len(m.Events)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Keyspace == nil { + m.Keyspace = &topodata.Keyspace{} + } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -28633,7 +33227,7 @@ func (m *ReloadSchemaShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceShardingInfoRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28656,10 +33250,10 @@ func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveBackupRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceShardingInfoRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceShardingInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -28694,43 +33288,11 @@ func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { } m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -28740,24 +33302,12 @@ func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.Force = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28780,7 +33330,7 @@ func (m *RemoveBackupRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveBackupResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetKeyspaceShardingInfoResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28803,12 +33353,48 @@ func (m *RemoveBackupResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveBackupResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetKeyspaceShardingInfoResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveBackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetKeyspaceShardingInfoResponse: 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 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.Keyspace == nil { + m.Keyspace = &topodata.Keyspace{} + } + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28831,7 +33417,7 @@ func (m *RemoveBackupResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetShardIsPrimaryServingRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -28854,10 +33440,10 @@ func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveKeyspaceCellRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetShardIsPrimaryServingRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveKeyspaceCellRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetShardIsPrimaryServingRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -28894,7 +33480,7 @@ func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -28922,31 +33508,11 @@ func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cell = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", 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.Force = bool(v != 0) - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IsServing", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -28963,7 +33529,7 @@ func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { break } } - m.Recursive = bool(v != 0) + m.IsServing = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28986,7 +33552,7 @@ func (m *RemoveKeyspaceCellRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveKeyspaceCellResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetShardIsPrimaryServingResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29009,12 +33575,48 @@ func (m *RemoveKeyspaceCellResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveKeyspaceCellResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetShardIsPrimaryServingResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveKeyspaceCellResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetShardIsPrimaryServingResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard == nil { + m.Shard = &topodata.Shard{} + } + if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -29037,7 +33639,7 @@ func (m *RemoveKeyspaceCellResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29060,10 +33662,10 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveShardCellRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetShardTabletControlRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveShardCellRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetShardTabletControlRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -29100,7 +33702,7 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -29128,11 +33730,30 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ShardName = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) + } + m.TabletType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletType |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -29160,11 +33781,43 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cell = string(dAtA[iNdEx:postIndex]) + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 4: + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DeniedTables", 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.DeniedTables = append(m.DeniedTables, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 6: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DisableQueryService", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -29181,10 +33834,10 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { break } } - m.Force = bool(v != 0) - case 5: + m.DisableQueryService = bool(v != 0) + case 7: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Remove", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -29201,7 +33854,7 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { break } } - m.Recursive = bool(v != 0) + m.Remove = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -29224,7 +33877,7 @@ func (m *RemoveShardCellRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RemoveShardCellResponse) UnmarshalVT(dAtA []byte) error { +func (m *SetShardTabletControlResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29247,12 +33900,48 @@ func (m *RemoveShardCellResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RemoveShardCellResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SetShardTabletControlResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RemoveShardCellResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetShardTabletControlResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard == nil { + m.Shard = &topodata.Shard{} + } + if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -29275,7 +33964,7 @@ func (m *RemoveShardCellResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReparentTabletRequest) UnmarshalVT(dAtA []byte) error { +func (m *SetWritableRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29298,15 +33987,15 @@ func (m *ReparentTabletRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReparentTabletRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SetWritableRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReparentTabletRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SetWritableRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -29330,16 +34019,87 @@ func (m *ReparentTabletRequest) UnmarshalVT(dAtA []byte) error { if postIndex < 0 { return ErrInvalidLength } - if postIndex > l { + 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 + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Writable", 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.Writable = bool(v != 0) + 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 *SetWritableResponse) 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 } - if m.Tablet == nil { - m.Tablet = &topodata.TabletAlias{} - } - if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - iNdEx = postIndex + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetWritableResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetWritableResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -29362,7 +34122,7 @@ func (m *ReparentTabletRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationAddRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29385,10 +34145,10 @@ func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ReparentTabletResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationAddRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ReparentTabletResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationAddRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -29457,7 +34217,7 @@ func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Primary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -29484,10 +34244,10 @@ func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Primary == nil { - m.Primary = &topodata.TabletAlias{} + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - if err := m.Primary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -29513,7 +34273,7 @@ func (m *ReparentTabletResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationAddResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29536,53 +34296,68 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RestoreFromBackupRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationAddResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RestoreFromBackupRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationAddResponse: 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 + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ShardReplicationFixRequest) 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 err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + if iNdEx >= l { + return io.ErrUnexpectedEOF } - iNdEx = postIndex - case 2: + 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: ShardReplicationFixRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardReplicationFixRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BackupTime", 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 @@ -29592,31 +34367,27 @@ func (m *RestoreFromBackupRequest) 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.BackupTime == nil { - m.BackupTime = &vttime.Time{} - } - if err := m.BackupTime.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RestoreToPos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -29644,33 +34415,13 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.RestoreToPos = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DryRun", 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.DryRun = bool(v != 0) - case 5: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RestoreToTimestamp", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -29680,27 +34431,23 @@ func (m *RestoreFromBackupRequest) 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.RestoreToTimestamp == nil { - m.RestoreToTimestamp = &vttime.Time{} - } - if err := m.RestoreToTimestamp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Cell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -29724,7 +34471,7 @@ func (m *RestoreFromBackupRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationFixResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29747,15 +34494,15 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RestoreFromBackupResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationFixResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RestoreFromBackupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationFixResponse: 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 Error", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -29782,14 +34529,65 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + if m.Error == nil { + m.Error = &topodata.ShardReplicationError{} } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Error.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + 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 *ShardReplicationPositionsRequest) 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: ShardReplicationPositionsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardReplicationPositionsRequest: 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) } @@ -29821,7 +34619,7 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { } m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } @@ -29853,42 +34651,6 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { } m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Event", 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.Event == nil { - m.Event = &logutil.Event{} - } - if err := m.Event.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -29911,7 +34673,7 @@ func (m *RestoreFromBackupResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationPositionsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -29934,15 +34696,15 @@ func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RunHealthCheckRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationPositionsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RunHealthCheckRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationPositionsResponse: 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 ReplicationStatuses", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -29969,12 +34731,234 @@ func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + if m.ReplicationStatuses == nil { + m.ReplicationStatuses = make(map[string]*replicationdata.Status) } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey string + var mapvalue *replicationdata.Status + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &replicationdata.Status{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.ReplicationStatuses[mapkey] = mapvalue + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletMap", 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.TabletMap == nil { + m.TabletMap = make(map[string]*topodata.Tablet) + } + var mapkey string + var mapvalue *topodata.Tablet + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &topodata.Tablet{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.TabletMap[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -29998,58 +34982,7 @@ func (m *RunHealthCheckRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *RunHealthCheckResponse) 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: RunHealthCheckResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RunHealthCheckResponse: 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 *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30072,10 +35005,10 @@ func (m *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationRemoveRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationRemoveRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -30112,7 +35045,7 @@ func (m *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DurabilityPolicy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -30140,7 +35073,43 @@ func (m *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.DurabilityPolicy = string(dAtA[iNdEx:postIndex]) + m.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + 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 @@ -30164,7 +35133,7 @@ func (m *SetKeyspaceDurabilityPolicyRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetKeyspaceDurabilityPolicyResponse) UnmarshalVT(dAtA []byte) error { +func (m *ShardReplicationRemoveResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30187,48 +35156,12 @@ func (m *SetKeyspaceDurabilityPolicyResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ShardReplicationRemoveResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceDurabilityPolicyResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShardReplicationRemoveResponse: 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 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.Keyspace == nil { - m.Keyspace = &topodata.Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -30251,7 +35184,7 @@ func (m *SetKeyspaceDurabilityPolicyResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { +func (m *SleepTabletRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30274,17 +35207,17 @@ func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceServedFromRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SleepTabletRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceServedFromRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SleepTabletRequest: 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 TabletAlias", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30294,48 +35227,33 @@ func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - m.TabletType = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TabletType |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - case 3: + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30345,75 +35263,27 @@ func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Remove", 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.Remove = bool(v != 0) - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", 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 m.Duration == nil { + m.Duration = &vttime.Duration{} } - if postIndex > l { - return io.ErrUnexpectedEOF + if err := m.Duration.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.SourceKeyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -30437,7 +35307,7 @@ func (m *SetKeyspaceServedFromRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetKeyspaceServedFromResponse) UnmarshalVT(dAtA []byte) error { +func (m *SleepTabletResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30460,48 +35330,12 @@ func (m *SetKeyspaceServedFromResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceServedFromResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SleepTabletResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceServedFromResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SleepTabletResponse: 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 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.Keyspace == nil { - m.Keyspace = &topodata.Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -30524,7 +35358,7 @@ func (m *SetKeyspaceServedFromResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetKeyspaceShardingInfoRequest) UnmarshalVT(dAtA []byte) error { +func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30547,10 +35381,10 @@ func (m *SetKeyspaceShardingInfoRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceShardingInfoRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SourceShardAddRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceShardingInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SourceShardAddRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -30585,11 +35419,11 @@ func (m *SetKeyspaceShardingInfoRequest) UnmarshalVT(dAtA []byte) error { } m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30599,68 +35433,48 @@ func (m *SetKeyspaceShardingInfoRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.Force = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - if (skippy < 0) || (iNdEx+skippy) < 0 { + postIndex := iNdEx + intStringLen + if postIndex < 0 { return ErrInvalidLength } - if (iNdEx + skippy) > l { + if postIndex > 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 *SetKeyspaceShardingInfoResponse) 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 + m.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Uid", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.Uid = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Uid |= int32(b&0x7F) << shift + if b < 0x80 { + break + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SetKeyspaceShardingInfoResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SetKeyspaceShardingInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30670,82 +35484,27 @@ func (m *SetKeyspaceShardingInfoResponse) 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.Keyspace == nil { - m.Keyspace = &topodata.Keyspace{} - } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.SourceKeyspace = 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 *SetShardIsPrimaryServingRequest) 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: SetShardIsPrimaryServingRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SetShardIsPrimaryServingRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourceShard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -30773,13 +35532,13 @@ func (m *SetShardIsPrimaryServingRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.SourceShard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeyRange", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30789,29 +35548,33 @@ func (m *SetShardIsPrimaryServingRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + if m.KeyRange == nil { + m.KeyRange = &topodata.KeyRange{} + } + if err := m.KeyRange.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsServing", wireType) + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -30821,12 +35584,24 @@ func (m *SetShardIsPrimaryServingRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.IsServing = bool(v != 0) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -30849,7 +35624,7 @@ func (m *SetShardIsPrimaryServingRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetShardIsPrimaryServingResponse) UnmarshalVT(dAtA []byte) error { +func (m *SourceShardAddResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30872,10 +35647,10 @@ func (m *SetShardIsPrimaryServingResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetShardIsPrimaryServingResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SourceShardAddResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetShardIsPrimaryServingResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SourceShardAddResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -30936,7 +35711,7 @@ func (m *SetShardIsPrimaryServingResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { +func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30959,10 +35734,10 @@ func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetShardTabletControlRequest: wiretype end group for non-group") + return fmt.Errorf("proto: SourceShardDeleteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetShardTabletControlRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SourceShardDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -31031,112 +35806,9 @@ func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletType", wireType) - } - m.TabletType = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TabletType |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", 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.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DeniedTables", 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.DeniedTables = append(m.DeniedTables, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DisableQueryService", 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.DisableQueryService = bool(v != 0) - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Remove", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Uid", wireType) } - var v int + m.Uid = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -31146,12 +35818,11 @@ func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.Uid |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Remove = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -31174,7 +35845,7 @@ func (m *SetShardTabletControlRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetShardTabletControlResponse) UnmarshalVT(dAtA []byte) error { +func (m *SourceShardDeleteResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31197,10 +35868,10 @@ func (m *SetShardTabletControlResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetShardTabletControlResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SourceShardDeleteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetShardTabletControlResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SourceShardDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -31261,7 +35932,7 @@ func (m *SetShardTabletControlResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetWritableRequest) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31284,10 +35955,10 @@ func (m *SetWritableRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetWritableRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetWritableRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -31326,26 +35997,6 @@ func (m *SetWritableRequest) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Writable", 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.Writable = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -31368,7 +36019,7 @@ func (m *SetWritableRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SetWritableResponse) UnmarshalVT(dAtA []byte) error { +func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31391,10 +36042,10 @@ func (m *SetWritableResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SetWritableResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StartReplicationResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SetWritableResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { default: @@ -31419,7 +36070,7 @@ func (m *SetWritableResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationAddRequest) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31442,77 +36093,13 @@ func (m *ShardReplicationAddRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationAddRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StopReplicationRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationAddRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StopReplicationRequest: 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 - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } @@ -31570,7 +36157,7 @@ func (m *ShardReplicationAddRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationAddResponse) UnmarshalVT(dAtA []byte) error { +func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31580,25 +36167,112 @@ func (m *ShardReplicationAddResponse) UnmarshalVT(dAtA []byte) error { if shift >= 64 { return ErrIntOverflow } - if iNdEx >= l { + 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: StopReplicationResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StopReplicationResponse: 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 *TabletExternallyReparentedRequest) 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: TabletExternallyReparentedRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TabletExternallyReparentedRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", 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 } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationAddResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationAddResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -31621,7 +36295,7 @@ func (m *ShardReplicationAddResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationFixRequest) UnmarshalVT(dAtA []byte) error { +func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31644,10 +36318,10 @@ func (m *ShardReplicationFixRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationFixRequest: wiretype end group for non-group") + return fmt.Errorf("proto: TabletExternallyReparentedResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationFixRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TabletExternallyReparentedResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -31716,9 +36390,9 @@ func (m *ShardReplicationFixRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cell", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -31728,78 +36402,31 @@ func (m *ShardReplicationFixRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Cell = 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 *ShardReplicationFixResponse) 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 + if m.NewPrimary == nil { + m.NewPrimary = &topodata.TabletAlias{} } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationFixResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationFixResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OldPrimary", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -31826,10 +36453,10 @@ func (m *ShardReplicationFixResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Error == nil { - m.Error = &topodata.ShardReplicationError{} + if m.OldPrimary == nil { + m.OldPrimary = &topodata.TabletAlias{} } - if err := m.Error.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.OldPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -31855,7 +36482,7 @@ func (m *ShardReplicationFixResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationPositionsRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateCellInfoRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31878,15 +36505,15 @@ func (m *ShardReplicationPositionsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationPositionsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateCellInfoRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationPositionsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateCellInfoRequest: 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 Name", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -31914,13 +36541,13 @@ func (m *ShardReplicationPositionsRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -31930,23 +36557,27 @@ func (m *ShardReplicationPositionsRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + if m.CellInfo == nil { + m.CellInfo = &topodata.CellInfo{} + } + if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -31970,7 +36601,7 @@ func (m *ShardReplicationPositionsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationPositionsResponse) UnmarshalVT(dAtA []byte) error { +func (m *UpdateCellInfoResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -31993,17 +36624,17 @@ func (m *ShardReplicationPositionsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationPositionsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateCellInfoResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationPositionsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReplicationStatuses", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -32013,124 +36644,27 @@ func (m *ShardReplicationPositionsResponse) 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.ReplicationStatuses == nil { - m.ReplicationStatuses = make(map[string]*replicationdata.Status) - } - var mapkey string - var mapvalue *replicationdata.Status - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &replicationdata.Status{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.ReplicationStatuses[mapkey] = mapvalue + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletMap", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -32157,105 +36691,12 @@ func (m *ShardReplicationPositionsResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletMap == nil { - m.TabletMap = make(map[string]*topodata.Tablet) + if m.CellInfo == nil { + m.CellInfo = &topodata.CellInfo{} } - var mapkey string - var mapvalue *topodata.Tablet - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &topodata.Tablet{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.TabletMap[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -32279,7 +36720,7 @@ func (m *ShardReplicationPositionsResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateCellsAliasRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -32302,15 +36743,15 @@ func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShardReplicationRemoveRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateCellsAliasRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationRemoveRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateCellsAliasRequest: 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 Name", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -32338,43 +36779,11 @@ func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CellsAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -32401,10 +36810,10 @@ func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + if m.CellsAlias == nil { + m.CellsAlias = &topodata.CellsAlias{} } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CellsAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -32430,58 +36839,7 @@ func (m *ShardReplicationRemoveRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ShardReplicationRemoveResponse) 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: ShardReplicationRemoveResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ShardReplicationRemoveResponse: 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 *SleepTabletRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -32504,17 +36862,17 @@ func (m *SleepTabletRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SleepTabletRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateCellsAliasResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SleepTabletRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateCellsAliasResponse: 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 Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -32524,31 +36882,27 @@ func (m *SleepTabletRequest) 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.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CellsAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -32575,10 +36929,10 @@ func (m *SleepTabletRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Duration == nil { - m.Duration = &vttime.Duration{} + if m.CellsAlias == nil { + m.CellsAlias = &topodata.CellsAlias{} } - if err := m.Duration.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.CellsAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -32604,7 +36958,7 @@ func (m *SleepTabletRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SleepTabletResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -32627,12 +36981,32 @@ func (m *SleepTabletResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SleepTabletResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SleepTabletResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", 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.PingTablets = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -32655,7 +37029,7 @@ func (m *SleepTabletResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -32678,15 +37052,15 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SourceShardAddRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SourceShardAddRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateResponse: 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 Results", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -32714,13 +37088,13 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResultsByKeyspace", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -32730,78 +37104,175 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Uid", wireType) + if m.ResultsByKeyspace == nil { + m.ResultsByKeyspace = make(map[string]*ValidateKeyspaceResponse) } - m.Uid = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF + var mapkey string + var mapvalue *ValidateKeyspaceResponse + for iNdEx < postIndex { + entryPreIndex := 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 + } } - b := dAtA[iNdEx] - iNdEx++ - m.Uid |= int32(b&0x7F) << shift - if b < 0x80 { - break + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ValidateKeyspaceResponse{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } } - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspace", 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 - } + m.ResultsByKeyspace[mapkey] = mapvalue + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - if postIndex > l { + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ValidateKeyspaceRequest) 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 } - m.SourceKeyspace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: + 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: ValidateKeyspaceRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ValidateKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -32829,49 +37300,13 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SourceShard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyRange", 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.KeyRange == nil { - m.KeyRange = &topodata.KeyRange{} - } - if err := m.KeyRange.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -32881,24 +37316,12 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(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.Tables = append(m.Tables, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex + m.PingTablets = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -32921,7 +37344,7 @@ func (m *SourceShardAddRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SourceShardAddResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -32944,15 +37367,47 @@ func (m *SourceShardAddResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SourceShardAddResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SourceShardAddResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Results", 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.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -32979,12 +37434,105 @@ func (m *SourceShardAddResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Shard == nil { - m.Shard = &topodata.Shard{} + if m.ResultsByShard == nil { + m.ResultsByShard = make(map[string]*ValidateShardResponse) } - if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey string + var mapvalue *ValidateShardResponse + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ValidateShardResponse{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.ResultsByShard[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -33008,7 +37556,7 @@ func (m *SourceShardAddResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33031,10 +37579,10 @@ func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SourceShardDeleteRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateSchemaKeyspaceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SourceShardDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateSchemaKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -33071,7 +37619,7 @@ func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -33099,13 +37647,13 @@ func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Uid", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", wireType) } - m.Uid = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -33115,67 +37663,37 @@ func (m *SourceShardDeleteRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Uid |= int32(b&0x7F) << shift + v |= int(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 { - return io.ErrUnexpectedEOF - } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SourceShardDeleteResponse) 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 + m.IncludeViews = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipNoPrimary", wireType) } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + 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 + } } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SourceShardDeleteResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SourceShardDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + m.SkipNoPrimary = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeVschema", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -33185,28 +37703,12 @@ func (m *SourceShardDeleteResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.Shard == nil { - m.Shard = &topodata.Shard{} - } - if err := m.Shard.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.IncludeVschema = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -33229,7 +37731,7 @@ func (m *SourceShardDeleteResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33252,15 +37754,47 @@ func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartReplicationRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateSchemaKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateSchemaKeyspaceResponse: 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 Results", 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.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33287,12 +37821,105 @@ func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} + if m.ResultsByShard == nil { + m.ResultsByShard = make(map[string]*ValidateShardResponse) } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey string + var mapvalue *ValidateShardResponse + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ValidateShardResponse{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.ResultsByShard[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -33316,7 +37943,7 @@ func (m *StartReplicationRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33339,68 +37966,49 @@ func (m *StartReplicationResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartReplicationResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength + 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 + } } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *StopReplicationRequest) 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 + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > 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: StopReplicationRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.Keyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -33410,28 +38018,44 @@ func (m *StopReplicationRequest) 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{} + m.Shard = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", wireType) } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + 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 + } } - iNdEx = postIndex + m.PingTablets = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -33454,7 +38078,7 @@ func (m *StopReplicationRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33477,12 +38101,44 @@ func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StopReplicationResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StopReplicationResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Results", 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.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -33505,7 +38161,7 @@ func (m *StopReplicationResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *TabletExternallyReparentedRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVersionKeyspaceRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33528,17 +38184,17 @@ func (m *TabletExternallyReparentedRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TabletExternallyReparentedRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVersionKeyspaceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TabletExternallyReparentedRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVersionKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tablet", 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 @@ -33548,27 +38204,23 @@ func (m *TabletExternallyReparentedRequest) 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.Tablet == nil { - m.Tablet = &topodata.TabletAlias{} - } - if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -33592,7 +38244,7 @@ func (m *TabletExternallyReparentedRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33615,15 +38267,15 @@ func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TabletExternallyReparentedResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVersionKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TabletExternallyReparentedResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVersionKeyspaceResponse: 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 Results", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -33651,43 +38303,11 @@ func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", 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.Shard = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field NewPrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -33714,48 +38334,105 @@ func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.NewPrimary == nil { - m.NewPrimary = &topodata.TabletAlias{} - } - if err := m.NewPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OldPrimary", wireType) + if m.ResultsByShard == nil { + m.ResultsByShard = make(map[string]*ValidateShardResponse) } - 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 + var mapkey string + var mapvalue *ValidateShardResponse + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ValidateShardResponse{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.OldPrimary == nil { - m.OldPrimary = &topodata.TabletAlias{} - } - if err := m.OldPrimary.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.ResultsByShard[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -33779,7 +38456,7 @@ func (m *TabletExternallyReparentedResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateCellInfoRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVersionShardRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33802,15 +38479,15 @@ func (m *UpdateCellInfoRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateCellInfoRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVersionShardRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateCellInfoRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVersionShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -33838,13 +38515,13 @@ func (m *UpdateCellInfoRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -33854,27 +38531,23 @@ func (m *UpdateCellInfoRequest) 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.CellInfo == nil { - m.CellInfo = &topodata.CellInfo{} - } - if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -33898,7 +38571,7 @@ func (m *UpdateCellInfoRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateCellInfoResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVersionShardResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -33921,15 +38594,15 @@ func (m *UpdateCellInfoResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateCellInfoResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVersionShardResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateCellInfoResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVersionShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -33957,43 +38630,7 @@ func (m *UpdateCellInfoResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellInfo", 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.CellInfo == nil { - m.CellInfo = &topodata.CellInfo{} - } - if err := m.CellInfo.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -34017,7 +38654,7 @@ func (m *UpdateCellInfoResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateCellsAliasRequest) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34040,15 +38677,15 @@ func (m *UpdateCellsAliasRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateCellsAliasRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVSchemaRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateCellsAliasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -34076,13 +38713,13 @@ func (m *UpdateCellsAliasRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellsAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34092,28 +38729,76 @@ func (m *UpdateCellsAliasRequest) 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.CellsAlias == nil { - m.CellsAlias = &topodata.CellsAlias{} + m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) } - if err := m.CellsAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + 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.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", 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.IncludeViews = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -34136,7 +38821,7 @@ func (m *UpdateCellsAliasRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { +func (m *ValidateVSchemaResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34159,15 +38844,15 @@ func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateCellsAliasResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ValidateVSchemaResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateCellsAliasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ValidateVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -34195,11 +38880,11 @@ func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CellsAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -34226,12 +38911,105 @@ func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.CellsAlias == nil { - m.CellsAlias = &topodata.CellsAlias{} + if m.ResultsByShard == nil { + m.ResultsByShard = make(map[string]*ValidateShardResponse) } - if err := m.CellsAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey string + var mapvalue *ValidateShardResponse + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ValidateShardResponse{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.ResultsByShard[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -34255,7 +39033,7 @@ func (m *UpdateCellsAliasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateRequest) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowDeleteRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34278,15 +39056,79 @@ func (m *ValidateRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowDeleteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowDeleteRequest: 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 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", 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.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeepData", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -34303,7 +39145,27 @@ func (m *ValidateRequest) UnmarshalVT(dAtA []byte) error { break } } - m.PingTablets = bool(v != 0) + m.KeepData = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeepRoutingRules", 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.KeepRoutingRules = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -34326,7 +39188,7 @@ func (m *ValidateRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowDeleteResponse_TabletInfo) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34349,17 +39211,17 @@ func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowDeleteResponse_TabletInfo: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowDeleteResponse_TabletInfo: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34369,29 +39231,33 @@ func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + if m.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} + } + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultsByKeyspace", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Deleted", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34401,121 +39267,12 @@ func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= 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.ResultsByKeyspace == nil { - m.ResultsByKeyspace = make(map[string]*ValidateKeyspaceResponse) - } - var mapkey string - var mapvalue *ValidateKeyspaceResponse - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &ValidateKeyspaceResponse{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.ResultsByKeyspace[mapkey] = mapvalue - iNdEx = postIndex + m.Deleted = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -34538,7 +39295,7 @@ func (m *ValidateResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowDeleteResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34561,15 +39318,15 @@ func (m *ValidateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowDeleteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowDeleteResponse: 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 Summary", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -34597,13 +39354,13 @@ func (m *ValidateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Summary = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Details", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34613,12 +39370,26 @@ func (m *ValidateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.PingTablets = bool(v != 0) + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Details = append(m.Details, &WorkflowDeleteResponse_TabletInfo{}) + if err := m.Details[len(m.Details)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -34641,7 +39412,7 @@ func (m *ValidateKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowStatusRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34664,15 +39435,15 @@ func (m *ValidateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowStatusRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -34700,13 +39471,13 @@ func (m *ValidateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34716,120 +39487,23 @@ func (m *ValidateKeyspaceResponse) 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.ResultsByShard == nil { - m.ResultsByShard = make(map[string]*ValidateShardResponse) - } - var mapkey string - var mapvalue *ValidateShardResponse - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &ValidateShardResponse{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.ResultsByShard[mapkey] = mapvalue + m.Workflow = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -34853,7 +39527,7 @@ func (m *ValidateKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowStatusResponse_TableCopyState) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -34876,17 +39550,17 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateSchemaKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowStatusResponse_TableCopyState: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateSchemaKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowStatusResponse_TableCopyState: 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) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsCopied", wireType) } - var stringLen uint64 + m.RowsCopied = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34896,29 +39570,16 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.RowsCopied |= int64(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 case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsTotal", wireType) } - var stringLen uint64 + m.RowsTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34928,49 +39589,27 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.RowsTotal |= int64(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.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", wireType) + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsPercentage", 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 - } + var v uint32 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF } - m.IncludeViews = bool(v != 0) + v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + m.RowsPercentage = float32(math.Float32frombits(v)) case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SkipNoPrimary", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field BytesCopied", wireType) } - var v int + m.BytesCopied = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -34980,17 +39619,16 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.BytesCopied |= int64(b&0x7F) << shift if b < 0x80 { break } } - m.SkipNoPrimary = bool(v != 0) case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeVschema", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field BytesTotal", wireType) } - var v int + m.BytesTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35000,12 +39638,22 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.BytesTotal |= int64(b&0x7F) << shift if b < 0x80 { break } } - m.IncludeVschema = bool(v != 0) + case 6: + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesPercentage", wireType) + } + var v uint32 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF + } + v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + m.BytesPercentage = float32(math.Float32frombits(v)) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -35028,7 +39676,7 @@ func (m *ValidateSchemaKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowStatusResponse_ShardStreamState) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -35051,17 +39699,17 @@ func (m *ValidateSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateSchemaKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowStatusResponse_ShardStreamState: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateSchemaKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowStatusResponse_ShardStreamState: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) } - var stringLen uint64 + m.Id = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35071,27 +39719,14 @@ func (m *ValidateSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.Id |= int32(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.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -35118,160 +39753,16 @@ func (m *ValidateSchemaKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ResultsByShard == nil { - m.ResultsByShard = make(map[string]*ValidateShardResponse) - } - var mapkey string - var mapvalue *ValidateShardResponse - for iNdEx < postIndex { - entryPreIndex := 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) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &ValidateShardResponse{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } + if m.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} } - m.ResultsByShard[mapkey] = mapvalue - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + if err := m.Tablet.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 *ValidateShardRequest) 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: ValidateShardRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourceShard", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -35299,11 +39790,11 @@ func (m *ValidateShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.SourceShard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -35331,13 +39822,13 @@ func (m *ValidateShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.Position = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PingTablets", wireType) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35347,66 +39838,27 @@ func (m *ValidateShardRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.PingTablets = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { + intStringLen := int(stringLen) + if intStringLen < 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 *ValidateShardResponse) 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 + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > 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: ValidateShardResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Info", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -35434,7 +39886,7 @@ func (m *ValidateShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + m.Info = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -35458,7 +39910,7 @@ func (m *ValidateShardResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateVersionKeyspaceRequest) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowStatusResponse_ShardStreams) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -35481,17 +39933,17 @@ func (m *ValidateVersionKeyspaceRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateVersionKeyspaceRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowStatusResponse_ShardStreams: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVersionKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowStatusResponse_ShardStreams: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35501,23 +39953,25 @@ func (m *ValidateVersionKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + m.Streams = append(m.Streams, &WorkflowStatusResponse_ShardStreamState{}) + if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -35541,7 +39995,7 @@ func (m *ValidateVersionKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowStatusResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -35564,17 +40018,17 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateVersionKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowStatusResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVersionKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableCopyState", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35584,27 +40038,124 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(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.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TableCopyState == nil { + m.TableCopyState = make(map[string]*WorkflowStatusResponse_TableCopyState) + } + var mapkey string + var mapvalue *WorkflowStatusResponse_TableCopyState + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLength + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLength + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLength + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLength + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &WorkflowStatusResponse_TableCopyState{} + if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.TableCopyState[mapkey] = mapvalue iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ShardStreams", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -35631,11 +40182,11 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ResultsByShard == nil { - m.ResultsByShard = make(map[string]*ValidateShardResponse) + if m.ShardStreams == nil { + m.ShardStreams = make(map[string]*WorkflowStatusResponse_ShardStreams) } var mapkey string - var mapvalue *ValidateShardResponse + var mapvalue *WorkflowStatusResponse_ShardStreams for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -35709,7 +40260,7 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &ValidateShardResponse{} + mapvalue = &WorkflowStatusResponse_ShardStreams{} if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -35729,7 +40280,7 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { iNdEx += skippy } } - m.ResultsByShard[mapkey] = mapvalue + m.ShardStreams[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -35753,7 +40304,7 @@ func (m *ValidateVersionKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateVersionShardRequest) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowSwitchTrafficRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -35776,10 +40327,10 @@ func (m *ValidateVersionShardRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateVersionShardRequest: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowSwitchTrafficRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVersionShardRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowSwitchTrafficRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -35816,7 +40367,7 @@ func (m *ValidateVersionShardRequest) UnmarshalVT(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -35844,62 +40395,11 @@ func (m *ValidateVersionShardRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shard = string(dAtA[iNdEx:postIndex]) + m.Workflow = 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 *ValidateVersionShardResponse) 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: ValidateVersionShardResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVersionShardResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -35927,64 +40427,82 @@ func (m *ValidateVersionShardResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + m.Cells = append(m.Cells, 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 *ValidateVSchemaRequest) 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 + case 4: + if wireType == 0 { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ValidateVSchemaRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVSchemaRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxReplicationLagAllowed", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -35994,29 +40512,33 @@ func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspace = string(dAtA[iNdEx:postIndex]) + if m.MaxReplicationLagAllowed == nil { + m.MaxReplicationLagAllowed = &vttime.Duration{} + } + if err := m.MaxReplicationLagAllowed.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EnableReverseReplication", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -36026,29 +40548,36 @@ func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + m.EnableReverseReplication = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) } - if postIndex > l { - return io.ErrUnexpectedEOF + m.Direction = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Direction |= int32(b&0x7F) << shift + if b < 0x80 { + break + } } - m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 3: + case 8: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExcludeTables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Timeout", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -36058,27 +40587,31 @@ func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.ExcludeTables = append(m.ExcludeTables, string(dAtA[iNdEx:postIndex])) + if m.Timeout == nil { + m.Timeout = &vttime.Duration{} + } + if err := m.Timeout.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 4: + case 9: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IncludeViews", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DryRun", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -36095,7 +40628,7 @@ func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { break } } - m.IncludeViews = bool(v != 0) + m.DryRun = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -36118,7 +40651,7 @@ func (m *ValidateVSchemaRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ValidateVSchemaResponse) UnmarshalVT(dAtA []byte) error { +func (m *WorkflowSwitchTrafficResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -36141,15 +40674,15 @@ func (m *ValidateVSchemaResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValidateVSchemaResponse: wiretype end group for non-group") + return fmt.Errorf("proto: WorkflowSwitchTrafficResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValidateVSchemaResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: WorkflowSwitchTrafficResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Summary", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -36177,13 +40710,13 @@ func (m *ValidateVSchemaResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, string(dAtA[iNdEx:postIndex])) + m.Summary = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultsByShard", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartState", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -36193,120 +40726,87 @@ func (m *ValidateVSchemaResponse) 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.ResultsByShard == nil { - m.ResultsByShard = make(map[string]*ValidateShardResponse) + m.StartState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentState", wireType) } - var mapkey string - var mapvalue *ValidateShardResponse - for iNdEx < postIndex { - entryPreIndex := 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 - } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow } - fieldNum := int32(wire >> 3) - if fieldNum == 1 { - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLength - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey < 0 { - return ErrInvalidLength - } - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLength - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLength - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &ValidateShardResponse{} - if err := mapvalue.UnmarshalVT(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLength - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break } } - m.ResultsByShard[mapkey] = mapvalue + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CurrentState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DryRunResults", 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.DryRunResults = append(m.DryRunResults, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -36421,7 +40921,7 @@ func (m *WorkflowUpdateRequest) UnmarshalVT(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TabletRequest == nil { - m.TabletRequest = &tabletmanagerdata.UpdateVRWorkflowRequest{} + m.TabletRequest = &tabletmanagerdata.UpdateVReplicationWorkflowRequest{} } if err := m.TabletRequest.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err @@ -36482,7 +40982,7 @@ func (m *WorkflowUpdateResponse_TabletInfo) UnmarshalVT(dAtA []byte) error { if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Tablet", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -36492,23 +40992,27 @@ func (m *WorkflowUpdateResponse_TabletInfo) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Tablet = string(dAtA[iNdEx:postIndex]) + if m.Tablet == nil { + m.Tablet = &topodata.TabletAlias{} + } + if err := m.Tablet.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: if wireType != 0 { diff --git a/go/vt/proto/vtctlservice/vtctlservice.pb.go b/go/vt/proto/vtctlservice/vtctlservice.pb.go index 31efb4bd810..ed653012f10 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, 0xea, 0x3d, 0x0a, 0x06, 0x56, 0x74, 0x63, 0x74, 0x6c, + 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x32, 0xcc, 0x41, 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, @@ -306,250 +306,280 @@ var file_vtctlservice_proto_rawDesc = []byte{ 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, 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, + 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, - 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, + 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, - 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, + 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, - 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, + 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, 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, 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, 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, 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, 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, + 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, 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, 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, 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, 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, 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, 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, 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, + 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, 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, 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, + 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, 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, 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, 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, + 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, 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, + 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, 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, + 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, 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, 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, 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, + 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, 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, 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, + 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, 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, 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, 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, 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, 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, } var file_vtctlservice_proto_goTypes = []interface{}{ @@ -600,130 +630,139 @@ var file_vtctlservice_proto_goTypes = []interface{}{ (*vtctldata.GetVSchemaRequest)(nil), // 44: vtctldata.GetVSchemaRequest (*vtctldata.GetWorkflowsRequest)(nil), // 45: vtctldata.GetWorkflowsRequest (*vtctldata.InitShardPrimaryRequest)(nil), // 46: vtctldata.InitShardPrimaryRequest - (*vtctldata.PingTabletRequest)(nil), // 47: vtctldata.PingTabletRequest - (*vtctldata.PlannedReparentShardRequest)(nil), // 48: vtctldata.PlannedReparentShardRequest - (*vtctldata.RebuildKeyspaceGraphRequest)(nil), // 49: vtctldata.RebuildKeyspaceGraphRequest - (*vtctldata.RebuildVSchemaGraphRequest)(nil), // 50: vtctldata.RebuildVSchemaGraphRequest - (*vtctldata.RefreshStateRequest)(nil), // 51: vtctldata.RefreshStateRequest - (*vtctldata.RefreshStateByShardRequest)(nil), // 52: vtctldata.RefreshStateByShardRequest - (*vtctldata.ReloadSchemaRequest)(nil), // 53: vtctldata.ReloadSchemaRequest - (*vtctldata.ReloadSchemaKeyspaceRequest)(nil), // 54: vtctldata.ReloadSchemaKeyspaceRequest - (*vtctldata.ReloadSchemaShardRequest)(nil), // 55: vtctldata.ReloadSchemaShardRequest - (*vtctldata.RemoveBackupRequest)(nil), // 56: vtctldata.RemoveBackupRequest - (*vtctldata.RemoveKeyspaceCellRequest)(nil), // 57: vtctldata.RemoveKeyspaceCellRequest - (*vtctldata.RemoveShardCellRequest)(nil), // 58: vtctldata.RemoveShardCellRequest - (*vtctldata.ReparentTabletRequest)(nil), // 59: vtctldata.ReparentTabletRequest - (*vtctldata.RestoreFromBackupRequest)(nil), // 60: vtctldata.RestoreFromBackupRequest - (*vtctldata.RunHealthCheckRequest)(nil), // 61: vtctldata.RunHealthCheckRequest - (*vtctldata.SetKeyspaceDurabilityPolicyRequest)(nil), // 62: vtctldata.SetKeyspaceDurabilityPolicyRequest - (*vtctldata.SetShardIsPrimaryServingRequest)(nil), // 63: vtctldata.SetShardIsPrimaryServingRequest - (*vtctldata.SetShardTabletControlRequest)(nil), // 64: vtctldata.SetShardTabletControlRequest - (*vtctldata.SetWritableRequest)(nil), // 65: vtctldata.SetWritableRequest - (*vtctldata.ShardReplicationAddRequest)(nil), // 66: vtctldata.ShardReplicationAddRequest - (*vtctldata.ShardReplicationFixRequest)(nil), // 67: vtctldata.ShardReplicationFixRequest - (*vtctldata.ShardReplicationPositionsRequest)(nil), // 68: vtctldata.ShardReplicationPositionsRequest - (*vtctldata.ShardReplicationRemoveRequest)(nil), // 69: vtctldata.ShardReplicationRemoveRequest - (*vtctldata.SleepTabletRequest)(nil), // 70: vtctldata.SleepTabletRequest - (*vtctldata.SourceShardAddRequest)(nil), // 71: vtctldata.SourceShardAddRequest - (*vtctldata.SourceShardDeleteRequest)(nil), // 72: vtctldata.SourceShardDeleteRequest - (*vtctldata.StartReplicationRequest)(nil), // 73: vtctldata.StartReplicationRequest - (*vtctldata.StopReplicationRequest)(nil), // 74: vtctldata.StopReplicationRequest - (*vtctldata.TabletExternallyReparentedRequest)(nil), // 75: vtctldata.TabletExternallyReparentedRequest - (*vtctldata.UpdateCellInfoRequest)(nil), // 76: vtctldata.UpdateCellInfoRequest - (*vtctldata.UpdateCellsAliasRequest)(nil), // 77: vtctldata.UpdateCellsAliasRequest - (*vtctldata.ValidateRequest)(nil), // 78: vtctldata.ValidateRequest - (*vtctldata.ValidateKeyspaceRequest)(nil), // 79: vtctldata.ValidateKeyspaceRequest - (*vtctldata.ValidateSchemaKeyspaceRequest)(nil), // 80: vtctldata.ValidateSchemaKeyspaceRequest - (*vtctldata.ValidateShardRequest)(nil), // 81: vtctldata.ValidateShardRequest - (*vtctldata.ValidateVersionKeyspaceRequest)(nil), // 82: vtctldata.ValidateVersionKeyspaceRequest - (*vtctldata.ValidateVersionShardRequest)(nil), // 83: vtctldata.ValidateVersionShardRequest - (*vtctldata.ValidateVSchemaRequest)(nil), // 84: vtctldata.ValidateVSchemaRequest - (*vtctldata.WorkflowUpdateRequest)(nil), // 85: vtctldata.WorkflowUpdateRequest - (*vtctldata.ExecuteVtctlCommandResponse)(nil), // 86: vtctldata.ExecuteVtctlCommandResponse - (*vtctldata.AddCellInfoResponse)(nil), // 87: vtctldata.AddCellInfoResponse - (*vtctldata.AddCellsAliasResponse)(nil), // 88: vtctldata.AddCellsAliasResponse - (*vtctldata.ApplyRoutingRulesResponse)(nil), // 89: vtctldata.ApplyRoutingRulesResponse - (*vtctldata.ApplySchemaResponse)(nil), // 90: vtctldata.ApplySchemaResponse - (*vtctldata.ApplyShardRoutingRulesResponse)(nil), // 91: vtctldata.ApplyShardRoutingRulesResponse - (*vtctldata.ApplyVSchemaResponse)(nil), // 92: vtctldata.ApplyVSchemaResponse - (*vtctldata.BackupResponse)(nil), // 93: vtctldata.BackupResponse - (*vtctldata.ChangeTabletTypeResponse)(nil), // 94: vtctldata.ChangeTabletTypeResponse - (*vtctldata.CreateKeyspaceResponse)(nil), // 95: vtctldata.CreateKeyspaceResponse - (*vtctldata.CreateShardResponse)(nil), // 96: vtctldata.CreateShardResponse - (*vtctldata.DeleteCellInfoResponse)(nil), // 97: vtctldata.DeleteCellInfoResponse - (*vtctldata.DeleteCellsAliasResponse)(nil), // 98: vtctldata.DeleteCellsAliasResponse - (*vtctldata.DeleteKeyspaceResponse)(nil), // 99: vtctldata.DeleteKeyspaceResponse - (*vtctldata.DeleteShardsResponse)(nil), // 100: vtctldata.DeleteShardsResponse - (*vtctldata.DeleteSrvVSchemaResponse)(nil), // 101: vtctldata.DeleteSrvVSchemaResponse - (*vtctldata.DeleteTabletsResponse)(nil), // 102: vtctldata.DeleteTabletsResponse - (*vtctldata.EmergencyReparentShardResponse)(nil), // 103: vtctldata.EmergencyReparentShardResponse - (*vtctldata.ExecuteFetchAsAppResponse)(nil), // 104: vtctldata.ExecuteFetchAsAppResponse - (*vtctldata.ExecuteFetchAsDBAResponse)(nil), // 105: vtctldata.ExecuteFetchAsDBAResponse - (*vtctldata.ExecuteHookResponse)(nil), // 106: vtctldata.ExecuteHookResponse - (*vtctldata.FindAllShardsInKeyspaceResponse)(nil), // 107: vtctldata.FindAllShardsInKeyspaceResponse - (*vtctldata.GetBackupsResponse)(nil), // 108: vtctldata.GetBackupsResponse - (*vtctldata.GetCellInfoResponse)(nil), // 109: vtctldata.GetCellInfoResponse - (*vtctldata.GetCellInfoNamesResponse)(nil), // 110: vtctldata.GetCellInfoNamesResponse - (*vtctldata.GetCellsAliasesResponse)(nil), // 111: vtctldata.GetCellsAliasesResponse - (*vtctldata.GetFullStatusResponse)(nil), // 112: vtctldata.GetFullStatusResponse - (*vtctldata.GetKeyspaceResponse)(nil), // 113: vtctldata.GetKeyspaceResponse - (*vtctldata.GetKeyspacesResponse)(nil), // 114: vtctldata.GetKeyspacesResponse - (*vtctldata.GetPermissionsResponse)(nil), // 115: vtctldata.GetPermissionsResponse - (*vtctldata.GetRoutingRulesResponse)(nil), // 116: vtctldata.GetRoutingRulesResponse - (*vtctldata.GetSchemaResponse)(nil), // 117: vtctldata.GetSchemaResponse - (*vtctldata.GetShardResponse)(nil), // 118: vtctldata.GetShardResponse - (*vtctldata.GetShardRoutingRulesResponse)(nil), // 119: vtctldata.GetShardRoutingRulesResponse - (*vtctldata.GetSrvKeyspaceNamesResponse)(nil), // 120: vtctldata.GetSrvKeyspaceNamesResponse - (*vtctldata.GetSrvKeyspacesResponse)(nil), // 121: vtctldata.GetSrvKeyspacesResponse - (*vtctldata.UpdateThrottlerConfigResponse)(nil), // 122: vtctldata.UpdateThrottlerConfigResponse - (*vtctldata.GetSrvVSchemaResponse)(nil), // 123: vtctldata.GetSrvVSchemaResponse - (*vtctldata.GetSrvVSchemasResponse)(nil), // 124: vtctldata.GetSrvVSchemasResponse - (*vtctldata.GetTabletResponse)(nil), // 125: vtctldata.GetTabletResponse - (*vtctldata.GetTabletsResponse)(nil), // 126: vtctldata.GetTabletsResponse - (*vtctldata.GetTopologyPathResponse)(nil), // 127: vtctldata.GetTopologyPathResponse - (*vtctldata.GetVersionResponse)(nil), // 128: vtctldata.GetVersionResponse - (*vtctldata.GetVSchemaResponse)(nil), // 129: vtctldata.GetVSchemaResponse - (*vtctldata.GetWorkflowsResponse)(nil), // 130: vtctldata.GetWorkflowsResponse - (*vtctldata.InitShardPrimaryResponse)(nil), // 131: vtctldata.InitShardPrimaryResponse - (*vtctldata.PingTabletResponse)(nil), // 132: vtctldata.PingTabletResponse - (*vtctldata.PlannedReparentShardResponse)(nil), // 133: vtctldata.PlannedReparentShardResponse - (*vtctldata.RebuildKeyspaceGraphResponse)(nil), // 134: vtctldata.RebuildKeyspaceGraphResponse - (*vtctldata.RebuildVSchemaGraphResponse)(nil), // 135: vtctldata.RebuildVSchemaGraphResponse - (*vtctldata.RefreshStateResponse)(nil), // 136: vtctldata.RefreshStateResponse - (*vtctldata.RefreshStateByShardResponse)(nil), // 137: vtctldata.RefreshStateByShardResponse - (*vtctldata.ReloadSchemaResponse)(nil), // 138: vtctldata.ReloadSchemaResponse - (*vtctldata.ReloadSchemaKeyspaceResponse)(nil), // 139: vtctldata.ReloadSchemaKeyspaceResponse - (*vtctldata.ReloadSchemaShardResponse)(nil), // 140: vtctldata.ReloadSchemaShardResponse - (*vtctldata.RemoveBackupResponse)(nil), // 141: vtctldata.RemoveBackupResponse - (*vtctldata.RemoveKeyspaceCellResponse)(nil), // 142: vtctldata.RemoveKeyspaceCellResponse - (*vtctldata.RemoveShardCellResponse)(nil), // 143: vtctldata.RemoveShardCellResponse - (*vtctldata.ReparentTabletResponse)(nil), // 144: vtctldata.ReparentTabletResponse - (*vtctldata.RestoreFromBackupResponse)(nil), // 145: vtctldata.RestoreFromBackupResponse - (*vtctldata.RunHealthCheckResponse)(nil), // 146: vtctldata.RunHealthCheckResponse - (*vtctldata.SetKeyspaceDurabilityPolicyResponse)(nil), // 147: vtctldata.SetKeyspaceDurabilityPolicyResponse - (*vtctldata.SetShardIsPrimaryServingResponse)(nil), // 148: vtctldata.SetShardIsPrimaryServingResponse - (*vtctldata.SetShardTabletControlResponse)(nil), // 149: vtctldata.SetShardTabletControlResponse - (*vtctldata.SetWritableResponse)(nil), // 150: vtctldata.SetWritableResponse - (*vtctldata.ShardReplicationAddResponse)(nil), // 151: vtctldata.ShardReplicationAddResponse - (*vtctldata.ShardReplicationFixResponse)(nil), // 152: vtctldata.ShardReplicationFixResponse - (*vtctldata.ShardReplicationPositionsResponse)(nil), // 153: vtctldata.ShardReplicationPositionsResponse - (*vtctldata.ShardReplicationRemoveResponse)(nil), // 154: vtctldata.ShardReplicationRemoveResponse - (*vtctldata.SleepTabletResponse)(nil), // 155: vtctldata.SleepTabletResponse - (*vtctldata.SourceShardAddResponse)(nil), // 156: vtctldata.SourceShardAddResponse - (*vtctldata.SourceShardDeleteResponse)(nil), // 157: vtctldata.SourceShardDeleteResponse - (*vtctldata.StartReplicationResponse)(nil), // 158: vtctldata.StartReplicationResponse - (*vtctldata.StopReplicationResponse)(nil), // 159: vtctldata.StopReplicationResponse - (*vtctldata.TabletExternallyReparentedResponse)(nil), // 160: vtctldata.TabletExternallyReparentedResponse - (*vtctldata.UpdateCellInfoResponse)(nil), // 161: vtctldata.UpdateCellInfoResponse - (*vtctldata.UpdateCellsAliasResponse)(nil), // 162: vtctldata.UpdateCellsAliasResponse - (*vtctldata.ValidateResponse)(nil), // 163: vtctldata.ValidateResponse - (*vtctldata.ValidateKeyspaceResponse)(nil), // 164: vtctldata.ValidateKeyspaceResponse - (*vtctldata.ValidateSchemaKeyspaceResponse)(nil), // 165: vtctldata.ValidateSchemaKeyspaceResponse - (*vtctldata.ValidateShardResponse)(nil), // 166: vtctldata.ValidateShardResponse - (*vtctldata.ValidateVersionKeyspaceResponse)(nil), // 167: vtctldata.ValidateVersionKeyspaceResponse - (*vtctldata.ValidateVersionShardResponse)(nil), // 168: vtctldata.ValidateVersionShardResponse - (*vtctldata.ValidateVSchemaResponse)(nil), // 169: vtctldata.ValidateVSchemaResponse - (*vtctldata.WorkflowUpdateResponse)(nil), // 170: vtctldata.WorkflowUpdateResponse + (*vtctldata.MoveTablesCreateRequest)(nil), // 47: vtctldata.MoveTablesCreateRequest + (*vtctldata.MoveTablesCompleteRequest)(nil), // 48: vtctldata.MoveTablesCompleteRequest + (*vtctldata.PingTabletRequest)(nil), // 49: vtctldata.PingTabletRequest + (*vtctldata.PlannedReparentShardRequest)(nil), // 50: vtctldata.PlannedReparentShardRequest + (*vtctldata.RebuildKeyspaceGraphRequest)(nil), // 51: vtctldata.RebuildKeyspaceGraphRequest + (*vtctldata.RebuildVSchemaGraphRequest)(nil), // 52: vtctldata.RebuildVSchemaGraphRequest + (*vtctldata.RefreshStateRequest)(nil), // 53: vtctldata.RefreshStateRequest + (*vtctldata.RefreshStateByShardRequest)(nil), // 54: vtctldata.RefreshStateByShardRequest + (*vtctldata.ReloadSchemaRequest)(nil), // 55: vtctldata.ReloadSchemaRequest + (*vtctldata.ReloadSchemaKeyspaceRequest)(nil), // 56: vtctldata.ReloadSchemaKeyspaceRequest + (*vtctldata.ReloadSchemaShardRequest)(nil), // 57: vtctldata.ReloadSchemaShardRequest + (*vtctldata.RemoveBackupRequest)(nil), // 58: vtctldata.RemoveBackupRequest + (*vtctldata.RemoveKeyspaceCellRequest)(nil), // 59: vtctldata.RemoveKeyspaceCellRequest + (*vtctldata.RemoveShardCellRequest)(nil), // 60: vtctldata.RemoveShardCellRequest + (*vtctldata.ReparentTabletRequest)(nil), // 61: vtctldata.ReparentTabletRequest + (*vtctldata.RestoreFromBackupRequest)(nil), // 62: vtctldata.RestoreFromBackupRequest + (*vtctldata.RunHealthCheckRequest)(nil), // 63: vtctldata.RunHealthCheckRequest + (*vtctldata.SetKeyspaceDurabilityPolicyRequest)(nil), // 64: vtctldata.SetKeyspaceDurabilityPolicyRequest + (*vtctldata.SetShardIsPrimaryServingRequest)(nil), // 65: vtctldata.SetShardIsPrimaryServingRequest + (*vtctldata.SetShardTabletControlRequest)(nil), // 66: vtctldata.SetShardTabletControlRequest + (*vtctldata.SetWritableRequest)(nil), // 67: vtctldata.SetWritableRequest + (*vtctldata.ShardReplicationAddRequest)(nil), // 68: vtctldata.ShardReplicationAddRequest + (*vtctldata.ShardReplicationFixRequest)(nil), // 69: vtctldata.ShardReplicationFixRequest + (*vtctldata.ShardReplicationPositionsRequest)(nil), // 70: vtctldata.ShardReplicationPositionsRequest + (*vtctldata.ShardReplicationRemoveRequest)(nil), // 71: vtctldata.ShardReplicationRemoveRequest + (*vtctldata.SleepTabletRequest)(nil), // 72: vtctldata.SleepTabletRequest + (*vtctldata.SourceShardAddRequest)(nil), // 73: vtctldata.SourceShardAddRequest + (*vtctldata.SourceShardDeleteRequest)(nil), // 74: vtctldata.SourceShardDeleteRequest + (*vtctldata.StartReplicationRequest)(nil), // 75: vtctldata.StartReplicationRequest + (*vtctldata.StopReplicationRequest)(nil), // 76: vtctldata.StopReplicationRequest + (*vtctldata.TabletExternallyReparentedRequest)(nil), // 77: vtctldata.TabletExternallyReparentedRequest + (*vtctldata.UpdateCellInfoRequest)(nil), // 78: vtctldata.UpdateCellInfoRequest + (*vtctldata.UpdateCellsAliasRequest)(nil), // 79: vtctldata.UpdateCellsAliasRequest + (*vtctldata.ValidateRequest)(nil), // 80: vtctldata.ValidateRequest + (*vtctldata.ValidateKeyspaceRequest)(nil), // 81: vtctldata.ValidateKeyspaceRequest + (*vtctldata.ValidateSchemaKeyspaceRequest)(nil), // 82: vtctldata.ValidateSchemaKeyspaceRequest + (*vtctldata.ValidateShardRequest)(nil), // 83: vtctldata.ValidateShardRequest + (*vtctldata.ValidateVersionKeyspaceRequest)(nil), // 84: vtctldata.ValidateVersionKeyspaceRequest + (*vtctldata.ValidateVersionShardRequest)(nil), // 85: vtctldata.ValidateVersionShardRequest + (*vtctldata.ValidateVSchemaRequest)(nil), // 86: vtctldata.ValidateVSchemaRequest + (*vtctldata.WorkflowDeleteRequest)(nil), // 87: vtctldata.WorkflowDeleteRequest + (*vtctldata.WorkflowStatusRequest)(nil), // 88: vtctldata.WorkflowStatusRequest + (*vtctldata.WorkflowSwitchTrafficRequest)(nil), // 89: vtctldata.WorkflowSwitchTrafficRequest + (*vtctldata.WorkflowUpdateRequest)(nil), // 90: vtctldata.WorkflowUpdateRequest + (*vtctldata.ExecuteVtctlCommandResponse)(nil), // 91: vtctldata.ExecuteVtctlCommandResponse + (*vtctldata.AddCellInfoResponse)(nil), // 92: vtctldata.AddCellInfoResponse + (*vtctldata.AddCellsAliasResponse)(nil), // 93: vtctldata.AddCellsAliasResponse + (*vtctldata.ApplyRoutingRulesResponse)(nil), // 94: vtctldata.ApplyRoutingRulesResponse + (*vtctldata.ApplySchemaResponse)(nil), // 95: vtctldata.ApplySchemaResponse + (*vtctldata.ApplyShardRoutingRulesResponse)(nil), // 96: vtctldata.ApplyShardRoutingRulesResponse + (*vtctldata.ApplyVSchemaResponse)(nil), // 97: vtctldata.ApplyVSchemaResponse + (*vtctldata.BackupResponse)(nil), // 98: vtctldata.BackupResponse + (*vtctldata.ChangeTabletTypeResponse)(nil), // 99: vtctldata.ChangeTabletTypeResponse + (*vtctldata.CreateKeyspaceResponse)(nil), // 100: vtctldata.CreateKeyspaceResponse + (*vtctldata.CreateShardResponse)(nil), // 101: vtctldata.CreateShardResponse + (*vtctldata.DeleteCellInfoResponse)(nil), // 102: vtctldata.DeleteCellInfoResponse + (*vtctldata.DeleteCellsAliasResponse)(nil), // 103: vtctldata.DeleteCellsAliasResponse + (*vtctldata.DeleteKeyspaceResponse)(nil), // 104: vtctldata.DeleteKeyspaceResponse + (*vtctldata.DeleteShardsResponse)(nil), // 105: vtctldata.DeleteShardsResponse + (*vtctldata.DeleteSrvVSchemaResponse)(nil), // 106: vtctldata.DeleteSrvVSchemaResponse + (*vtctldata.DeleteTabletsResponse)(nil), // 107: vtctldata.DeleteTabletsResponse + (*vtctldata.EmergencyReparentShardResponse)(nil), // 108: vtctldata.EmergencyReparentShardResponse + (*vtctldata.ExecuteFetchAsAppResponse)(nil), // 109: vtctldata.ExecuteFetchAsAppResponse + (*vtctldata.ExecuteFetchAsDBAResponse)(nil), // 110: vtctldata.ExecuteFetchAsDBAResponse + (*vtctldata.ExecuteHookResponse)(nil), // 111: vtctldata.ExecuteHookResponse + (*vtctldata.FindAllShardsInKeyspaceResponse)(nil), // 112: vtctldata.FindAllShardsInKeyspaceResponse + (*vtctldata.GetBackupsResponse)(nil), // 113: vtctldata.GetBackupsResponse + (*vtctldata.GetCellInfoResponse)(nil), // 114: vtctldata.GetCellInfoResponse + (*vtctldata.GetCellInfoNamesResponse)(nil), // 115: vtctldata.GetCellInfoNamesResponse + (*vtctldata.GetCellsAliasesResponse)(nil), // 116: vtctldata.GetCellsAliasesResponse + (*vtctldata.GetFullStatusResponse)(nil), // 117: vtctldata.GetFullStatusResponse + (*vtctldata.GetKeyspaceResponse)(nil), // 118: vtctldata.GetKeyspaceResponse + (*vtctldata.GetKeyspacesResponse)(nil), // 119: vtctldata.GetKeyspacesResponse + (*vtctldata.GetPermissionsResponse)(nil), // 120: vtctldata.GetPermissionsResponse + (*vtctldata.GetRoutingRulesResponse)(nil), // 121: vtctldata.GetRoutingRulesResponse + (*vtctldata.GetSchemaResponse)(nil), // 122: vtctldata.GetSchemaResponse + (*vtctldata.GetShardResponse)(nil), // 123: vtctldata.GetShardResponse + (*vtctldata.GetShardRoutingRulesResponse)(nil), // 124: vtctldata.GetShardRoutingRulesResponse + (*vtctldata.GetSrvKeyspaceNamesResponse)(nil), // 125: vtctldata.GetSrvKeyspaceNamesResponse + (*vtctldata.GetSrvKeyspacesResponse)(nil), // 126: vtctldata.GetSrvKeyspacesResponse + (*vtctldata.UpdateThrottlerConfigResponse)(nil), // 127: vtctldata.UpdateThrottlerConfigResponse + (*vtctldata.GetSrvVSchemaResponse)(nil), // 128: vtctldata.GetSrvVSchemaResponse + (*vtctldata.GetSrvVSchemasResponse)(nil), // 129: vtctldata.GetSrvVSchemasResponse + (*vtctldata.GetTabletResponse)(nil), // 130: vtctldata.GetTabletResponse + (*vtctldata.GetTabletsResponse)(nil), // 131: vtctldata.GetTabletsResponse + (*vtctldata.GetTopologyPathResponse)(nil), // 132: vtctldata.GetTopologyPathResponse + (*vtctldata.GetVersionResponse)(nil), // 133: vtctldata.GetVersionResponse + (*vtctldata.GetVSchemaResponse)(nil), // 134: vtctldata.GetVSchemaResponse + (*vtctldata.GetWorkflowsResponse)(nil), // 135: vtctldata.GetWorkflowsResponse + (*vtctldata.InitShardPrimaryResponse)(nil), // 136: vtctldata.InitShardPrimaryResponse + (*vtctldata.WorkflowStatusResponse)(nil), // 137: vtctldata.WorkflowStatusResponse + (*vtctldata.MoveTablesCompleteResponse)(nil), // 138: vtctldata.MoveTablesCompleteResponse + (*vtctldata.PingTabletResponse)(nil), // 139: vtctldata.PingTabletResponse + (*vtctldata.PlannedReparentShardResponse)(nil), // 140: vtctldata.PlannedReparentShardResponse + (*vtctldata.RebuildKeyspaceGraphResponse)(nil), // 141: vtctldata.RebuildKeyspaceGraphResponse + (*vtctldata.RebuildVSchemaGraphResponse)(nil), // 142: vtctldata.RebuildVSchemaGraphResponse + (*vtctldata.RefreshStateResponse)(nil), // 143: vtctldata.RefreshStateResponse + (*vtctldata.RefreshStateByShardResponse)(nil), // 144: vtctldata.RefreshStateByShardResponse + (*vtctldata.ReloadSchemaResponse)(nil), // 145: vtctldata.ReloadSchemaResponse + (*vtctldata.ReloadSchemaKeyspaceResponse)(nil), // 146: vtctldata.ReloadSchemaKeyspaceResponse + (*vtctldata.ReloadSchemaShardResponse)(nil), // 147: vtctldata.ReloadSchemaShardResponse + (*vtctldata.RemoveBackupResponse)(nil), // 148: vtctldata.RemoveBackupResponse + (*vtctldata.RemoveKeyspaceCellResponse)(nil), // 149: vtctldata.RemoveKeyspaceCellResponse + (*vtctldata.RemoveShardCellResponse)(nil), // 150: vtctldata.RemoveShardCellResponse + (*vtctldata.ReparentTabletResponse)(nil), // 151: vtctldata.ReparentTabletResponse + (*vtctldata.RestoreFromBackupResponse)(nil), // 152: vtctldata.RestoreFromBackupResponse + (*vtctldata.RunHealthCheckResponse)(nil), // 153: vtctldata.RunHealthCheckResponse + (*vtctldata.SetKeyspaceDurabilityPolicyResponse)(nil), // 154: vtctldata.SetKeyspaceDurabilityPolicyResponse + (*vtctldata.SetShardIsPrimaryServingResponse)(nil), // 155: vtctldata.SetShardIsPrimaryServingResponse + (*vtctldata.SetShardTabletControlResponse)(nil), // 156: vtctldata.SetShardTabletControlResponse + (*vtctldata.SetWritableResponse)(nil), // 157: vtctldata.SetWritableResponse + (*vtctldata.ShardReplicationAddResponse)(nil), // 158: vtctldata.ShardReplicationAddResponse + (*vtctldata.ShardReplicationFixResponse)(nil), // 159: vtctldata.ShardReplicationFixResponse + (*vtctldata.ShardReplicationPositionsResponse)(nil), // 160: vtctldata.ShardReplicationPositionsResponse + (*vtctldata.ShardReplicationRemoveResponse)(nil), // 161: vtctldata.ShardReplicationRemoveResponse + (*vtctldata.SleepTabletResponse)(nil), // 162: vtctldata.SleepTabletResponse + (*vtctldata.SourceShardAddResponse)(nil), // 163: vtctldata.SourceShardAddResponse + (*vtctldata.SourceShardDeleteResponse)(nil), // 164: vtctldata.SourceShardDeleteResponse + (*vtctldata.StartReplicationResponse)(nil), // 165: vtctldata.StartReplicationResponse + (*vtctldata.StopReplicationResponse)(nil), // 166: vtctldata.StopReplicationResponse + (*vtctldata.TabletExternallyReparentedResponse)(nil), // 167: vtctldata.TabletExternallyReparentedResponse + (*vtctldata.UpdateCellInfoResponse)(nil), // 168: vtctldata.UpdateCellInfoResponse + (*vtctldata.UpdateCellsAliasResponse)(nil), // 169: vtctldata.UpdateCellsAliasResponse + (*vtctldata.ValidateResponse)(nil), // 170: vtctldata.ValidateResponse + (*vtctldata.ValidateKeyspaceResponse)(nil), // 171: vtctldata.ValidateKeyspaceResponse + (*vtctldata.ValidateSchemaKeyspaceResponse)(nil), // 172: vtctldata.ValidateSchemaKeyspaceResponse + (*vtctldata.ValidateShardResponse)(nil), // 173: vtctldata.ValidateShardResponse + (*vtctldata.ValidateVersionKeyspaceResponse)(nil), // 174: vtctldata.ValidateVersionKeyspaceResponse + (*vtctldata.ValidateVersionShardResponse)(nil), // 175: vtctldata.ValidateVersionShardResponse + (*vtctldata.ValidateVSchemaResponse)(nil), // 176: vtctldata.ValidateVSchemaResponse + (*vtctldata.WorkflowDeleteResponse)(nil), // 177: vtctldata.WorkflowDeleteResponse + (*vtctldata.WorkflowSwitchTrafficResponse)(nil), // 178: vtctldata.WorkflowSwitchTrafficResponse + (*vtctldata.WorkflowUpdateResponse)(nil), // 179: vtctldata.WorkflowUpdateResponse } var file_vtctlservice_proto_depIdxs = []int32{ 0, // 0: vtctlservice.Vtctl.ExecuteVtctlCommand:input_type -> vtctldata.ExecuteVtctlCommandRequest @@ -773,133 +812,143 @@ var file_vtctlservice_proto_depIdxs = []int32{ 44, // 44: vtctlservice.Vtctld.GetVSchema:input_type -> vtctldata.GetVSchemaRequest 45, // 45: vtctlservice.Vtctld.GetWorkflows:input_type -> vtctldata.GetWorkflowsRequest 46, // 46: vtctlservice.Vtctld.InitShardPrimary:input_type -> vtctldata.InitShardPrimaryRequest - 47, // 47: vtctlservice.Vtctld.PingTablet:input_type -> vtctldata.PingTabletRequest - 48, // 48: vtctlservice.Vtctld.PlannedReparentShard:input_type -> vtctldata.PlannedReparentShardRequest - 49, // 49: vtctlservice.Vtctld.RebuildKeyspaceGraph:input_type -> vtctldata.RebuildKeyspaceGraphRequest - 50, // 50: vtctlservice.Vtctld.RebuildVSchemaGraph:input_type -> vtctldata.RebuildVSchemaGraphRequest - 51, // 51: vtctlservice.Vtctld.RefreshState:input_type -> vtctldata.RefreshStateRequest - 52, // 52: vtctlservice.Vtctld.RefreshStateByShard:input_type -> vtctldata.RefreshStateByShardRequest - 53, // 53: vtctlservice.Vtctld.ReloadSchema:input_type -> vtctldata.ReloadSchemaRequest - 54, // 54: vtctlservice.Vtctld.ReloadSchemaKeyspace:input_type -> vtctldata.ReloadSchemaKeyspaceRequest - 55, // 55: vtctlservice.Vtctld.ReloadSchemaShard:input_type -> vtctldata.ReloadSchemaShardRequest - 56, // 56: vtctlservice.Vtctld.RemoveBackup:input_type -> vtctldata.RemoveBackupRequest - 57, // 57: vtctlservice.Vtctld.RemoveKeyspaceCell:input_type -> vtctldata.RemoveKeyspaceCellRequest - 58, // 58: vtctlservice.Vtctld.RemoveShardCell:input_type -> vtctldata.RemoveShardCellRequest - 59, // 59: vtctlservice.Vtctld.ReparentTablet:input_type -> vtctldata.ReparentTabletRequest - 60, // 60: vtctlservice.Vtctld.RestoreFromBackup:input_type -> vtctldata.RestoreFromBackupRequest - 61, // 61: vtctlservice.Vtctld.RunHealthCheck:input_type -> vtctldata.RunHealthCheckRequest - 62, // 62: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:input_type -> vtctldata.SetKeyspaceDurabilityPolicyRequest - 63, // 63: vtctlservice.Vtctld.SetShardIsPrimaryServing:input_type -> vtctldata.SetShardIsPrimaryServingRequest - 64, // 64: vtctlservice.Vtctld.SetShardTabletControl:input_type -> vtctldata.SetShardTabletControlRequest - 65, // 65: vtctlservice.Vtctld.SetWritable:input_type -> vtctldata.SetWritableRequest - 66, // 66: vtctlservice.Vtctld.ShardReplicationAdd:input_type -> vtctldata.ShardReplicationAddRequest - 67, // 67: vtctlservice.Vtctld.ShardReplicationFix:input_type -> vtctldata.ShardReplicationFixRequest - 68, // 68: vtctlservice.Vtctld.ShardReplicationPositions:input_type -> vtctldata.ShardReplicationPositionsRequest - 69, // 69: vtctlservice.Vtctld.ShardReplicationRemove:input_type -> vtctldata.ShardReplicationRemoveRequest - 70, // 70: vtctlservice.Vtctld.SleepTablet:input_type -> vtctldata.SleepTabletRequest - 71, // 71: vtctlservice.Vtctld.SourceShardAdd:input_type -> vtctldata.SourceShardAddRequest - 72, // 72: vtctlservice.Vtctld.SourceShardDelete:input_type -> vtctldata.SourceShardDeleteRequest - 73, // 73: vtctlservice.Vtctld.StartReplication:input_type -> vtctldata.StartReplicationRequest - 74, // 74: vtctlservice.Vtctld.StopReplication:input_type -> vtctldata.StopReplicationRequest - 75, // 75: vtctlservice.Vtctld.TabletExternallyReparented:input_type -> vtctldata.TabletExternallyReparentedRequest - 76, // 76: vtctlservice.Vtctld.UpdateCellInfo:input_type -> vtctldata.UpdateCellInfoRequest - 77, // 77: vtctlservice.Vtctld.UpdateCellsAlias:input_type -> vtctldata.UpdateCellsAliasRequest - 78, // 78: vtctlservice.Vtctld.Validate:input_type -> vtctldata.ValidateRequest - 79, // 79: vtctlservice.Vtctld.ValidateKeyspace:input_type -> vtctldata.ValidateKeyspaceRequest - 80, // 80: vtctlservice.Vtctld.ValidateSchemaKeyspace:input_type -> vtctldata.ValidateSchemaKeyspaceRequest - 81, // 81: vtctlservice.Vtctld.ValidateShard:input_type -> vtctldata.ValidateShardRequest - 82, // 82: vtctlservice.Vtctld.ValidateVersionKeyspace:input_type -> vtctldata.ValidateVersionKeyspaceRequest - 83, // 83: vtctlservice.Vtctld.ValidateVersionShard:input_type -> vtctldata.ValidateVersionShardRequest - 84, // 84: vtctlservice.Vtctld.ValidateVSchema:input_type -> vtctldata.ValidateVSchemaRequest - 85, // 85: vtctlservice.Vtctld.WorkflowUpdate:input_type -> vtctldata.WorkflowUpdateRequest - 86, // 86: vtctlservice.Vtctl.ExecuteVtctlCommand:output_type -> vtctldata.ExecuteVtctlCommandResponse - 87, // 87: vtctlservice.Vtctld.AddCellInfo:output_type -> vtctldata.AddCellInfoResponse - 88, // 88: vtctlservice.Vtctld.AddCellsAlias:output_type -> vtctldata.AddCellsAliasResponse - 89, // 89: vtctlservice.Vtctld.ApplyRoutingRules:output_type -> vtctldata.ApplyRoutingRulesResponse - 90, // 90: vtctlservice.Vtctld.ApplySchema:output_type -> vtctldata.ApplySchemaResponse - 91, // 91: vtctlservice.Vtctld.ApplyShardRoutingRules:output_type -> vtctldata.ApplyShardRoutingRulesResponse - 92, // 92: vtctlservice.Vtctld.ApplyVSchema:output_type -> vtctldata.ApplyVSchemaResponse - 93, // 93: vtctlservice.Vtctld.Backup:output_type -> vtctldata.BackupResponse - 93, // 94: vtctlservice.Vtctld.BackupShard:output_type -> vtctldata.BackupResponse - 94, // 95: vtctlservice.Vtctld.ChangeTabletType:output_type -> vtctldata.ChangeTabletTypeResponse - 95, // 96: vtctlservice.Vtctld.CreateKeyspace:output_type -> vtctldata.CreateKeyspaceResponse - 96, // 97: vtctlservice.Vtctld.CreateShard:output_type -> vtctldata.CreateShardResponse - 97, // 98: vtctlservice.Vtctld.DeleteCellInfo:output_type -> vtctldata.DeleteCellInfoResponse - 98, // 99: vtctlservice.Vtctld.DeleteCellsAlias:output_type -> vtctldata.DeleteCellsAliasResponse - 99, // 100: vtctlservice.Vtctld.DeleteKeyspace:output_type -> vtctldata.DeleteKeyspaceResponse - 100, // 101: vtctlservice.Vtctld.DeleteShards:output_type -> vtctldata.DeleteShardsResponse - 101, // 102: vtctlservice.Vtctld.DeleteSrvVSchema:output_type -> vtctldata.DeleteSrvVSchemaResponse - 102, // 103: vtctlservice.Vtctld.DeleteTablets:output_type -> vtctldata.DeleteTabletsResponse - 103, // 104: vtctlservice.Vtctld.EmergencyReparentShard:output_type -> vtctldata.EmergencyReparentShardResponse - 104, // 105: vtctlservice.Vtctld.ExecuteFetchAsApp:output_type -> vtctldata.ExecuteFetchAsAppResponse - 105, // 106: vtctlservice.Vtctld.ExecuteFetchAsDBA:output_type -> vtctldata.ExecuteFetchAsDBAResponse - 106, // 107: vtctlservice.Vtctld.ExecuteHook:output_type -> vtctldata.ExecuteHookResponse - 107, // 108: vtctlservice.Vtctld.FindAllShardsInKeyspace:output_type -> vtctldata.FindAllShardsInKeyspaceResponse - 108, // 109: vtctlservice.Vtctld.GetBackups:output_type -> vtctldata.GetBackupsResponse - 109, // 110: vtctlservice.Vtctld.GetCellInfo:output_type -> vtctldata.GetCellInfoResponse - 110, // 111: vtctlservice.Vtctld.GetCellInfoNames:output_type -> vtctldata.GetCellInfoNamesResponse - 111, // 112: vtctlservice.Vtctld.GetCellsAliases:output_type -> vtctldata.GetCellsAliasesResponse - 112, // 113: vtctlservice.Vtctld.GetFullStatus:output_type -> vtctldata.GetFullStatusResponse - 113, // 114: vtctlservice.Vtctld.GetKeyspace:output_type -> vtctldata.GetKeyspaceResponse - 114, // 115: vtctlservice.Vtctld.GetKeyspaces:output_type -> vtctldata.GetKeyspacesResponse - 115, // 116: vtctlservice.Vtctld.GetPermissions:output_type -> vtctldata.GetPermissionsResponse - 116, // 117: vtctlservice.Vtctld.GetRoutingRules:output_type -> vtctldata.GetRoutingRulesResponse - 117, // 118: vtctlservice.Vtctld.GetSchema:output_type -> vtctldata.GetSchemaResponse - 118, // 119: vtctlservice.Vtctld.GetShard:output_type -> vtctldata.GetShardResponse - 119, // 120: vtctlservice.Vtctld.GetShardRoutingRules:output_type -> vtctldata.GetShardRoutingRulesResponse - 120, // 121: vtctlservice.Vtctld.GetSrvKeyspaceNames:output_type -> vtctldata.GetSrvKeyspaceNamesResponse - 121, // 122: vtctlservice.Vtctld.GetSrvKeyspaces:output_type -> vtctldata.GetSrvKeyspacesResponse - 122, // 123: vtctlservice.Vtctld.UpdateThrottlerConfig:output_type -> vtctldata.UpdateThrottlerConfigResponse - 123, // 124: vtctlservice.Vtctld.GetSrvVSchema:output_type -> vtctldata.GetSrvVSchemaResponse - 124, // 125: vtctlservice.Vtctld.GetSrvVSchemas:output_type -> vtctldata.GetSrvVSchemasResponse - 125, // 126: vtctlservice.Vtctld.GetTablet:output_type -> vtctldata.GetTabletResponse - 126, // 127: vtctlservice.Vtctld.GetTablets:output_type -> vtctldata.GetTabletsResponse - 127, // 128: vtctlservice.Vtctld.GetTopologyPath:output_type -> vtctldata.GetTopologyPathResponse - 128, // 129: vtctlservice.Vtctld.GetVersion:output_type -> vtctldata.GetVersionResponse - 129, // 130: vtctlservice.Vtctld.GetVSchema:output_type -> vtctldata.GetVSchemaResponse - 130, // 131: vtctlservice.Vtctld.GetWorkflows:output_type -> vtctldata.GetWorkflowsResponse - 131, // 132: vtctlservice.Vtctld.InitShardPrimary:output_type -> vtctldata.InitShardPrimaryResponse - 132, // 133: vtctlservice.Vtctld.PingTablet:output_type -> vtctldata.PingTabletResponse - 133, // 134: vtctlservice.Vtctld.PlannedReparentShard:output_type -> vtctldata.PlannedReparentShardResponse - 134, // 135: vtctlservice.Vtctld.RebuildKeyspaceGraph:output_type -> vtctldata.RebuildKeyspaceGraphResponse - 135, // 136: vtctlservice.Vtctld.RebuildVSchemaGraph:output_type -> vtctldata.RebuildVSchemaGraphResponse - 136, // 137: vtctlservice.Vtctld.RefreshState:output_type -> vtctldata.RefreshStateResponse - 137, // 138: vtctlservice.Vtctld.RefreshStateByShard:output_type -> vtctldata.RefreshStateByShardResponse - 138, // 139: vtctlservice.Vtctld.ReloadSchema:output_type -> vtctldata.ReloadSchemaResponse - 139, // 140: vtctlservice.Vtctld.ReloadSchemaKeyspace:output_type -> vtctldata.ReloadSchemaKeyspaceResponse - 140, // 141: vtctlservice.Vtctld.ReloadSchemaShard:output_type -> vtctldata.ReloadSchemaShardResponse - 141, // 142: vtctlservice.Vtctld.RemoveBackup:output_type -> vtctldata.RemoveBackupResponse - 142, // 143: vtctlservice.Vtctld.RemoveKeyspaceCell:output_type -> vtctldata.RemoveKeyspaceCellResponse - 143, // 144: vtctlservice.Vtctld.RemoveShardCell:output_type -> vtctldata.RemoveShardCellResponse - 144, // 145: vtctlservice.Vtctld.ReparentTablet:output_type -> vtctldata.ReparentTabletResponse - 145, // 146: vtctlservice.Vtctld.RestoreFromBackup:output_type -> vtctldata.RestoreFromBackupResponse - 146, // 147: vtctlservice.Vtctld.RunHealthCheck:output_type -> vtctldata.RunHealthCheckResponse - 147, // 148: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:output_type -> vtctldata.SetKeyspaceDurabilityPolicyResponse - 148, // 149: vtctlservice.Vtctld.SetShardIsPrimaryServing:output_type -> vtctldata.SetShardIsPrimaryServingResponse - 149, // 150: vtctlservice.Vtctld.SetShardTabletControl:output_type -> vtctldata.SetShardTabletControlResponse - 150, // 151: vtctlservice.Vtctld.SetWritable:output_type -> vtctldata.SetWritableResponse - 151, // 152: vtctlservice.Vtctld.ShardReplicationAdd:output_type -> vtctldata.ShardReplicationAddResponse - 152, // 153: vtctlservice.Vtctld.ShardReplicationFix:output_type -> vtctldata.ShardReplicationFixResponse - 153, // 154: vtctlservice.Vtctld.ShardReplicationPositions:output_type -> vtctldata.ShardReplicationPositionsResponse - 154, // 155: vtctlservice.Vtctld.ShardReplicationRemove:output_type -> vtctldata.ShardReplicationRemoveResponse - 155, // 156: vtctlservice.Vtctld.SleepTablet:output_type -> vtctldata.SleepTabletResponse - 156, // 157: vtctlservice.Vtctld.SourceShardAdd:output_type -> vtctldata.SourceShardAddResponse - 157, // 158: vtctlservice.Vtctld.SourceShardDelete:output_type -> vtctldata.SourceShardDeleteResponse - 158, // 159: vtctlservice.Vtctld.StartReplication:output_type -> vtctldata.StartReplicationResponse - 159, // 160: vtctlservice.Vtctld.StopReplication:output_type -> vtctldata.StopReplicationResponse - 160, // 161: vtctlservice.Vtctld.TabletExternallyReparented:output_type -> vtctldata.TabletExternallyReparentedResponse - 161, // 162: vtctlservice.Vtctld.UpdateCellInfo:output_type -> vtctldata.UpdateCellInfoResponse - 162, // 163: vtctlservice.Vtctld.UpdateCellsAlias:output_type -> vtctldata.UpdateCellsAliasResponse - 163, // 164: vtctlservice.Vtctld.Validate:output_type -> vtctldata.ValidateResponse - 164, // 165: vtctlservice.Vtctld.ValidateKeyspace:output_type -> vtctldata.ValidateKeyspaceResponse - 165, // 166: vtctlservice.Vtctld.ValidateSchemaKeyspace:output_type -> vtctldata.ValidateSchemaKeyspaceResponse - 166, // 167: vtctlservice.Vtctld.ValidateShard:output_type -> vtctldata.ValidateShardResponse - 167, // 168: vtctlservice.Vtctld.ValidateVersionKeyspace:output_type -> vtctldata.ValidateVersionKeyspaceResponse - 168, // 169: vtctlservice.Vtctld.ValidateVersionShard:output_type -> vtctldata.ValidateVersionShardResponse - 169, // 170: vtctlservice.Vtctld.ValidateVSchema:output_type -> vtctldata.ValidateVSchemaResponse - 170, // 171: vtctlservice.Vtctld.WorkflowUpdate:output_type -> vtctldata.WorkflowUpdateResponse - 86, // [86:172] is the sub-list for method output_type - 0, // [0:86] is the sub-list for method input_type + 47, // 47: vtctlservice.Vtctld.MoveTablesCreate:input_type -> vtctldata.MoveTablesCreateRequest + 48, // 48: vtctlservice.Vtctld.MoveTablesComplete:input_type -> vtctldata.MoveTablesCompleteRequest + 49, // 49: vtctlservice.Vtctld.PingTablet:input_type -> vtctldata.PingTabletRequest + 50, // 50: vtctlservice.Vtctld.PlannedReparentShard:input_type -> vtctldata.PlannedReparentShardRequest + 51, // 51: vtctlservice.Vtctld.RebuildKeyspaceGraph:input_type -> vtctldata.RebuildKeyspaceGraphRequest + 52, // 52: vtctlservice.Vtctld.RebuildVSchemaGraph:input_type -> vtctldata.RebuildVSchemaGraphRequest + 53, // 53: vtctlservice.Vtctld.RefreshState:input_type -> vtctldata.RefreshStateRequest + 54, // 54: vtctlservice.Vtctld.RefreshStateByShard:input_type -> vtctldata.RefreshStateByShardRequest + 55, // 55: vtctlservice.Vtctld.ReloadSchema:input_type -> vtctldata.ReloadSchemaRequest + 56, // 56: vtctlservice.Vtctld.ReloadSchemaKeyspace:input_type -> vtctldata.ReloadSchemaKeyspaceRequest + 57, // 57: vtctlservice.Vtctld.ReloadSchemaShard:input_type -> vtctldata.ReloadSchemaShardRequest + 58, // 58: vtctlservice.Vtctld.RemoveBackup:input_type -> vtctldata.RemoveBackupRequest + 59, // 59: vtctlservice.Vtctld.RemoveKeyspaceCell:input_type -> vtctldata.RemoveKeyspaceCellRequest + 60, // 60: vtctlservice.Vtctld.RemoveShardCell:input_type -> vtctldata.RemoveShardCellRequest + 61, // 61: vtctlservice.Vtctld.ReparentTablet:input_type -> vtctldata.ReparentTabletRequest + 62, // 62: vtctlservice.Vtctld.RestoreFromBackup:input_type -> vtctldata.RestoreFromBackupRequest + 63, // 63: vtctlservice.Vtctld.RunHealthCheck:input_type -> vtctldata.RunHealthCheckRequest + 64, // 64: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:input_type -> vtctldata.SetKeyspaceDurabilityPolicyRequest + 65, // 65: vtctlservice.Vtctld.SetShardIsPrimaryServing:input_type -> vtctldata.SetShardIsPrimaryServingRequest + 66, // 66: vtctlservice.Vtctld.SetShardTabletControl:input_type -> vtctldata.SetShardTabletControlRequest + 67, // 67: vtctlservice.Vtctld.SetWritable:input_type -> vtctldata.SetWritableRequest + 68, // 68: vtctlservice.Vtctld.ShardReplicationAdd:input_type -> vtctldata.ShardReplicationAddRequest + 69, // 69: vtctlservice.Vtctld.ShardReplicationFix:input_type -> vtctldata.ShardReplicationFixRequest + 70, // 70: vtctlservice.Vtctld.ShardReplicationPositions:input_type -> vtctldata.ShardReplicationPositionsRequest + 71, // 71: vtctlservice.Vtctld.ShardReplicationRemove:input_type -> vtctldata.ShardReplicationRemoveRequest + 72, // 72: vtctlservice.Vtctld.SleepTablet:input_type -> vtctldata.SleepTabletRequest + 73, // 73: vtctlservice.Vtctld.SourceShardAdd:input_type -> vtctldata.SourceShardAddRequest + 74, // 74: vtctlservice.Vtctld.SourceShardDelete:input_type -> vtctldata.SourceShardDeleteRequest + 75, // 75: vtctlservice.Vtctld.StartReplication:input_type -> vtctldata.StartReplicationRequest + 76, // 76: vtctlservice.Vtctld.StopReplication:input_type -> vtctldata.StopReplicationRequest + 77, // 77: vtctlservice.Vtctld.TabletExternallyReparented:input_type -> vtctldata.TabletExternallyReparentedRequest + 78, // 78: vtctlservice.Vtctld.UpdateCellInfo:input_type -> vtctldata.UpdateCellInfoRequest + 79, // 79: vtctlservice.Vtctld.UpdateCellsAlias:input_type -> vtctldata.UpdateCellsAliasRequest + 80, // 80: vtctlservice.Vtctld.Validate:input_type -> vtctldata.ValidateRequest + 81, // 81: vtctlservice.Vtctld.ValidateKeyspace:input_type -> vtctldata.ValidateKeyspaceRequest + 82, // 82: vtctlservice.Vtctld.ValidateSchemaKeyspace:input_type -> vtctldata.ValidateSchemaKeyspaceRequest + 83, // 83: vtctlservice.Vtctld.ValidateShard:input_type -> vtctldata.ValidateShardRequest + 84, // 84: vtctlservice.Vtctld.ValidateVersionKeyspace:input_type -> vtctldata.ValidateVersionKeyspaceRequest + 85, // 85: vtctlservice.Vtctld.ValidateVersionShard:input_type -> vtctldata.ValidateVersionShardRequest + 86, // 86: vtctlservice.Vtctld.ValidateVSchema:input_type -> vtctldata.ValidateVSchemaRequest + 87, // 87: vtctlservice.Vtctld.WorkflowDelete:input_type -> vtctldata.WorkflowDeleteRequest + 88, // 88: vtctlservice.Vtctld.WorkflowStatus:input_type -> vtctldata.WorkflowStatusRequest + 89, // 89: vtctlservice.Vtctld.WorkflowSwitchTraffic:input_type -> vtctldata.WorkflowSwitchTrafficRequest + 90, // 90: vtctlservice.Vtctld.WorkflowUpdate:input_type -> vtctldata.WorkflowUpdateRequest + 91, // 91: vtctlservice.Vtctl.ExecuteVtctlCommand:output_type -> vtctldata.ExecuteVtctlCommandResponse + 92, // 92: vtctlservice.Vtctld.AddCellInfo:output_type -> vtctldata.AddCellInfoResponse + 93, // 93: vtctlservice.Vtctld.AddCellsAlias:output_type -> vtctldata.AddCellsAliasResponse + 94, // 94: vtctlservice.Vtctld.ApplyRoutingRules:output_type -> vtctldata.ApplyRoutingRulesResponse + 95, // 95: vtctlservice.Vtctld.ApplySchema:output_type -> vtctldata.ApplySchemaResponse + 96, // 96: vtctlservice.Vtctld.ApplyShardRoutingRules:output_type -> vtctldata.ApplyShardRoutingRulesResponse + 97, // 97: vtctlservice.Vtctld.ApplyVSchema:output_type -> vtctldata.ApplyVSchemaResponse + 98, // 98: vtctlservice.Vtctld.Backup:output_type -> vtctldata.BackupResponse + 98, // 99: vtctlservice.Vtctld.BackupShard:output_type -> vtctldata.BackupResponse + 99, // 100: vtctlservice.Vtctld.ChangeTabletType:output_type -> vtctldata.ChangeTabletTypeResponse + 100, // 101: vtctlservice.Vtctld.CreateKeyspace:output_type -> vtctldata.CreateKeyspaceResponse + 101, // 102: vtctlservice.Vtctld.CreateShard:output_type -> vtctldata.CreateShardResponse + 102, // 103: vtctlservice.Vtctld.DeleteCellInfo:output_type -> vtctldata.DeleteCellInfoResponse + 103, // 104: vtctlservice.Vtctld.DeleteCellsAlias:output_type -> vtctldata.DeleteCellsAliasResponse + 104, // 105: vtctlservice.Vtctld.DeleteKeyspace:output_type -> vtctldata.DeleteKeyspaceResponse + 105, // 106: vtctlservice.Vtctld.DeleteShards:output_type -> vtctldata.DeleteShardsResponse + 106, // 107: vtctlservice.Vtctld.DeleteSrvVSchema:output_type -> vtctldata.DeleteSrvVSchemaResponse + 107, // 108: vtctlservice.Vtctld.DeleteTablets:output_type -> vtctldata.DeleteTabletsResponse + 108, // 109: vtctlservice.Vtctld.EmergencyReparentShard:output_type -> vtctldata.EmergencyReparentShardResponse + 109, // 110: vtctlservice.Vtctld.ExecuteFetchAsApp:output_type -> vtctldata.ExecuteFetchAsAppResponse + 110, // 111: vtctlservice.Vtctld.ExecuteFetchAsDBA:output_type -> vtctldata.ExecuteFetchAsDBAResponse + 111, // 112: vtctlservice.Vtctld.ExecuteHook:output_type -> vtctldata.ExecuteHookResponse + 112, // 113: vtctlservice.Vtctld.FindAllShardsInKeyspace:output_type -> vtctldata.FindAllShardsInKeyspaceResponse + 113, // 114: vtctlservice.Vtctld.GetBackups:output_type -> vtctldata.GetBackupsResponse + 114, // 115: vtctlservice.Vtctld.GetCellInfo:output_type -> vtctldata.GetCellInfoResponse + 115, // 116: vtctlservice.Vtctld.GetCellInfoNames:output_type -> vtctldata.GetCellInfoNamesResponse + 116, // 117: vtctlservice.Vtctld.GetCellsAliases:output_type -> vtctldata.GetCellsAliasesResponse + 117, // 118: vtctlservice.Vtctld.GetFullStatus:output_type -> vtctldata.GetFullStatusResponse + 118, // 119: vtctlservice.Vtctld.GetKeyspace:output_type -> vtctldata.GetKeyspaceResponse + 119, // 120: vtctlservice.Vtctld.GetKeyspaces:output_type -> vtctldata.GetKeyspacesResponse + 120, // 121: vtctlservice.Vtctld.GetPermissions:output_type -> vtctldata.GetPermissionsResponse + 121, // 122: vtctlservice.Vtctld.GetRoutingRules:output_type -> vtctldata.GetRoutingRulesResponse + 122, // 123: vtctlservice.Vtctld.GetSchema:output_type -> vtctldata.GetSchemaResponse + 123, // 124: vtctlservice.Vtctld.GetShard:output_type -> vtctldata.GetShardResponse + 124, // 125: vtctlservice.Vtctld.GetShardRoutingRules:output_type -> vtctldata.GetShardRoutingRulesResponse + 125, // 126: vtctlservice.Vtctld.GetSrvKeyspaceNames:output_type -> vtctldata.GetSrvKeyspaceNamesResponse + 126, // 127: vtctlservice.Vtctld.GetSrvKeyspaces:output_type -> vtctldata.GetSrvKeyspacesResponse + 127, // 128: vtctlservice.Vtctld.UpdateThrottlerConfig:output_type -> vtctldata.UpdateThrottlerConfigResponse + 128, // 129: vtctlservice.Vtctld.GetSrvVSchema:output_type -> vtctldata.GetSrvVSchemaResponse + 129, // 130: vtctlservice.Vtctld.GetSrvVSchemas:output_type -> vtctldata.GetSrvVSchemasResponse + 130, // 131: vtctlservice.Vtctld.GetTablet:output_type -> vtctldata.GetTabletResponse + 131, // 132: vtctlservice.Vtctld.GetTablets:output_type -> vtctldata.GetTabletsResponse + 132, // 133: vtctlservice.Vtctld.GetTopologyPath:output_type -> vtctldata.GetTopologyPathResponse + 133, // 134: vtctlservice.Vtctld.GetVersion:output_type -> vtctldata.GetVersionResponse + 134, // 135: vtctlservice.Vtctld.GetVSchema:output_type -> vtctldata.GetVSchemaResponse + 135, // 136: vtctlservice.Vtctld.GetWorkflows:output_type -> vtctldata.GetWorkflowsResponse + 136, // 137: vtctlservice.Vtctld.InitShardPrimary:output_type -> vtctldata.InitShardPrimaryResponse + 137, // 138: vtctlservice.Vtctld.MoveTablesCreate:output_type -> vtctldata.WorkflowStatusResponse + 138, // 139: vtctlservice.Vtctld.MoveTablesComplete:output_type -> vtctldata.MoveTablesCompleteResponse + 139, // 140: vtctlservice.Vtctld.PingTablet:output_type -> vtctldata.PingTabletResponse + 140, // 141: vtctlservice.Vtctld.PlannedReparentShard:output_type -> vtctldata.PlannedReparentShardResponse + 141, // 142: vtctlservice.Vtctld.RebuildKeyspaceGraph:output_type -> vtctldata.RebuildKeyspaceGraphResponse + 142, // 143: vtctlservice.Vtctld.RebuildVSchemaGraph:output_type -> vtctldata.RebuildVSchemaGraphResponse + 143, // 144: vtctlservice.Vtctld.RefreshState:output_type -> vtctldata.RefreshStateResponse + 144, // 145: vtctlservice.Vtctld.RefreshStateByShard:output_type -> vtctldata.RefreshStateByShardResponse + 145, // 146: vtctlservice.Vtctld.ReloadSchema:output_type -> vtctldata.ReloadSchemaResponse + 146, // 147: vtctlservice.Vtctld.ReloadSchemaKeyspace:output_type -> vtctldata.ReloadSchemaKeyspaceResponse + 147, // 148: vtctlservice.Vtctld.ReloadSchemaShard:output_type -> vtctldata.ReloadSchemaShardResponse + 148, // 149: vtctlservice.Vtctld.RemoveBackup:output_type -> vtctldata.RemoveBackupResponse + 149, // 150: vtctlservice.Vtctld.RemoveKeyspaceCell:output_type -> vtctldata.RemoveKeyspaceCellResponse + 150, // 151: vtctlservice.Vtctld.RemoveShardCell:output_type -> vtctldata.RemoveShardCellResponse + 151, // 152: vtctlservice.Vtctld.ReparentTablet:output_type -> vtctldata.ReparentTabletResponse + 152, // 153: vtctlservice.Vtctld.RestoreFromBackup:output_type -> vtctldata.RestoreFromBackupResponse + 153, // 154: vtctlservice.Vtctld.RunHealthCheck:output_type -> vtctldata.RunHealthCheckResponse + 154, // 155: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:output_type -> vtctldata.SetKeyspaceDurabilityPolicyResponse + 155, // 156: vtctlservice.Vtctld.SetShardIsPrimaryServing:output_type -> vtctldata.SetShardIsPrimaryServingResponse + 156, // 157: vtctlservice.Vtctld.SetShardTabletControl:output_type -> vtctldata.SetShardTabletControlResponse + 157, // 158: vtctlservice.Vtctld.SetWritable:output_type -> vtctldata.SetWritableResponse + 158, // 159: vtctlservice.Vtctld.ShardReplicationAdd:output_type -> vtctldata.ShardReplicationAddResponse + 159, // 160: vtctlservice.Vtctld.ShardReplicationFix:output_type -> vtctldata.ShardReplicationFixResponse + 160, // 161: vtctlservice.Vtctld.ShardReplicationPositions:output_type -> vtctldata.ShardReplicationPositionsResponse + 161, // 162: vtctlservice.Vtctld.ShardReplicationRemove:output_type -> vtctldata.ShardReplicationRemoveResponse + 162, // 163: vtctlservice.Vtctld.SleepTablet:output_type -> vtctldata.SleepTabletResponse + 163, // 164: vtctlservice.Vtctld.SourceShardAdd:output_type -> vtctldata.SourceShardAddResponse + 164, // 165: vtctlservice.Vtctld.SourceShardDelete:output_type -> vtctldata.SourceShardDeleteResponse + 165, // 166: vtctlservice.Vtctld.StartReplication:output_type -> vtctldata.StartReplicationResponse + 166, // 167: vtctlservice.Vtctld.StopReplication:output_type -> vtctldata.StopReplicationResponse + 167, // 168: vtctlservice.Vtctld.TabletExternallyReparented:output_type -> vtctldata.TabletExternallyReparentedResponse + 168, // 169: vtctlservice.Vtctld.UpdateCellInfo:output_type -> vtctldata.UpdateCellInfoResponse + 169, // 170: vtctlservice.Vtctld.UpdateCellsAlias:output_type -> vtctldata.UpdateCellsAliasResponse + 170, // 171: vtctlservice.Vtctld.Validate:output_type -> vtctldata.ValidateResponse + 171, // 172: vtctlservice.Vtctld.ValidateKeyspace:output_type -> vtctldata.ValidateKeyspaceResponse + 172, // 173: vtctlservice.Vtctld.ValidateSchemaKeyspace:output_type -> vtctldata.ValidateSchemaKeyspaceResponse + 173, // 174: vtctlservice.Vtctld.ValidateShard:output_type -> vtctldata.ValidateShardResponse + 174, // 175: vtctlservice.Vtctld.ValidateVersionKeyspace:output_type -> vtctldata.ValidateVersionKeyspaceResponse + 175, // 176: vtctlservice.Vtctld.ValidateVersionShard:output_type -> vtctldata.ValidateVersionShardResponse + 176, // 177: vtctlservice.Vtctld.ValidateVSchema:output_type -> vtctldata.ValidateVSchemaResponse + 177, // 178: vtctlservice.Vtctld.WorkflowDelete:output_type -> vtctldata.WorkflowDeleteResponse + 137, // 179: vtctlservice.Vtctld.WorkflowStatus:output_type -> vtctldata.WorkflowStatusResponse + 178, // 180: vtctlservice.Vtctld.WorkflowSwitchTraffic:output_type -> vtctldata.WorkflowSwitchTrafficResponse + 179, // 181: vtctlservice.Vtctld.WorkflowUpdate:output_type -> vtctldata.WorkflowUpdateResponse + 91, // [91:182] is the sub-list for method output_type + 0, // [0:91] 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 89c04c59bf6..3ddb69414f4 100644 --- a/go/vt/proto/vtctlservice/vtctlservice_grpc.pb.go +++ b/go/vt/proto/vtctlservice/vtctlservice_grpc.pb.go @@ -261,6 +261,12 @@ type VtctldClient interface { // PlannedReparentShard or EmergencyReparentShard should be used in those // cases instead. InitShardPrimary(ctx context.Context, in *vtctldata.InitShardPrimaryRequest, opts ...grpc.CallOption) (*vtctldata.InitShardPrimaryResponse, error) + // MoveTablesCreate creates a workflow which moves one or more tables from a + // source keyspace to a target keyspace. + MoveTablesCreate(ctx context.Context, in *vtctldata.MoveTablesCreateRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowStatusResponse, error) + // MoveTablesComplete completes the move and cleans up the workflow and + // its related artifacts. + MoveTablesComplete(ctx context.Context, in *vtctldata.MoveTablesCompleteRequest, opts ...grpc.CallOption) (*vtctldata.MoveTablesCompleteResponse, error) // PingTablet checks that the specified tablet is awake and responding to RPCs. // This command can be blocked by other in-flight operations. PingTablet(ctx context.Context, in *vtctldata.PingTabletRequest, opts ...grpc.CallOption) (*vtctldata.PingTabletResponse, error) @@ -398,6 +404,10 @@ type VtctldClient interface { ValidateVersionShard(ctx context.Context, in *vtctldata.ValidateVersionShardRequest, opts ...grpc.CallOption) (*vtctldata.ValidateVersionShardResponse, error) // ValidateVSchema compares the schema of each primary tablet in "keyspace/shards..." to the vschema and errs if there are differences. ValidateVSchema(ctx context.Context, in *vtctldata.ValidateVSchemaRequest, opts ...grpc.CallOption) (*vtctldata.ValidateVSchemaResponse, error) + // WorkflowDelete deletes a vreplication workflow. + WorkflowDelete(ctx context.Context, in *vtctldata.WorkflowDeleteRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowDeleteResponse, error) + WorkflowStatus(ctx context.Context, in *vtctldata.WorkflowStatusRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowStatusResponse, error) + WorkflowSwitchTraffic(ctx context.Context, in *vtctldata.WorkflowSwitchTrafficRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowSwitchTrafficResponse, error) // WorkflowUpdate updates the configuration of a vreplication workflow // using the provided updated parameters. WorkflowUpdate(ctx context.Context, in *vtctldata.WorkflowUpdateRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowUpdateResponse, error) @@ -871,6 +881,24 @@ func (c *vtctldClient) InitShardPrimary(ctx context.Context, in *vtctldata.InitS return out, nil } +func (c *vtctldClient) MoveTablesCreate(ctx context.Context, in *vtctldata.MoveTablesCreateRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowStatusResponse, error) { + out := new(vtctldata.WorkflowStatusResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/MoveTablesCreate", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *vtctldClient) MoveTablesComplete(ctx context.Context, in *vtctldata.MoveTablesCompleteRequest, opts ...grpc.CallOption) (*vtctldata.MoveTablesCompleteResponse, error) { + out := new(vtctldata.MoveTablesCompleteResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/MoveTablesComplete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *vtctldClient) PingTablet(ctx context.Context, in *vtctldata.PingTabletRequest, opts ...grpc.CallOption) (*vtctldata.PingTabletResponse, error) { out := new(vtctldata.PingTabletResponse) err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/PingTablet", in, out, opts...) @@ -1236,6 +1264,33 @@ func (c *vtctldClient) ValidateVSchema(ctx context.Context, in *vtctldata.Valida return out, nil } +func (c *vtctldClient) WorkflowDelete(ctx context.Context, in *vtctldata.WorkflowDeleteRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowDeleteResponse, error) { + out := new(vtctldata.WorkflowDeleteResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/WorkflowDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *vtctldClient) WorkflowStatus(ctx context.Context, in *vtctldata.WorkflowStatusRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowStatusResponse, error) { + out := new(vtctldata.WorkflowStatusResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/WorkflowStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *vtctldClient) WorkflowSwitchTraffic(ctx context.Context, in *vtctldata.WorkflowSwitchTrafficRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowSwitchTrafficResponse, error) { + out := new(vtctldata.WorkflowSwitchTrafficResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/WorkflowSwitchTraffic", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *vtctldClient) WorkflowUpdate(ctx context.Context, in *vtctldata.WorkflowUpdateRequest, opts ...grpc.CallOption) (*vtctldata.WorkflowUpdateResponse, error) { out := new(vtctldata.WorkflowUpdateResponse) err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/WorkflowUpdate", in, out, opts...) @@ -1374,6 +1429,12 @@ type VtctldServer interface { // PlannedReparentShard or EmergencyReparentShard should be used in those // cases instead. InitShardPrimary(context.Context, *vtctldata.InitShardPrimaryRequest) (*vtctldata.InitShardPrimaryResponse, error) + // MoveTablesCreate creates a workflow which moves one or more tables from a + // source keyspace to a target keyspace. + MoveTablesCreate(context.Context, *vtctldata.MoveTablesCreateRequest) (*vtctldata.WorkflowStatusResponse, error) + // MoveTablesComplete completes the move and cleans up the workflow and + // its related artifacts. + MoveTablesComplete(context.Context, *vtctldata.MoveTablesCompleteRequest) (*vtctldata.MoveTablesCompleteResponse, error) // PingTablet checks that the specified tablet is awake and responding to RPCs. // This command can be blocked by other in-flight operations. PingTablet(context.Context, *vtctldata.PingTabletRequest) (*vtctldata.PingTabletResponse, error) @@ -1511,6 +1572,10 @@ type VtctldServer interface { ValidateVersionShard(context.Context, *vtctldata.ValidateVersionShardRequest) (*vtctldata.ValidateVersionShardResponse, error) // ValidateVSchema compares the schema of each primary tablet in "keyspace/shards..." to the vschema and errs if there are differences. ValidateVSchema(context.Context, *vtctldata.ValidateVSchemaRequest) (*vtctldata.ValidateVSchemaResponse, error) + // WorkflowDelete deletes a vreplication workflow. + WorkflowDelete(context.Context, *vtctldata.WorkflowDeleteRequest) (*vtctldata.WorkflowDeleteResponse, error) + WorkflowStatus(context.Context, *vtctldata.WorkflowStatusRequest) (*vtctldata.WorkflowStatusResponse, error) + WorkflowSwitchTraffic(context.Context, *vtctldata.WorkflowSwitchTrafficRequest) (*vtctldata.WorkflowSwitchTrafficResponse, error) // WorkflowUpdate updates the configuration of a vreplication workflow // using the provided updated parameters. WorkflowUpdate(context.Context, *vtctldata.WorkflowUpdateRequest) (*vtctldata.WorkflowUpdateResponse, error) @@ -1659,6 +1724,12 @@ func (UnimplementedVtctldServer) GetWorkflows(context.Context, *vtctldata.GetWor func (UnimplementedVtctldServer) InitShardPrimary(context.Context, *vtctldata.InitShardPrimaryRequest) (*vtctldata.InitShardPrimaryResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method InitShardPrimary not implemented") } +func (UnimplementedVtctldServer) MoveTablesCreate(context.Context, *vtctldata.MoveTablesCreateRequest) (*vtctldata.WorkflowStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MoveTablesCreate not implemented") +} +func (UnimplementedVtctldServer) MoveTablesComplete(context.Context, *vtctldata.MoveTablesCompleteRequest) (*vtctldata.MoveTablesCompleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MoveTablesComplete not implemented") +} func (UnimplementedVtctldServer) PingTablet(context.Context, *vtctldata.PingTabletRequest) (*vtctldata.PingTabletResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method PingTablet not implemented") } @@ -1773,6 +1844,15 @@ func (UnimplementedVtctldServer) ValidateVersionShard(context.Context, *vtctldat func (UnimplementedVtctldServer) ValidateVSchema(context.Context, *vtctldata.ValidateVSchemaRequest) (*vtctldata.ValidateVSchemaResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ValidateVSchema not implemented") } +func (UnimplementedVtctldServer) WorkflowDelete(context.Context, *vtctldata.WorkflowDeleteRequest) (*vtctldata.WorkflowDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method WorkflowDelete not implemented") +} +func (UnimplementedVtctldServer) WorkflowStatus(context.Context, *vtctldata.WorkflowStatusRequest) (*vtctldata.WorkflowStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method WorkflowStatus not implemented") +} +func (UnimplementedVtctldServer) WorkflowSwitchTraffic(context.Context, *vtctldata.WorkflowSwitchTrafficRequest) (*vtctldata.WorkflowSwitchTrafficResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method WorkflowSwitchTraffic not implemented") +} func (UnimplementedVtctldServer) WorkflowUpdate(context.Context, *vtctldata.WorkflowUpdateRequest) (*vtctldata.WorkflowUpdateResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method WorkflowUpdate not implemented") } @@ -2623,6 +2703,42 @@ func _Vtctld_InitShardPrimary_Handler(srv interface{}, ctx context.Context, dec return interceptor(ctx, in, info, handler) } +func _Vtctld_MoveTablesCreate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.MoveTablesCreateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).MoveTablesCreate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/MoveTablesCreate", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).MoveTablesCreate(ctx, req.(*vtctldata.MoveTablesCreateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Vtctld_MoveTablesComplete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.MoveTablesCompleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).MoveTablesComplete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/MoveTablesComplete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).MoveTablesComplete(ctx, req.(*vtctldata.MoveTablesCompleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Vtctld_PingTablet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(vtctldata.PingTabletRequest) if err := dec(in); err != nil { @@ -3310,6 +3426,60 @@ func _Vtctld_ValidateVSchema_Handler(srv interface{}, ctx context.Context, dec f return interceptor(ctx, in, info, handler) } +func _Vtctld_WorkflowDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.WorkflowDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).WorkflowDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/WorkflowDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).WorkflowDelete(ctx, req.(*vtctldata.WorkflowDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Vtctld_WorkflowStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.WorkflowStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).WorkflowStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/WorkflowStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).WorkflowStatus(ctx, req.(*vtctldata.WorkflowStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Vtctld_WorkflowSwitchTraffic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.WorkflowSwitchTrafficRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).WorkflowSwitchTraffic(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/WorkflowSwitchTraffic", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).WorkflowSwitchTraffic(ctx, req.(*vtctldata.WorkflowSwitchTrafficRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Vtctld_WorkflowUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(vtctldata.WorkflowUpdateRequest) if err := dec(in); err != nil { @@ -3511,6 +3681,14 @@ var Vtctld_ServiceDesc = grpc.ServiceDesc{ MethodName: "InitShardPrimary", Handler: _Vtctld_InitShardPrimary_Handler, }, + { + MethodName: "MoveTablesCreate", + Handler: _Vtctld_MoveTablesCreate_Handler, + }, + { + MethodName: "MoveTablesComplete", + Handler: _Vtctld_MoveTablesComplete_Handler, + }, { MethodName: "PingTablet", Handler: _Vtctld_PingTablet_Handler, @@ -3659,6 +3837,18 @@ var Vtctld_ServiceDesc = grpc.ServiceDesc{ MethodName: "ValidateVSchema", Handler: _Vtctld_ValidateVSchema_Handler, }, + { + MethodName: "WorkflowDelete", + Handler: _Vtctld_WorkflowDelete_Handler, + }, + { + MethodName: "WorkflowStatus", + Handler: _Vtctld_WorkflowStatus_Handler, + }, + { + MethodName: "WorkflowSwitchTraffic", + Handler: _Vtctld_WorkflowSwitchTraffic_Handler, + }, { MethodName: "WorkflowUpdate", Handler: _Vtctld_WorkflowUpdate_Handler, diff --git a/go/vt/topo/topoproto/tablet.go b/go/vt/topo/topoproto/tablet.go index c5b4c7c2341..63e71807119 100644 --- a/go/vt/topo/topoproto/tablet.go +++ b/go/vt/topo/topoproto/tablet.go @@ -186,6 +186,9 @@ func ParseTabletType(param string) (topodatapb.TabletType, error) { // ParseTabletTypes parses a comma separated list of tablet types and returns a slice with the respective enums. func ParseTabletTypes(param string) ([]topodatapb.TabletType, error) { var tabletTypes []topodatapb.TabletType + if param == "" { + return tabletTypes, nil + } for _, typeStr := range strings.Split(param, ",") { t, err := ParseTabletType(typeStr) if err != nil { @@ -227,6 +230,21 @@ func MakeStringTypeList(types []topodatapb.TabletType) []string { return strs } +// MakeStringTypeUnsortedList returns a list of strings that match the input +// without modifying the order in the list. +func MakeStringTypeUnsortedList(types []topodatapb.TabletType) []string { + strs := make([]string, len(types)) + for i, t := range types { + strs[i] = strings.ToLower(t.String()) + } + return strs +} + +// MakeStringTypeCSV returns the tablet types in CSV format. +func MakeStringTypeCSV(types []topodatapb.TabletType) string { + return strings.Join(MakeStringTypeUnsortedList(types), ",") +} + // MakeUniqueStringTypeList returns a unique list of strings that match // the input list -- with duplicate types removed. // This is needed as some types are aliases for others, like BATCH and diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index e2e4946d219..18923177def 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -861,6 +861,22 @@ func (itmc *internalTabletManagerClient) WaitForPosition(context.Context, *topod return fmt.Errorf("not implemented in vtcombo") } +// +// VReplication related methods +// + +func (itmc *internalTabletManagerClient) CreateVReplicationWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + +func (itmc *internalTabletManagerClient) DeleteVReplicationWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + +func (itmc *internalTabletManagerClient) ReadVReplicationWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + func (itmc *internalTabletManagerClient) VReplicationExec(context.Context, *topodatapb.Tablet, string) (*querypb.QueryResult, error) { return nil, fmt.Errorf("not implemented in vtcombo") } @@ -869,7 +885,7 @@ func (itmc *internalTabletManagerClient) VReplicationWaitForPos(context.Context, return fmt.Errorf("not implemented in vtcombo") } -func (itmc *internalTabletManagerClient) UpdateVRWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) { +func (itmc *internalTabletManagerClient) UpdateVReplicationWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { return nil, fmt.Errorf("not implemented in vtcombo") } diff --git a/go/vt/vtctl/grpcvtctldclient/client_gen.go b/go/vt/vtctl/grpcvtctldclient/client_gen.go index 7e04dcb693d..a8156fef5df 100644 --- a/go/vt/vtctl/grpcvtctldclient/client_gen.go +++ b/go/vt/vtctl/grpcvtctldclient/client_gen.go @@ -434,6 +434,24 @@ func (client *gRPCVtctldClient) InitShardPrimary(ctx context.Context, in *vtctld return client.c.InitShardPrimary(ctx, in, opts...) } +// MoveTablesComplete is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) MoveTablesComplete(ctx context.Context, in *vtctldatapb.MoveTablesCompleteRequest, opts ...grpc.CallOption) (*vtctldatapb.MoveTablesCompleteResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.MoveTablesComplete(ctx, in, opts...) +} + +// MoveTablesCreate is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) MoveTablesCreate(ctx context.Context, in *vtctldatapb.MoveTablesCreateRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowStatusResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.MoveTablesCreate(ctx, in, opts...) +} + // PingTablet is part of the vtctlservicepb.VtctldClient interface. func (client *gRPCVtctldClient) PingTablet(ctx context.Context, in *vtctldatapb.PingTabletRequest, opts ...grpc.CallOption) (*vtctldatapb.PingTabletResponse, error) { if client.c == nil { @@ -785,6 +803,33 @@ func (client *gRPCVtctldClient) ValidateVersionShard(ctx context.Context, in *vt return client.c.ValidateVersionShard(ctx, in, opts...) } +// WorkflowDelete is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) WorkflowDelete(ctx context.Context, in *vtctldatapb.WorkflowDeleteRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowDeleteResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.WorkflowDelete(ctx, in, opts...) +} + +// WorkflowStatus is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) WorkflowStatus(ctx context.Context, in *vtctldatapb.WorkflowStatusRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowStatusResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.WorkflowStatus(ctx, in, opts...) +} + +// WorkflowSwitchTraffic is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) WorkflowSwitchTraffic(ctx context.Context, in *vtctldatapb.WorkflowSwitchTrafficRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowSwitchTrafficResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.WorkflowSwitchTraffic(ctx, in, opts...) +} + // WorkflowUpdate is part of the vtctlservicepb.VtctldClient interface. func (client *gRPCVtctldClient) WorkflowUpdate(ctx context.Context, in *vtctldatapb.WorkflowUpdateRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowUpdateResponse, error) { if client.c == nil { diff --git a/go/vt/vtctl/grpcvtctldserver/server.go b/go/vt/vtctl/grpcvtctldserver/server.go index 9c58bd1ec77..68cff9702a8 100644 --- a/go/vt/vtctl/grpcvtctldserver/server.go +++ b/go/vt/vtctl/grpcvtctldserver/server.go @@ -71,7 +71,7 @@ import ( vschemapb "vitess.io/vitess/go/vt/proto/vschema" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" vtctlservicepb "vitess.io/vitess/go/vt/proto/vtctlservice" - "vitess.io/vitess/go/vt/proto/vtrpc" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) const ( @@ -121,7 +121,7 @@ func (s *VtctldServer) AddCellInfo(ctx context.Context, req *vtctldatapb.AddCell defer panicHandler(&err) if req.CellInfo.Root == "" { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "CellInfo.Root must be non-empty") + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "CellInfo.Root must be non-empty") return nil, err } @@ -225,7 +225,7 @@ func (s *VtctldServer) ApplySchema(ctx context.Context, req *vtctldatapb.ApplySc span.Annotate("ddl_strategy", req.DdlStrategy) if len(req.Sql) == 0 { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "Sql must be a non-empty array") + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "Sql must be a non-empty array") return nil, err } @@ -315,7 +315,7 @@ func (s *VtctldServer) ApplyVSchema(ctx context.Context, req *vtctldatapb.ApplyV } if (req.Sql != "" && req.VSchema != nil) || (req.Sql == "" && req.VSchema == nil) { - err = vterrors.New(vtrpc.Code_INVALID_ARGUMENT, "must pass exactly one of req.VSchema and req.Sql") + err = vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "must pass exactly one of req.VSchema and req.Sql") return nil, err } @@ -332,7 +332,7 @@ func (s *VtctldServer) ApplyVSchema(ctx context.Context, req *vtctldatapb.ApplyV } ddl, ok := stmt.(*sqlparser.AlterVschema) if !ok { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "error parsing VSchema DDL statement `%s`", req.Sql) + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "error parsing VSchema DDL statement `%s`", req.Sql) return nil, err } @@ -446,7 +446,7 @@ func (s *VtctldServer) BackupShard(req *vtctldatapb.BackupShardRequest, stream v } if backupTablet == nil { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "no tablet available for backup") + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no tablet available for backup") return err } @@ -547,7 +547,7 @@ func (s *VtctldServer) ChangeTabletType(ctx context.Context, req *vtctldatapb.Ch } if !shard.HasPrimary() { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) return nil, err } @@ -558,12 +558,12 @@ func (s *VtctldServer) ChangeTabletType(ctx context.Context, req *vtctldatapb.Ch } if shardPrimary.Type != topodatapb.TabletType_PRIMARY { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) return nil, err } if shardPrimary.Keyspace != tablet.Keyspace || shardPrimary.Shard != tablet.Shard { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "primary %v and potential replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), req.TabletAlias, tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "primary %v and potential replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), req.TabletAlias, tablet.Keyspace, tablet.Shard) return nil, err } @@ -843,7 +843,7 @@ func (s *VtctldServer) DeleteKeyspace(ctx context.Context, req *vtctldatapb.Dele if len(shards) > 0 { if !req.Recursive { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "keyspace %v still has %d shards; use Recursive=true or remove them manually", req.Keyspace, len(shards)) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "keyspace %v still has %d shards; use Recursive=true or remove them manually", req.Keyspace, len(shards)) return nil, err } @@ -915,7 +915,7 @@ func (s *VtctldServer) DeleteSrvVSchema(ctx context.Context, req *vtctldatapb.De defer panicHandler(&err) if req.Cell == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "cell must be non-empty") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "cell must be non-empty") return nil, err } @@ -1085,14 +1085,14 @@ func (s *VtctldServer) ExecuteHook(ctx context.Context, req *vtctldatapb.Execute span.Annotate("tablet_alias", topoproto.TabletAliasString(req.TabletAlias)) if req.TabletHookRequest == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "TabletHookRequest cannot be nil") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "TabletHookRequest cannot be nil") return nil, err } span.Annotate("hook_name", req.TabletHookRequest.Name) if strings.Contains(req.TabletHookRequest.Name, "/") { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "hook name cannot contain a '/'; was %v", req.TabletHookRequest.Name) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "hook name cannot contain a '/'; was %v", req.TabletHookRequest.Name) return nil, err } @@ -1230,7 +1230,7 @@ func (s *VtctldServer) GetCellInfo(ctx context.Context, req *vtctldatapb.GetCell defer panicHandler(&err) if req.Cell == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "cell field is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "cell field is required") return nil, err } @@ -1346,7 +1346,7 @@ func (s *VtctldServer) GetPermissions(ctx context.Context, req *vtctldatapb.GetP span.Annotate("tablet_alias", topoproto.TabletAliasString(req.TabletAlias)) ti, err := s.ts.GetTablet(ctx, req.TabletAlias) if err != nil { - err = vterrors.Errorf(vtrpc.Code_NOT_FOUND, "Failed to get tablet %v: %v", req.TabletAlias, err) + err = vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, "Failed to get tablet %v: %v", req.TabletAlias, err) return nil, err } @@ -1977,12 +1977,12 @@ func (s *VtctldServer) InitShardPrimary(ctx context.Context, req *vtctldatapb.In defer panicHandler(&err) if req.Keyspace == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "keyspace field is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "keyspace field is required") return nil, err } if req.Shard == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "shard field is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "shard field is required") return nil, err } @@ -2247,6 +2247,40 @@ func (s *VtctldServer) InitShardPrimaryLocked( return nil } +// MoveTablesCreate is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) MoveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTablesCreateRequest) (resp *vtctldatapb.WorkflowStatusResponse, err error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.MoveTablesCreate") + defer span.Finish() + + defer panicHandler(&err) + + span.Annotate("keyspace", req.TargetKeyspace) + span.Annotate("workflow", req.Workflow) + span.Annotate("cells", req.Cells) + span.Annotate("tablet_types", req.TabletTypes) + span.Annotate("on_ddl", req.OnDdl) + + resp, err = s.ws.MoveTablesCreate(ctx, req) + return resp, err +} + +// MoveTablesComplete is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) MoveTablesComplete(ctx context.Context, req *vtctldatapb.MoveTablesCompleteRequest) (resp *vtctldatapb.MoveTablesCompleteResponse, err error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.MoveTablesComplete") + defer span.Finish() + + defer panicHandler(&err) + + span.Annotate("keyspace", req.TargetKeyspace) + span.Annotate("workflow", req.Workflow) + span.Annotate("keep_data", req.KeepData) + span.Annotate("keep_routing_rules", req.KeepRoutingRules) + span.Annotate("dry_run", req.DryRun) + + resp, err = s.ws.MoveTablesComplete(ctx, req) + return resp, err +} + // PingTablet is part of the vtctlservicepb.VtctldServer interface. func (s *VtctldServer) PingTablet(ctx context.Context, req *vtctldatapb.PingTabletRequest) (resp *vtctldatapb.PingTabletResponse, err error) { span, ctx := trace.NewSpan(ctx, "VtctldServer.PingTablet") @@ -2379,7 +2413,7 @@ func (s *VtctldServer) RefreshState(ctx context.Context, req *vtctldatapb.Refres defer panicHandler(&err) if req.TabletAlias == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "RefreshState requires a tablet alias") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "RefreshState requires a tablet alias") return nil, err } @@ -2407,12 +2441,12 @@ func (s *VtctldServer) RefreshStateByShard(ctx context.Context, req *vtctldatapb defer panicHandler(&err) if req.Keyspace == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "RefreshStateByShard requires a keyspace") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "RefreshStateByShard requires a keyspace") return nil, err } if req.Shard == "" { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "RefreshStateByShard requires a shard") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "RefreshStateByShard requires a shard") return nil, err } @@ -2456,7 +2490,7 @@ func (s *VtctldServer) ReloadSchema(ctx context.Context, req *vtctldatapb.Reload ti, err := s.ts.GetTablet(ctx, req.TabletAlias) if err != nil { - err = vterrors.Errorf(vtrpc.Code_NOT_FOUND, "GetTablet(%v) failed: %v", req.TabletAlias, err) + err = vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, "GetTablet(%v) failed: %v", req.TabletAlias, err) return nil, err } @@ -2518,7 +2552,7 @@ func (s *VtctldServer) ReloadSchemaKeyspace(ctx context.Context, req *vtctldatap shards, err := s.ts.GetShardNames(ctx, req.Keyspace) if err != nil { - err = vterrors.Errorf(vtrpc.Code_INTERNAL, "GetShardNames(%v) failed: %v", req.Keyspace, err) + err = vterrors.Errorf(vtrpcpb.Code_INTERNAL, "GetShardNames(%v) failed: %v", req.Keyspace, err) return nil, err } @@ -2643,7 +2677,7 @@ func (s *VtctldServer) ReparentTablet(ctx context.Context, req *vtctldatapb.Repa defer panicHandler(&err) if req.Tablet == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "tablet alias must not be nil") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "tablet alias must not be nil") return nil, err } @@ -2660,7 +2694,7 @@ func (s *VtctldServer) ReparentTablet(ctx context.Context, req *vtctldatapb.Repa } if !shard.HasPrimary() { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) return nil, err } @@ -2671,17 +2705,17 @@ func (s *VtctldServer) ReparentTablet(ctx context.Context, req *vtctldatapb.Repa } if shardPrimary.Type != topodatapb.TabletType_PRIMARY { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) return nil, err } if shardPrimary.Keyspace != tablet.Keyspace || shardPrimary.Shard != tablet.Shard { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "primary %v and potential replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), topoproto.TabletAliasString(req.Tablet), tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "primary %v and potential replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), topoproto.TabletAliasString(req.Tablet), tablet.Keyspace, tablet.Shard) return nil, err } if topoproto.TabletAliasEqual(req.Tablet, shardPrimary.Alias) { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "cannot ReparentTablet current shard primary (%v) onto itself", topoproto.TabletAliasString(req.Tablet)) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot ReparentTablet current shard primary (%v) onto itself", topoproto.TabletAliasString(req.Tablet)) return nil, err } @@ -2830,7 +2864,7 @@ func (s *VtctldServer) SetKeyspaceDurabilityPolicy(ctx context.Context, req *vtc policyValid := reparentutil.CheckDurabilityPolicyExists(req.DurabilityPolicy) if !policyValid { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "durability policy <%v> is not a valid policy. Please register it as a policy first", req.DurabilityPolicy) + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "durability policy <%v> is not a valid policy. Please register it as a policy first", req.DurabilityPolicy) return nil, err } @@ -2977,7 +3011,7 @@ func (s *VtctldServer) SetWritable(ctx context.Context, req *vtctldatapb.SetWrit defer panicHandler(&err) if req.TabletAlias == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "SetWritable.TabletAlias is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "SetWritable.TabletAlias is required") return nil, err } @@ -3346,7 +3380,7 @@ func (s *VtctldServer) StartReplication(ctx context.Context, req *vtctldatapb.St defer panicHandler(&err) if req.TabletAlias == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "StartReplication.TabletAlias is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "StartReplication.TabletAlias is required") return nil, err } @@ -3365,7 +3399,7 @@ func (s *VtctldServer) StartReplication(ctx context.Context, req *vtctldatapb.St } if !shard.HasPrimary() { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no primary tablet for shard %v/%v", tablet.Keyspace, tablet.Shard) return nil, err } @@ -3376,12 +3410,12 @@ func (s *VtctldServer) StartReplication(ctx context.Context, req *vtctldatapb.St } if shardPrimary.Type != topodatapb.TabletType_PRIMARY { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "TopologyServer has incosistent state for shard primary %v", topoproto.TabletAliasString(shard.PrimaryAlias)) return nil, err } if shardPrimary.Keyspace != tablet.Keyspace || shardPrimary.Shard != tablet.Shard { - err = vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "primary %v and replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), topoproto.TabletAliasString(tablet.Alias), tablet.Keyspace, tablet.Shard) + err = vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "primary %v and replica %v not in same keypace shard (%v/%v)", topoproto.TabletAliasString(shard.PrimaryAlias), topoproto.TabletAliasString(tablet.Alias), tablet.Keyspace, tablet.Shard) return nil, err } @@ -3411,7 +3445,7 @@ func (s *VtctldServer) StopReplication(ctx context.Context, req *vtctldatapb.Sto defer panicHandler(&err) if req.TabletAlias == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "StopReplication.TabletAlias is required") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "StopReplication.TabletAlias is required") return nil, err } @@ -3440,7 +3474,7 @@ func (s *VtctldServer) TabletExternallyReparented(ctx context.Context, req *vtct defer panicHandler(&err) if req.Tablet == nil { - err = vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "TabletExternallyReparentedRequest.Tablet must not be nil") + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "TabletExternallyReparentedRequest.Tablet must not be nil") return nil, err } @@ -4347,6 +4381,51 @@ func (s *VtctldServer) ValidateVSchema(ctx context.Context, req *vtctldatapb.Val return resp, err } +// WorkflowDelete is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) WorkflowDelete(ctx context.Context, req *vtctldatapb.WorkflowDeleteRequest) (resp *vtctldatapb.WorkflowDeleteResponse, err error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.WorkflowDelete") + defer span.Finish() + + defer panicHandler(&err) + + span.Annotate("keyspace", req.Keyspace) + span.Annotate("workflow", req.Workflow) + + resp, err = s.ws.WorkflowDelete(ctx, req) + return resp, err +} + +// WorkflowStatus is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) WorkflowStatus(ctx context.Context, req *vtctldatapb.WorkflowStatusRequest) (resp *vtctldatapb.WorkflowStatusResponse, err error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.WorkflowStatus") + defer span.Finish() + + defer panicHandler(&err) + + span.Annotate("keyspace", req.Keyspace) + span.Annotate("workflow", req.Workflow) + + resp, err = s.ws.WorkflowStatus(ctx, req) + return resp, err +} + +// WorkflowSwitchTraffic is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) WorkflowSwitchTraffic(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest) (resp *vtctldatapb.WorkflowSwitchTrafficResponse, err error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.WorkflowSwitchTraffic") + defer span.Finish() + + defer panicHandler(&err) + + span.Annotate("keyspace", req.Keyspace) + span.Annotate("workflow", req.Workflow) + span.Annotate("tablet-types", req.TabletTypes) + span.Annotate("direction", req.Direction) + span.Annotate("enable-reverse-replication", req.EnableReverseReplication) + + resp, err = s.ws.WorkflowSwitchTraffic(ctx, req) + return resp, err +} + // WorkflowUpdate is part of the vtctlservicepb.VtctldServer interface. func (s *VtctldServer) WorkflowUpdate(ctx context.Context, req *vtctldatapb.WorkflowUpdateRequest) (resp *vtctldatapb.WorkflowUpdateResponse, err error) { span, ctx := trace.NewSpan(ctx, "VtctldServer.WorkflowUpdate") @@ -4374,7 +4453,7 @@ func (s *VtctldServer) getTopologyCell(ctx context.Context, cellPath string) (*v // extract cell and relative path parts := strings.Split(cellPath, "/") if parts[0] != "" || len(parts) < 2 { - err := vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "invalid path: %s", cellPath) + err := vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid path: %s", cellPath) return nil, err } cell := parts[1] @@ -4383,7 +4462,7 @@ func (s *VtctldServer) getTopologyCell(ctx context.Context, cellPath string) (*v conn, err := s.ts.ConnForCell(ctx, cell) if err != nil { - err := vterrors.Errorf(vtrpc.Code_UNAVAILABLE, "error fetching connection to cell %s: %v", cell, err) + err := vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "error fetching connection to cell %s: %v", cell, err) return nil, err } @@ -4392,7 +4471,7 @@ func (s *VtctldServer) getTopologyCell(ctx context.Context, cellPath string) (*v if dataErr == nil { result, err := topo.DecodeContent(relativePath, data, false) if err != nil { - err := vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "error decoding file content for cell %s: %v", cellPath, err) + err := vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "error decoding file content for cell %s: %v", cellPath, err) return nil, err } topoCell.Data = result @@ -4404,7 +4483,7 @@ func (s *VtctldServer) getTopologyCell(ctx context.Context, cellPath string) (*v children, childrenErr := conn.ListDir(ctx, relativePath, false /*full*/) if childrenErr != nil && dataErr != nil { - err := vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "cell %s with path %s has no file contents and no children: %v", cell, cellPath, err) + err := vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cell %s with path %s has no file contents and no children: %v", cell, cellPath, err) return nil, err } diff --git a/go/vt/vtctl/localvtctldclient/client_gen.go b/go/vt/vtctl/localvtctldclient/client_gen.go index e516bb3063c..dddd8067cb3 100644 --- a/go/vt/vtctl/localvtctldclient/client_gen.go +++ b/go/vt/vtctl/localvtctldclient/client_gen.go @@ -346,6 +346,16 @@ func (client *localVtctldClient) InitShardPrimary(ctx context.Context, in *vtctl return client.s.InitShardPrimary(ctx, in) } +// MoveTablesComplete is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) MoveTablesComplete(ctx context.Context, in *vtctldatapb.MoveTablesCompleteRequest, opts ...grpc.CallOption) (*vtctldatapb.MoveTablesCompleteResponse, error) { + return client.s.MoveTablesComplete(ctx, in) +} + +// MoveTablesCreate is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) MoveTablesCreate(ctx context.Context, in *vtctldatapb.MoveTablesCreateRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowStatusResponse, error) { + return client.s.MoveTablesCreate(ctx, in) +} + // PingTablet is part of the vtctlservicepb.VtctldClient interface. func (client *localVtctldClient) PingTablet(ctx context.Context, in *vtctldatapb.PingTabletRequest, opts ...grpc.CallOption) (*vtctldatapb.PingTabletResponse, error) { return client.s.PingTablet(ctx, in) @@ -587,6 +597,21 @@ func (client *localVtctldClient) ValidateVersionShard(ctx context.Context, in *v return client.s.ValidateVersionShard(ctx, in) } +// WorkflowDelete is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) WorkflowDelete(ctx context.Context, in *vtctldatapb.WorkflowDeleteRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowDeleteResponse, error) { + return client.s.WorkflowDelete(ctx, in) +} + +// WorkflowStatus is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) WorkflowStatus(ctx context.Context, in *vtctldatapb.WorkflowStatusRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowStatusResponse, error) { + return client.s.WorkflowStatus(ctx, in) +} + +// WorkflowSwitchTraffic is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) WorkflowSwitchTraffic(ctx context.Context, in *vtctldatapb.WorkflowSwitchTrafficRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowSwitchTrafficResponse, error) { + return client.s.WorkflowSwitchTraffic(ctx, in) +} + // WorkflowUpdate is part of the vtctlservicepb.VtctldClient interface. func (client *localVtctldClient) WorkflowUpdate(ctx context.Context, in *vtctldatapb.WorkflowUpdateRequest, opts ...grpc.CallOption) (*vtctldatapb.WorkflowUpdateResponse, error) { return client.s.WorkflowUpdate(ctx, in) diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index 023669a4619..fa9d8a1e1d0 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -105,6 +105,7 @@ import ( "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/discovery" hk "vitess.io/vitess/go/vt/hook" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" @@ -2042,11 +2043,11 @@ func commandValidateKeyspace(ctx context.Context, wr *wrangler.Wrangler, subFlag } func commandReshard(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string) error { - return commandVRWorkflow(ctx, wr, subFlags, args, wrangler.ReshardWorkflow) + return commandVReplicationWorkflow(ctx, wr, subFlags, args, wrangler.ReshardWorkflow) } func commandMoveTables(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string) error { - return commandVRWorkflow(ctx, wr, subFlags, args, wrangler.MoveTablesWorkflow) + return commandVReplicationWorkflow(ctx, wr, subFlags, args, wrangler.MoveTablesWorkflow) } // VReplicationWorkflowAction defines subcommands passed to vtctl for movetables or reshard @@ -2064,7 +2065,7 @@ const ( ) func commandMigrate(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string) error { - return commandVRWorkflow(ctx, wr, subFlags, args, wrangler.MigrateWorkflow) + return commandVReplicationWorkflow(ctx, wr, subFlags, args, wrangler.MigrateWorkflow) } // getSourceKeyspace expects a keyspace of the form "externalClusterName.keyspaceName" and returns the components @@ -2076,9 +2077,9 @@ func getSourceKeyspace(clusterKeyspace string) (clusterName string, sourceKeyspa return splits[0], splits[1], nil } -// commandVRWorkflow is the common entry point for MoveTables/Reshard/Migrate workflows +// commandVReplicationWorkflow is the common entry point for MoveTables/Reshard/Migrate workflows // FIXME: this function needs a refactor. Also validations for params should to be done per workflow type -func commandVRWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string, +func commandVReplicationWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string, workflowType wrangler.VReplicationWorkflowType) error { const defaultWaitTime = time.Duration(30 * time.Second) @@ -2087,7 +2088,7 @@ func commandVRWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pfl const defaultMaxReplicationLagAllowed = defaultWaitTime cells := subFlags.String("cells", "", "Cell(s) or CellAlias(es) (comma-separated) to replicate from.") - tabletTypes := subFlags.String("tablet_types", "in_order:REPLICA,PRIMARY", "Source tablet types to replicate from (e.g. PRIMARY, REPLICA, RDONLY). Defaults to --vreplication_tablet_type parameter value for the tablet, which has the default value of in_order:REPLICA,PRIMARY. Note: SwitchTraffic overrides this default and uses in_order:RDONLY,REPLICA,PRIMARY to switch all traffic by default.") + tabletTypesStr := subFlags.String("tablet_types", "in_order:REPLICA,PRIMARY", "Source tablet types to replicate from (e.g. PRIMARY, REPLICA, RDONLY). Defaults to --vreplication_tablet_type parameter value for the tablet, which has the default value of in_order:REPLICA,PRIMARY. Note: SwitchTraffic overrides this default and uses in_order:RDONLY,REPLICA,PRIMARY to switch all traffic by default.") dryRun := subFlags.Bool("dry_run", false, "Does a dry run of SwitchTraffic and only reports the actions to be taken. --dry_run is only supported for SwitchTraffic, ReverseTraffic and Complete.") timeout := subFlags.Duration("timeout", defaultWaitTime, "Specifies the maximum time to wait, in seconds, for vreplication to catch up on primary migrations. The migration will be cancelled on a timeout. --timeout is only supported for SwitchTraffic and ReverseTraffic.") reverseReplication := subFlags.Bool("reverse_replication", true, "Also reverse the replication (default true). --reverse_replication is only supported for SwitchTraffic.") @@ -2175,7 +2176,7 @@ func commandVRWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pfl statuses := res.ShardStatuses[ksShard].PrimaryReplicationStatuses for _, st := range statuses { msg := "" - if st.State == "Error" { + if st.State == binlogdatapb.VReplicationWorkflowState_Error.String() { msg += fmt.Sprintf(": %s.", st.Message) } else if st.Pos == "" { msg += ". VStream has not started." @@ -2274,11 +2275,11 @@ func commandVRWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pfl vrwp.OnDDL = onDDL vrwp.DeferSecondaryKeys = *deferNonPKeys vrwp.Cells = *cells - vrwp.TabletTypes = *tabletTypes + vrwp.TabletTypes = *tabletTypesStr case vReplicationWorkflowActionSwitchTraffic, vReplicationWorkflowActionReverseTraffic: vrwp.Cells = *cells if subFlags.Changed("tablet_types") { - vrwp.TabletTypes = *tabletTypes + vrwp.TabletTypes = *tabletTypesStr } else { // When no tablet types are specified we are supposed to switch all traffic so // we override the normal default for tablet_types. @@ -2489,7 +2490,7 @@ func commandExternalizeVindex(ctx context.Context, wr *wrangler.Wrangler, subFla func commandMaterialize(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.FlagSet, args []string) error { cells := subFlags.String("cells", "", "Source cells to replicate from.") - tabletTypes := subFlags.String("tablet_types", "", "Source tablet types to replicate from.") + tabletTypesStr := subFlags.String("tablet_types", "", "Source tablet types to replicate from.") if err := subFlags.Parse(args); err != nil { return err } @@ -2501,7 +2502,16 @@ func commandMaterialize(ctx context.Context, wr *wrangler.Wrangler, subFlags *pf return err } ms.Cell = *cells - ms.TabletTypes = *tabletTypes + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(*tabletTypesStr) + if err != nil { + return err + } + tsp := tabletmanagerdatapb.TabletSelectionPreference_ANY + if inorder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + ms.TabletTypes = topoproto.MakeStringTypeCSV(tabletTypes) + ms.TabletSelectionPreference = tsp return wr.Materialize(ctx, ms) } @@ -2522,7 +2532,7 @@ func commandVDiff(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.Fl sourceCell := subFlags.String("source_cell", "", "The source cell to compare from; default is any available cell") targetCell := subFlags.String("target_cell", "", "The target cell to compare with; default is any available cell") - tabletTypes := subFlags.String("tablet_types", "in_order:RDONLY,REPLICA,PRIMARY", "Tablet types for source and target") + tabletTypesStr := subFlags.String("tablet_types", "in_order:RDONLY,REPLICA,PRIMARY", "Tablet types for source and target") filteredReplicationWaitTime := subFlags.Duration("filtered_replication_wait_time", 30*time.Second, "Specifies the maximum time to wait, in seconds, for filtered replication to catch up on primary migrations. The migration will be cancelled on a timeout.") maxRows := subFlags.Int64("limit", math.MaxInt64, "Max rows to stop comparing after") debugQuery := subFlags.Bool("debug_query", false, "Adds a mysql query to the report that can be used for further debugging") @@ -2552,7 +2562,7 @@ func commandVDiff(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.Fl } }() - _, err = wr.VDiff(ctx, keyspace, workflow, *sourceCell, *targetCell, *tabletTypes, *filteredReplicationWaitTime, *format, + _, err = wr.VDiff(ctx, keyspace, workflow, *sourceCell, *targetCell, *tabletTypesStr, *filteredReplicationWaitTime, *format, *maxRows, *tables, *debugQuery, *onlyPks, *maxExtraRowsToCompare) if err != nil { log.Errorf("vdiff returning with error: %v", err) @@ -3674,7 +3684,7 @@ func commandWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag usage := "usage: Workflow [--dry-run] [--cells] [--tablet-types] [.] start/stop/update/delete/show/listall/tags []" dryRun := subFlags.Bool("dry-run", false, "Does a dry run of the Workflow action and reports the query and list of tablets on which the operation will be applied") cells := subFlags.StringSlice("cells", []string{}, "New Cell(s) or CellAlias(es) (comma-separated) to replicate from. (Update only)") - tabletTypes := subFlags.StringSlice("tablet-types", []string{}, "New source tablet types to replicate from (e.g. PRIMARY, REPLICA, RDONLY). (Update only)") + tabletTypesStrs := subFlags.StringSlice("tablet-types", []string{}, "New source tablet types to replicate from (e.g. PRIMARY, REPLICA, RDONLY). (Update only)") onDDL := subFlags.String("on-ddl", "", "New instruction on what to do when DDL is encountered in the VReplication stream. Possible values are IGNORE, STOP, EXEC, and EXEC_IGNORE. (Update only)") if err := subFlags.Parse(args); err != nil { return err @@ -3727,16 +3737,22 @@ func commandWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag } else { cells = &textutil.SimulatedNullStringSlice } + tabletTypes := make([]topodatapb.TabletType, len(*tabletTypesStrs)) + inorder := false if subFlags.Lookup("tablet-types").Changed { // Validate the provided value(s) changes = true - for i, tabletType := range *tabletTypes { - (*tabletTypes)[i] = strings.ToUpper(strings.TrimSpace(tabletType)) - if _, err = topoproto.ParseTabletType((*tabletTypes)[i]); err != nil { + if len(*tabletTypesStrs) > 0 && strings.HasPrefix((*tabletTypesStrs)[0], discovery.InOrderHint) { + (*tabletTypesStrs)[0] = strings.TrimPrefix((*tabletTypesStrs)[0], discovery.InOrderHint) + inorder = true + } + for i, tabletType := range *tabletTypesStrs { + tabletTypes[i], err = topoproto.ParseTabletType(tabletType) + if err != nil { return err } } } else { - tabletTypes = &textutil.SimulatedNullStringSlice + tabletTypes = []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)} } onddl := int32(textutil.SimulatedNullInt) // To signify no value has been provided if subFlags.Lookup("on-ddl").Changed { // Validate the provided value @@ -3750,11 +3766,16 @@ func commandWorkflow(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag if !changes { return fmt.Errorf(errWorkflowUpdateWithoutChanges) } - rpcReq = &tabletmanagerdatapb.UpdateVRWorkflowRequest{ - Workflow: workflow, - Cells: *cells, - TabletTypes: *tabletTypes, - OnDdl: binlogdatapb.OnDDLAction(onddl), + tsp := tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN + if inorder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + rpcReq = &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ + Workflow: workflow, + Cells: *cells, + TabletTypes: tabletTypes, + TabletSelectionPreference: tsp, + OnDdl: binlogdatapb.OnDDLAction(onddl), } } results, err = wr.WorkflowAction(ctx, workflow, keyspace, action, *dryRun, rpcReq) // Only update currently uses the new RPC path diff --git a/go/vt/vtctl/vtctl_test.go b/go/vt/vtctl/vtctl_test.go index 0ee4096a333..94937e06ce2 100644 --- a/go/vt/vtctl/vtctl_test.go +++ b/go/vt/vtctl/vtctl_test.go @@ -139,7 +139,7 @@ Dry run: Skipping update of VSchema`, } // TestMoveTables tests the the MoveTables client command -// via the commandVRWorkflow() cmd handler. +// via the commandVReplicationWorkflow() cmd handler. // This currently only tests the Progress action (which is // a parent of the Show action) but it can be used to test // other actions as well. @@ -349,7 +349,7 @@ func TestMoveTables(t *testing.T) { subFlags := pflag.NewFlagSet("test", pflag.ContinueOnError) expectGlobalResults() tt.expectResults() - err := commandVRWorkflow(ctx, env.wr, subFlags, tt.args, tt.workflowType) + err := commandVReplicationWorkflow(ctx, env.wr, subFlags, tt.args, tt.workflowType) require.NoError(t, err) if strings.HasPrefix(tt.want, "/") { require.Regexp(t, tt.want[1:], env.cmdlog.String()) diff --git a/go/vt/vtctl/workflow/log_recorder.go b/go/vt/vtctl/workflow/log_recorder.go new file mode 100644 index 00000000000..c35ef562354 --- /dev/null +++ b/go/vt/vtctl/workflow/log_recorder.go @@ -0,0 +1,58 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "fmt" + "sort" +) + +// LogRecorder is used to collect logs for a specific purpose. +// Not thread-safe since it is expected to be generated in repeatable sequence +type LogRecorder struct { + logs []string +} + +// NewLogRecorder creates a new instance of LogRecorder +func NewLogRecorder() *LogRecorder { + lr := LogRecorder{} + return &lr +} + +// Log records a new log message +func (lr *LogRecorder) Log(log string) { + lr.logs = append(lr.logs, log) +} + +// Logf records a new log message with interpolation parameters using fmt.Sprintf. +func (lr *LogRecorder) Logf(log string, args ...any) { + lr.logs = append(lr.logs, fmt.Sprintf(log, args...)) +} + +// LogSlice sorts a given slice using natural sort, so that the result is predictable. +// Useful when logging arrays or maps where order of objects can vary +func (lr *LogRecorder) LogSlice(logs []string) { + sort.Strings(logs) + for _, log := range logs { + lr.Log(log) + } +} + +// GetLogs returns all recorded logs in sequence +func (lr *LogRecorder) GetLogs() []string { + return lr.logs +} diff --git a/go/vt/vtctl/workflow/log_recorder_test.go b/go/vt/vtctl/workflow/log_recorder_test.go new file mode 100644 index 00000000000..b58d1d42a79 --- /dev/null +++ b/go/vt/vtctl/workflow/log_recorder_test.go @@ -0,0 +1,35 @@ +/* +Copyright 2020 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 workflow + +import ( + "testing" + "time" + + "github.com/magiconair/properties/assert" +) + +func TestLogRecorder(t *testing.T) { + lr := NewLogRecorder() + now := time.August + lr.Log("log 1") + lr.Log("log 2") + lr.Logf("log 3 with params: %s, %v, %d", "param1", now, 3) + lr.LogSlice([]string{"log 4", "log 5"}) + want := []string{"log 1", "log 2", "log 3 with params: param1, August, 3", "log 4", "log 5"} + assert.Equal(t, lr.GetLogs(), want) +} diff --git a/go/vt/vtctl/workflow/materializer.go b/go/vt/vtctl/workflow/materializer.go new file mode 100644 index 00000000000..15da7a20107 --- /dev/null +++ b/go/vt/vtctl/workflow/materializer.go @@ -0,0 +1,607 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "fmt" + "strings" + "sync" + "text/template" + "time" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/mysqlctl/tmutils" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/vtctl/schematools" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/vindexes" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" + "vitess.io/vitess/go/vt/vttablet/tmclient" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" +) + +const ( + createDDLAsCopy = "copy" + createDDLAsCopyDropConstraint = "copy:drop_constraint" + createDDLAsCopyDropForeignKeys = "copy:drop_foreign_keys" +) + +type materializer struct { + ctx context.Context + ts *topo.Server + sourceTs *topo.Server + tmc tmclient.TabletManagerClient + + ms *vtctldatapb.MaterializeSettings + targetVSchema *vindexes.KeyspaceSchema + sourceShards []*topo.ShardInfo + targetShards []*topo.ShardInfo + isPartial bool +} + +func (mz *materializer) prepareMaterializerStreams(req *vtctldatapb.MoveTablesCreateRequest) error { + if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { + return err + } + err := mz.buildMaterializer() + if err != nil { + return err + } + if mz.isPartial { + if err := createDefaultShardRoutingRules(mz.ctx, mz.ms, mz.ts); err != nil { + return err + } + } + if err := mz.deploySchema(); err != nil { + return err + } + workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None + if mz.isPartial { + workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial + } + return mz.forAllTargets(func(target *topo.ShardInfo) error { + targetPrimary, err := mz.ts.GetTablet(mz.ctx, target.PrimaryAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) + } + blses, err := mz.generateBinlogSources(mz.ctx, target) + if err != nil { + return err + } + _, err = mz.tmc.CreateVReplicationWorkflow(mz.ctx, targetPrimary.Tablet, &tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + Workflow: req.Workflow, + BinlogSource: blses, + Cells: req.Cells, + TabletTypes: req.TabletTypes, + TabletSelectionPreference: req.TabletSelectionPreference, + WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, + WorkflowSubType: workflowSubType, + DeferSecondaryKeys: req.DeferSecondaryKeys, + AutoStart: req.AutoStart, + StopAfterCopy: req.StopAfterCopy, + }) + return err + }) +} + +func (mz *materializer) createMaterializerStreams() error { + if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { + return err + } + err := mz.buildMaterializer() + if err != nil { + return err + } + if mz.isPartial { + if err := createDefaultShardRoutingRules(mz.ctx, mz.ms, mz.ts); err != nil { + return err + } + } + if err := mz.deploySchema(); err != nil { + return err + } + insertMap := make(map[string]string, len(mz.targetShards)) + for _, targetShard := range mz.targetShards { + inserts, err := mz.generateInserts(mz.ctx, targetShard) + if err != nil { + return err + } + insertMap[key.KeyRangeString(targetShard.KeyRange)] = inserts + } + if err := mz.createStreams(mz.ctx, insertMap); err != nil { + return err + } + return nil +} + +func (mz *materializer) generateInserts(ctx context.Context, targetShard *topo.ShardInfo) (string, error) { + ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, "{{.dbname}}") + + for _, sourceShard := range mz.sourceShards { + // Don't create streams from sources which won't contain data for the target shard. + // We only do it for MoveTables for now since this doesn't hold for materialize flows + // where the target's sharding key might differ from that of the source + if mz.ms.MaterializationIntent == vtctldatapb.MaterializationIntent_MOVETABLES && + !key.KeyRangeIntersect(sourceShard.KeyRange, targetShard.KeyRange) { + continue + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: mz.ms.SourceKeyspace, + Shard: sourceShard.ShardName(), + Filter: &binlogdatapb.Filter{}, + StopAfterCopy: mz.ms.StopAfterCopy, + ExternalCluster: mz.ms.ExternalCluster, + SourceTimeZone: mz.ms.SourceTimeZone, + TargetTimeZone: mz.ms.TargetTimeZone, + OnDdl: binlogdatapb.OnDDLAction(binlogdatapb.OnDDLAction_value[mz.ms.OnDdl]), + } + for _, ts := range mz.ms.TableSettings { + rule := &binlogdatapb.Rule{ + Match: ts.TargetTable, + } + + if ts.SourceExpression == "" { + bls.Filter.Rules = append(bls.Filter.Rules, rule) + continue + } + + // Validate non-empty query. + stmt, err := sqlparser.Parse(ts.SourceExpression) + if err != nil { + return "", err + } + sel, ok := stmt.(*sqlparser.Select) + if !ok { + return "", fmt.Errorf("unrecognized statement: %s", ts.SourceExpression) + } + filter := ts.SourceExpression + if mz.targetVSchema.Keyspace.Sharded && mz.targetVSchema.Tables[ts.TargetTable].Type != vindexes.TypeReference { + cv, err := vindexes.FindBestColVindex(mz.targetVSchema.Tables[ts.TargetTable]) + if err != nil { + return "", err + } + mappedCols := make([]*sqlparser.ColName, 0, len(cv.Columns)) + for _, col := range cv.Columns { + colName, err := matchColInSelect(col, sel) + if err != nil { + return "", err + } + mappedCols = append(mappedCols, colName) + } + subExprs := make(sqlparser.SelectExprs, 0, len(mappedCols)+2) + for _, mappedCol := range mappedCols { + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: mappedCol}) + } + vindexName := fmt.Sprintf("%s.%s", mz.ms.TargetKeyspace, cv.Name) + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral(vindexName)}) + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral("{{.keyrange}}")}) + inKeyRange := &sqlparser.FuncExpr{ + 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) + } + + rule.Filter = filter + + bls.Filter.Rules = append(bls.Filter.Rules, rule) + } + workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None + if mz.isPartial { + workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial + } + var workflowType binlogdatapb.VReplicationWorkflowType + switch mz.ms.MaterializationIntent { + case vtctldatapb.MaterializationIntent_CUSTOM: + workflowType = binlogdatapb.VReplicationWorkflowType_Materialize + case vtctldatapb.MaterializationIntent_MOVETABLES: + workflowType = binlogdatapb.VReplicationWorkflowType_MoveTables + case vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX: + workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex + } + ig.AddRow(mz.ms.Workflow, bls, "", mz.ms.Cell, mz.ms.TabletTypes, + workflowType, + workflowSubType, mz.ms.DeferSecondaryKeys) + } + return ig.String(), nil +} + +func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard *topo.ShardInfo) ([]*binlogdatapb.BinlogSource, error) { + blses := make([]*binlogdatapb.BinlogSource, 0, len(mz.sourceShards)) + for _, sourceShard := range mz.sourceShards { + // Don't create streams from sources which won't contain data for the target shard. + // We only do it for MoveTables for now since this doesn't hold for materialize flows + // where the target's sharding key might differ from that of the source + if mz.ms.MaterializationIntent == vtctldatapb.MaterializationIntent_MOVETABLES && + !key.KeyRangeIntersect(sourceShard.KeyRange, targetShard.KeyRange) { + continue + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: mz.ms.SourceKeyspace, + Shard: sourceShard.ShardName(), + Filter: &binlogdatapb.Filter{}, + StopAfterCopy: mz.ms.StopAfterCopy, + ExternalCluster: mz.ms.ExternalCluster, + SourceTimeZone: mz.ms.SourceTimeZone, + TargetTimeZone: mz.ms.TargetTimeZone, + OnDdl: binlogdatapb.OnDDLAction(binlogdatapb.OnDDLAction_value[mz.ms.OnDdl]), + } + for _, ts := range mz.ms.TableSettings { + rule := &binlogdatapb.Rule{ + Match: ts.TargetTable, + } + + if ts.SourceExpression == "" { + bls.Filter.Rules = append(bls.Filter.Rules, rule) + continue + } + + // Validate non-empty query. + stmt, err := sqlparser.Parse(ts.SourceExpression) + if err != nil { + return nil, err + } + sel, ok := stmt.(*sqlparser.Select) + if !ok { + return nil, fmt.Errorf("unrecognized statement: %s", ts.SourceExpression) + } + filter := ts.SourceExpression + if mz.targetVSchema.Keyspace.Sharded && mz.targetVSchema.Tables[ts.TargetTable].Type != vindexes.TypeReference { + cv, err := vindexes.FindBestColVindex(mz.targetVSchema.Tables[ts.TargetTable]) + if err != nil { + return nil, err + } + mappedCols := make([]*sqlparser.ColName, 0, len(cv.Columns)) + for _, col := range cv.Columns { + colName, err := matchColInSelect(col, sel) + if err != nil { + return nil, err + } + mappedCols = append(mappedCols, colName) + } + subExprs := make(sqlparser.SelectExprs, 0, len(mappedCols)+2) + for _, mappedCol := range mappedCols { + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: mappedCol}) + } + vindexName := fmt.Sprintf("%s.%s", mz.ms.TargetKeyspace, cv.Name) + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral(vindexName)}) + subExprs = append(subExprs, &sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral(key.KeyRangeString(targetShard.KeyRange))}) + inKeyRange := &sqlparser.FuncExpr{ + 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) + } + + rule.Filter = filter + bls.Filter.Rules = append(bls.Filter.Rules, rule) + } + blses = append(blses, bls) + } + return blses, nil +} + +func (mz *materializer) deploySchema() error { + var sourceDDLs map[string]string + var mu sync.Mutex + + return forAllShards(mz.targetShards, func(target *topo.ShardInfo) error { + allTables := []string{"/.*/"} + + hasTargetTable := map[string]bool{} + req := &tabletmanagerdatapb.GetSchemaRequest{Tables: allTables} + targetSchema, err := schematools.GetSchema(mz.ctx, mz.ts, mz.tmc, target.PrimaryAlias, req) + if err != nil { + return err + } + + for _, td := range targetSchema.TableDefinitions { + hasTargetTable[td.Name] = true + } + + targetTablet, err := mz.ts.GetTablet(mz.ctx, target.PrimaryAlias) + if err != nil { + return err + } + + var applyDDLs []string + for _, ts := range mz.ms.TableSettings { + if hasTargetTable[ts.TargetTable] { + // Table already exists. + continue + } + if ts.CreateDdl == "" { + return fmt.Errorf("target table %v does not exist and there is no create ddl defined", ts.TargetTable) + } + + var err error + mu.Lock() + if len(sourceDDLs) == 0 { + // Only get DDLs for tables once and lazily: if we need to copy the schema from source + // to target then we copy schemas from primaries on the source keyspace; we have found + // use cases where the user just has a replica (no primary) in the source keyspace. + sourceDDLs, err = getSourceTableDDLs(mz.ctx, mz.sourceTs, mz.tmc, mz.sourceShards) + } + mu.Unlock() + if err != nil { + log.Errorf("Error getting DDLs of source tables: %s", err.Error()) + return err + } + + createDDL := ts.CreateDdl + if createDDL == createDDLAsCopy || createDDL == createDDLAsCopyDropConstraint || createDDL == createDDLAsCopyDropForeignKeys { + if ts.SourceExpression != "" { + // Check for table if non-empty SourceExpression. + sourceTableName, err := sqlparser.TableFromStatement(ts.SourceExpression) + if err != nil { + return err + } + if sourceTableName.Name.String() != ts.TargetTable { + return fmt.Errorf("source and target table names must match for copying schema: %v vs %v", sqlparser.String(sourceTableName), ts.TargetTable) + + } + } + + ddl, ok := sourceDDLs[ts.TargetTable] + if !ok { + return fmt.Errorf("source table %v does not exist", ts.TargetTable) + } + + if createDDL == createDDLAsCopyDropConstraint { + strippedDDL, err := stripTableConstraints(ddl) + if err != nil { + return err + } + + ddl = strippedDDL + } + + if createDDL == createDDLAsCopyDropForeignKeys { + strippedDDL, err := stripTableForeignKeys(ddl) + if err != nil { + return err + } + + ddl = strippedDDL + } + createDDL = ddl + } + + applyDDLs = append(applyDDLs, createDDL) + } + + if len(applyDDLs) > 0 { + sql := strings.Join(applyDDLs, ";\n") + + _, err = mz.tmc.ApplySchema(mz.ctx, targetTablet.Tablet, &tmutils.SchemaChange{ + SQL: sql, + Force: false, + AllowReplication: true, + SQLMode: vreplication.SQLMode, + }) + if err != nil { + return err + } + } + + return nil + }) +} + +func (mz *materializer) buildMaterializer() error { + ctx := mz.ctx + ms := mz.ms + vschema, err := mz.ts.GetVSchema(ctx, ms.TargetKeyspace) + if err != nil { + return err + } + targetVSchema, err := vindexes.BuildKeyspaceSchema(vschema, ms.TargetKeyspace) + if err != nil { + return err + } + if targetVSchema.Keyspace.Sharded { + for _, ts := range ms.TableSettings { + if targetVSchema.Tables[ts.TargetTable] == nil { + return fmt.Errorf("table %s not found in vschema for keyspace %s", ts.TargetTable, ms.TargetKeyspace) + } + } + } + isPartial := false + sourceShards, err := mz.sourceTs.GetServingShards(ctx, ms.SourceKeyspace) + if err != nil { + return err + } + if len(ms.SourceShards) > 0 { + isPartial = true + var sourceShards2 []*topo.ShardInfo + for _, shard := range sourceShards { + for _, shard2 := range ms.SourceShards { + if shard.ShardName() == shard2 { + sourceShards2 = append(sourceShards2, shard) + break + } + } + } + sourceShards = sourceShards2 + } + if len(sourceShards) == 0 { + return fmt.Errorf("no source shards specified for workflow %s ", ms.Workflow) + } + + targetShards, err := mz.ts.GetServingShards(ctx, ms.TargetKeyspace) + if err != nil { + return err + } + if len(ms.SourceShards) > 0 { + var targetShards2 []*topo.ShardInfo + for _, shard := range targetShards { + for _, shard2 := range ms.SourceShards { + if shard.ShardName() == shard2 { + targetShards2 = append(targetShards2, shard) + break + } + } + } + targetShards = targetShards2 + } + if len(targetShards) == 0 { + return fmt.Errorf("no target shards specified for workflow %s ", ms.Workflow) + } + mz.targetVSchema = targetVSchema + mz.sourceShards = sourceShards + mz.targetShards = targetShards + mz.isPartial = isPartial + return nil +} + +func (mz *materializer) createStreams(ctx context.Context, insertsMap map[string]string) error { + return forAllShards(mz.targetShards, func(target *topo.ShardInfo) error { + keyRange := key.KeyRangeString(target.KeyRange) + inserts := insertsMap[keyRange] + targetPrimary, err := mz.ts.GetTablet(ctx, target.PrimaryAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) + } + buf := &strings.Builder{} + t := template.Must(template.New("").Parse(inserts)) + input := map[string]string{ + "keyrange": keyRange, + "dbname": targetPrimary.DbName(), + } + if err := t.Execute(buf, input); err != nil { + return err + } + if _, err := mz.tmc.VReplicationExec(ctx, targetPrimary.Tablet, buf.String()); err != nil { + return err + } + return nil + }) +} + +func (mz *materializer) startStreams(ctx context.Context) error { + return forAllShards(mz.targetShards, func(target *topo.ShardInfo) error { + targetPrimary, err := mz.ts.GetTablet(ctx, target.PrimaryAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) + } + query := fmt.Sprintf("update _vt.vreplication set state='Running' where db_name=%s and workflow=%s", encodeString(targetPrimary.DbName()), encodeString(mz.ms.Workflow)) + if _, err := mz.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + } + return nil + }) +} + +func Materialize(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, ms *vtctldatapb.MaterializeSettings) error { + mz := &materializer{ + ctx: ctx, + ts: ts, + sourceTs: ts, + tmc: tmc, + ms: ms, + } + + err := mz.createMaterializerStreams() + if err != nil { + return err + } + return mz.startStreams(ctx) +} + +func (mz *materializer) forAllTargets(f func(*topo.ShardInfo) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, target := range mz.targetShards { + wg.Add(1) + go func(target *topo.ShardInfo) { + defer wg.Done() + + if err := f(target); err != nil { + allErrors.RecordError(err) + } + }(target) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} + +// checkTZConversion is a light-weight consistency check to validate that, if a source time zone is specified to MoveTables, +// that the current primary has the time zone loaded in order to run the convert_tz() function used by VReplication to do the +// datetime conversions. We only check the current primaries on each shard and note here that it is possible a new primary +// gets elected: in this case user will either see errors during vreplication or vdiff will report mismatches. +func (mz *materializer) checkTZConversion(ctx context.Context, tz string) error { + err := mz.forAllTargets(func(target *topo.ShardInfo) error { + targetPrimary, err := mz.ts.GetTablet(ctx, target.PrimaryAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) + } + testDateTime := "2006-01-02 15:04:05" + query := fmt.Sprintf("select convert_tz(%s, %s, 'UTC')", encodeString(testDateTime), encodeString(tz)) + qrproto, err := mz.tmc.ExecuteFetchAsApp(ctx, targetPrimary.Tablet, false, &tabletmanagerdatapb.ExecuteFetchAsAppRequest{ + Query: []byte(query), + MaxRows: 1, + }) + if err != nil { + return vterrors.Wrapf(err, "ExecuteFetchAsApp(%v, %s)", targetPrimary.Tablet, query) + } + qr := sqltypes.Proto3ToResult(qrproto) + if gotDate, err := time.Parse(testDateTime, qr.Rows[0][0].ToString()); err != nil { + return fmt.Errorf("unable to perform time_zone conversions from %s to UTC — value from DB was: %+v and the result of the attempt was: %s. Either the specified source time zone is invalid or the time zone tables have not been loaded on the %s tablet", + tz, qr.Rows, gotDate, targetPrimary.Alias) + } + return nil + }) + return err +} diff --git a/go/vt/vtctl/workflow/materializer_env_test.go b/go/vt/vtctl/workflow/materializer_env_test.go new file mode 100644 index 00000000000..430cfb71053 --- /dev/null +++ b/go/vt/vtctl/workflow/materializer_env_test.go @@ -0,0 +1,308 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "fmt" + "os" + "regexp" + "strconv" + "strings" + "sync" + "testing" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/mysqlctl/tmutils" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/vttablet/tmclient" + + _flag "vitess.io/vitess/go/internal/flag" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" +) + +type queryResult struct { + query string + result *querypb.QueryResult +} + +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 + topoServ *topo.Server + cell string + tmc *testMaterializerTMClient +} + +//---------------------------------------------- +// testMaterializerEnv + +func TestMain(m *testing.M) { + _flag.ParseFlagsForTest() + os.Exit(m.Run()) +} + +func newTestMaterializerEnv(t *testing.T, ms *vtctldatapb.MaterializeSettings, sources, targets []string) *testMaterializerEnv { + t.Helper() + env := &testMaterializerEnv{ + ms: ms, + sources: sources, + targets: targets, + tablets: make(map[int]*topodatapb.Tablet), + topoServ: memorytopo.NewServer("cell"), + cell: "cell", + tmc: newTestMaterializerTMClient(), + } + env.ws = NewServer(env.topoServ, env.tmc) + tabletID := 100 + for _, shard := range sources { + _ = env.addTablet(tabletID, env.ms.SourceKeyspace, shard, topodatapb.TabletType_PRIMARY) + tabletID += 10 + } + if ms.SourceKeyspace != ms.TargetKeyspace { + tabletID = 200 + for _, shard := range targets { + _ = env.addTablet(tabletID, env.ms.TargetKeyspace, shard, topodatapb.TabletType_PRIMARY) + tabletID += 10 + } + } + + for _, ts := range ms.TableSettings { + tableName := ts.TargetTable + table, err := sqlparser.TableFromStatement(ts.SourceExpression) + if err == nil { + tableName = table.Name.String() + } + env.tmc.schema[ms.SourceKeyspace+"."+tableName] = &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{ + Name: tableName, + Schema: fmt.Sprintf("%s_schema", tableName), + }}, + } + env.tmc.schema[ms.TargetKeyspace+"."+ts.TargetTable] = &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{ + Name: ts.TargetTable, + Schema: fmt.Sprintf("%s_schema", ts.TargetTable), + }}, + } + } + if ms.Workflow != "" { + env.expectValidation() + } + return env +} + +func (env *testMaterializerEnv) expectValidation() { + for _, tablet := range env.tablets { + tabletID := int(tablet.Alias.Uid) + if tabletID < 200 { + continue + } + // wr.validateNewWorkflow + env.tmc.expectVRQuery(tabletID, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'", env.ms.TargetKeyspace, env.ms.Workflow), &sqltypes.Result{}) + } +} + +func (env *testMaterializerEnv) close() { + for _, t := range env.tablets { + env.deleteTablet(t) + } +} + +func (env *testMaterializerEnv) addTablet(id int, keyspace, shard string, tabletType topodatapb.TabletType) *topodatapb.Tablet { + tablet := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: env.cell, + Uid: uint32(id), + }, + Keyspace: keyspace, + Shard: shard, + KeyRange: &topodatapb.KeyRange{}, + Type: tabletType, + PortMap: map[string]int32{ + "test": int32(id), + }, + } + env.tablets[id] = tablet + if err := env.ws.ts.InitTablet(context.Background(), tablet, false /* allowPrimaryOverride */, true /* createShardAndKeyspace */, false /* allowUpdate */); err != nil { + panic(err) + } + if tabletType == topodatapb.TabletType_PRIMARY { + _, err := env.ws.ts.UpdateShardFields(context.Background(), keyspace, shard, func(si *topo.ShardInfo) error { + si.PrimaryAlias = tablet.Alias + return nil + }) + if err != nil { + panic(err) + } + } + return tablet +} + +func (env *testMaterializerEnv) deleteTablet(tablet *topodatapb.Tablet) { + _ = env.topoServ.DeleteTablet(context.Background(), tablet.Alias) + delete(env.tablets, int(tablet.Alias.Uid)) +} + +//---------------------------------------------- +// testMaterializerTMClient + +type testMaterializerTMClient struct { + tmclient.TabletManagerClient + schema map[string]*tabletmanagerdatapb.SchemaDefinition + + mu sync.Mutex + vrQueries map[int][]*queryResult + getSchemaCounts map[string]int + muSchemaCount sync.Mutex +} + +func newTestMaterializerTMClient() *testMaterializerTMClient { + return &testMaterializerTMClient{ + schema: make(map[string]*tabletmanagerdatapb.SchemaDefinition), + vrQueries: make(map[int][]*queryResult), + getSchemaCounts: make(map[string]int), + } +} + +func (tmc *testMaterializerTMClient) schemaRequested(uid uint32) { + tmc.muSchemaCount.Lock() + defer tmc.muSchemaCount.Unlock() + key := strconv.Itoa(int(uid)) + n, ok := tmc.getSchemaCounts[key] + if !ok { + tmc.getSchemaCounts[key] = 1 + } else { + tmc.getSchemaCounts[key] = n + 1 + } +} + +func (tmc *testMaterializerTMClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + res := sqltypes.MakeTestResult(sqltypes.MakeTestFields("rowsaffected", "int64"), "1") + return &tabletmanagerdatapb.CreateVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil +} + +func (tmc *testMaterializerTMClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + return &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ + Workflow: "workflow", + Streams: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + { + Id: 1, + Bls: &binlogdatapb.BinlogSource{ + Keyspace: "sourceks", + Shard: "0", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: ".*", + }, + }, + }, + }, + }, + }, + }, nil +} + +func (tmc *testMaterializerTMClient) GetSchema(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) { + tmc.schemaRequested(tablet.Alias.Uid) + schemaDefn := &tabletmanagerdatapb.SchemaDefinition{} + for _, table := range request.Tables { + if table == "/.*/" { + // Special case of all tables in keyspace. + for key, tableDefn := range tmc.schema { + if strings.HasPrefix(key, tablet.Keyspace+".") { + schemaDefn.TableDefinitions = append(schemaDefn.TableDefinitions, tableDefn.TableDefinitions...) + } + } + break + } + + key := tablet.Keyspace + "." + table + tableDefn := tmc.schema[key] + if tableDefn == nil { + continue + } + schemaDefn.TableDefinitions = append(schemaDefn.TableDefinitions, tableDefn.TableDefinitions...) + } + return schemaDefn, nil +} + +func (tmc *testMaterializerTMClient) expectVRQuery(tabletID int, query string, result *sqltypes.Result) { + tmc.mu.Lock() + defer tmc.mu.Unlock() + + tmc.vrQueries[tabletID] = append(tmc.vrQueries[tabletID], &queryResult{ + query: query, + result: sqltypes.ResultToProto3(result), + }) +} + +func (tmc *testMaterializerTMClient) VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) { + tmc.mu.Lock() + defer tmc.mu.Unlock() + + qrs := tmc.vrQueries[int(tablet.Alias.Uid)] + if len(qrs) == 0 { + return nil, fmt.Errorf("tablet %v does not expect any more queries: %s", tablet, query) + } + matched := false + if qrs[0].query[0] == '/' { + matched = regexp.MustCompile(qrs[0].query[1:]).MatchString(query) + } else { + matched = query == qrs[0].query + } + if !matched { + return nil, fmt.Errorf("tablet %v:\nunexpected query\n%s\nwant:\n%s", tablet, query, qrs[0].query) + } + tmc.vrQueries[int(tablet.Alias.Uid)] = qrs[1:] + return qrs[0].result, nil +} + +func (tmc *testMaterializerTMClient) ExecuteFetchAsDba(ctx context.Context, tablet *topodatapb.Tablet, usePool bool, req *tabletmanagerdatapb.ExecuteFetchAsDbaRequest) (*querypb.QueryResult, error) { + // Reuse VReplicationExec + return tmc.VReplicationExec(ctx, tablet, string(req.Query)) +} + +// Note: ONLY breaks up change.SQL into individual statements and executes it. Does NOT fully implement ApplySchema. +func (tmc *testMaterializerTMClient) ApplySchema(ctx context.Context, tablet *topodatapb.Tablet, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) { + stmts := strings.Split(change.SQL, ";") + + for _, stmt := range stmts { + _, err := tmc.ExecuteFetchAsDba(ctx, tablet, false, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(stmt), + MaxRows: 0, + ReloadSchema: true, + }) + if err != nil { + return nil, err + } + } + + return nil, nil +} diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go new file mode 100644 index 00000000000..3631b32fe6e --- /dev/null +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -0,0 +1,520 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/utils" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/vtgate/vindexes" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" +) + +const getWorkflowQuery = "select id from _vt.vreplication where db_name='vt_targetks' and workflow='workflow'" +const mzUpdateQuery = "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='workflow'" +const mzSelectFrozenQuery = "select 1 from _vt.vreplication where db_name='vt_targetks' and message='FROZEN' and workflow_sub_type != 1" +const mzCheckJournal = "/select val from _vt.resharding_journal where id=" +const mzGetWorkflowStatusQuery = "select id, workflow, source, pos, stop_pos, max_replication_lag, state, db_name, time_updated, transaction_timestamp, message, tags, workflow_type, workflow_sub_type from _vt.vreplication where workflow = 'workflow' and db_name = 'vt_targetks'" +const mzGetCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" +const mzGetLatestCopyState = "select table_name, lastpk from _vt.copy_state where vrepl_id = 1 and id in (select max(id) from _vt.copy_state where vrepl_id = 1 group by vrepl_id, table_name)" + +var defaultOnDDL = binlogdatapb.OnDDLAction_IGNORE.String() +var binlogSource = &binlogdatapb.BinlogSource{ + Keyspace: "sourceks", + Shard: "0", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1", + }}, + }, +} +var getWorkflowRes = sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source|message|cell|tablet_types|workflow_type|workflow_sub_type|defer_secondary_keys", + "int64|blob|varchar|varchar|varchar|int64|int64|int64", + ), + fmt.Sprintf("1|%s||zone1|replica|1|0|1", binlogSource), +) +var getWorkflowStatusRes = sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|workflow|source|pos|stop_pos|max_replication_log|state|db_name|time_updated|transaction_timestamp|message|tags|workflow_type|workflow_sub_type", + "int64|varchar|blob|varchar|varchar|int64|varchar|varchar|int64|int64|varchar|varchar|int64|int64", + ), + fmt.Sprintf("1|wf1|%s|MySQL56/9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97|NULL|0|running|vt_ks|1686577659|0|||1|0", binlogSource), +) + +func TestStripForeignKeys(t *testing.T) { + tcs := []struct { + desc string + ddl string + + hasErr bool + newDDL string + }{ + { + desc: "has FK constraints", + ddl: "CREATE TABLE `table1` (\n" + + "`id` int(11) NOT NULL AUTO_INCREMENT,\n" + + "`foreign_id` int(11) CHECK (foreign_id>10),\n" + + "PRIMARY KEY (`id`),\n" + + "KEY `fk_table1_ref_foreign_id` (`foreign_id`),\n" + + "CONSTRAINT `fk_table1_ref_foreign_id` FOREIGN KEY (`foreign_id`) REFERENCES `foreign` (`id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=latin1;", + + newDDL: "create table table1 (\n" + + "\tid int(11) not null auto_increment,\n" + + "\tforeign_id int(11),\n" + + "\tPRIMARY KEY (id),\n" + + "\tKEY fk_table1_ref_foreign_id (foreign_id),\n" + + "\tcheck (foreign_id > 10)\n" + + ") ENGINE InnoDB,\n" + + " CHARSET latin1", + + hasErr: false, + }, + { + desc: "no FK constraints", + ddl: "CREATE TABLE `table1` (\n" + + "`id` int(11) NOT NULL AUTO_INCREMENT,\n" + + "`foreign_id` int(11) NOT NULL CHECK (foreign_id>10),\n" + + "`user_id` int(11) NOT NULL,\n" + + "PRIMARY KEY (`id`),\n" + + "KEY `fk_table1_ref_foreign_id` (`foreign_id`),\n" + + "KEY `fk_table1_ref_user_id` (`user_id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=latin1;", + + newDDL: "create table table1 (\n" + + "\tid int(11) not null auto_increment,\n" + + "\tforeign_id int(11) not null,\n" + + "\tuser_id int(11) not null,\n" + + "\tPRIMARY KEY (id),\n" + + "\tKEY fk_table1_ref_foreign_id (foreign_id),\n" + + "\tKEY fk_table1_ref_user_id (user_id),\n" + + "\tcheck (foreign_id > 10)\n" + + ") ENGINE InnoDB,\n" + + " CHARSET latin1", + }, + } + + for _, tc := range tcs { + newDDL, err := stripTableForeignKeys(tc.ddl) + if tc.hasErr != (err != nil) { + t.Fatalf("hasErr does not match: err: %v, tc: %+v", err, tc) + } + + if newDDL != tc.newDDL { + utils.MustMatch(t, tc.newDDL, newDDL, fmt.Sprintf("newDDL does not match. tc: %+v", tc)) + } + } +} + +func TestStripConstraints(t *testing.T) { + tcs := []struct { + desc string + ddl string + + hasErr bool + newDDL string + }{ + { + desc: "constraints", + ddl: "CREATE TABLE `table1` (\n" + + "`id` int(11) NOT NULL AUTO_INCREMENT,\n" + + "`foreign_id` int(11) NOT NULL,\n" + + "`user_id` int(11) NOT NULL,\n" + + "PRIMARY KEY (`id`),\n" + + "KEY `fk_table1_ref_foreign_id` (`foreign_id`),\n" + + "KEY `fk_table1_ref_user_id` (`user_id`),\n" + + "CONSTRAINT `fk_table1_ref_foreign_id` FOREIGN KEY (`foreign_id`) REFERENCES `foreign` (`id`),\n" + + "CONSTRAINT `fk_table1_ref_user_id` FOREIGN KEY (`user_id`) REFERENCES `core_user` (`id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=latin1;", + + newDDL: "create table table1 (\n" + + "\tid int(11) not null auto_increment,\n" + + "\tforeign_id int(11) not null,\n" + + "\tuser_id int(11) not null,\n" + + "\tPRIMARY KEY (id),\n" + + "\tKEY fk_table1_ref_foreign_id (foreign_id),\n" + + "\tKEY fk_table1_ref_user_id (user_id)\n" + + ") ENGINE InnoDB,\n" + + " CHARSET latin1", + + hasErr: false, + }, + { + desc: "no constraints", + ddl: "CREATE TABLE `table1` (\n" + + "`id` int(11) NOT NULL AUTO_INCREMENT,\n" + + "`foreign_id` int(11) NOT NULL,\n" + + "`user_id` int(11) NOT NULL,\n" + + "PRIMARY KEY (`id`),\n" + + "KEY `fk_table1_ref_foreign_id` (`foreign_id`),\n" + + "KEY `fk_table1_ref_user_id` (`user_id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=latin1;", + + newDDL: "create table table1 (\n" + + "\tid int(11) not null auto_increment,\n" + + "\tforeign_id int(11) not null,\n" + + "\tuser_id int(11) not null,\n" + + "\tPRIMARY KEY (id),\n" + + "\tKEY fk_table1_ref_foreign_id (foreign_id),\n" + + "\tKEY fk_table1_ref_user_id (user_id)\n" + + ") ENGINE InnoDB,\n" + + " CHARSET latin1", + }, + { + desc: "bad ddl has error", + ddl: "bad ddl", + + hasErr: true, + }, + } + + for _, tc := range tcs { + newDDL, err := stripTableConstraints(tc.ddl) + if tc.hasErr != (err != nil) { + t.Fatalf("hasErr does not match: err: %v, tc: %+v", err, tc) + } + + if newDDL != tc.newDDL { + utils.MustMatch(t, tc.newDDL, newDDL, fmt.Sprintf("newDDL does not match. tc: %+v", tc)) + } + } +} + +func TestAddTablesToVSchema(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("zone1") + srcks := "source" + ws := &Server{ + ts: ts, + } + tests := []struct { + name string + sourceVSchema *vschemapb.Keyspace + inTargetVSchema *vschemapb.Keyspace + tables []string + copyVSchema bool + wantTargetVSchema *vschemapb.Keyspace + }{ + { + name: "no target vschema; copy source vschema", + sourceVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + }, + }, + }, + inTargetVSchema: &vschemapb.Keyspace{}, + tables: []string{"t1", "t2", "t3", "t4"}, + copyVSchema: true, + wantTargetVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + }, + "t4": {}, + }, + }, + }, + { + name: "no target vschema; copy source vschema; sharded source", + sourceVSchema: &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "hash": { + Type: "hash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + Pinned: "123456", + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + ColumnVindexes: []*vschemapb.ColumnVindex{ // Should be stripped on target + { + Column: "c1", + Name: "hash", + }, + }, + }, + "t4": { + ColumnVindexes: []*vschemapb.ColumnVindex{ // Should be stripped on target + { + Column: "c1", + Name: "hash", + }, + }, + }, + }, + }, + inTargetVSchema: &vschemapb.Keyspace{}, + tables: []string{"t1", "t2", "t3", "t4"}, + copyVSchema: true, + wantTargetVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + Pinned: "123456", + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + }, + "t4": {}, + }, + }, + }, + { + name: "target vschema; copy source vschema", + sourceVSchema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "hash": { + Type: "hash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + }, + "t4": { + ColumnVindexes: []*vschemapb.ColumnVindex{ // Should be stripped on target + { + Column: "c1", + Name: "hash", + }, + }, + }, + }, + }, + inTargetVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": {}, + "t3": {}, + "t4": {}, + }, + }, + tables: []string{"t1", "t2", "t3", "t4"}, + copyVSchema: true, + wantTargetVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": {}, + "t3": {}, + "t4": {}, + }, + }, + }, + { + name: "no target vschema; do not copy source vschema", + sourceVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: vindexes.TypeReference, + }, + "t2": { + Type: vindexes.TypeSequence, + }, + "t3": { + AutoIncrement: &vschemapb.AutoIncrement{ + Column: "c1", + Sequence: "t2", + }, + }, + }, + }, + inTargetVSchema: &vschemapb.Keyspace{}, + tables: []string{"t1", "t2"}, + copyVSchema: false, + wantTargetVSchema: &vschemapb.Keyspace{ + Tables: map[string]*vschemapb.Table{ + "t1": {}, + "t2": {}, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := ts.SaveVSchema(ctx, srcks, tt.sourceVSchema) + require.NoError(t, err) + err = ws.addTablesToVSchema(ctx, srcks, tt.inTargetVSchema, tt.tables, tt.copyVSchema) + require.NoError(t, err) + require.Equal(t, tt.wantTargetVSchema, tt.inTargetVSchema) + }) + } +} + +func TestMigrateVSchema(t *testing.T) { + ms := &vtctldatapb.MaterializeSettings{ + Workflow: "workflow", + Cell: "cell", + SourceKeyspace: "sourceks", + TargetKeyspace: "targetks", + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + }}, + } + env := newTestMaterializerEnv(t, ms, []string{"0"}, []string{"0"}) + defer env.close() + + env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, getWorkflowQuery, getWorkflowRes) + env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzGetWorkflowStatusQuery, getWorkflowStatusRes) + env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) + + ctx := context.Background() + _, err := env.ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ + Workflow: ms.Workflow, + Cells: []string{ms.Cell}, + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY}, + SourceKeyspace: ms.SourceKeyspace, + TargetKeyspace: ms.TargetKeyspace, + IncludeTables: []string{"t1"}, + AutoStart: true, + OnDdl: defaultOnDDL, + }) + require.NoError(t, err) + vschema, err := env.ws.ts.GetSrvVSchema(ctx, env.cell) + require.NoError(t, err) + got := fmt.Sprintf("%v", vschema) + want := []string{`keyspaces:{key:"sourceks" value:{}}`, + `keyspaces:{key:"sourceks" value:{}} keyspaces:{key:"targetks" value:{tables:{key:"t1" value:{}}}}`, + `rules:{from_table:"t1" to_tables:"sourceks.t1"}`, + `rules:{from_table:"targetks.t1" to_tables:"sourceks.t1"}`, + } + for _, wantstr := range want { + require.Contains(t, got, wantstr) + } +} + +// TestMoveTablesDDLFlag tests that we save the on-ddl flag value in the workflow. +// Note: +// - TestPlayerDDL tests that the vplayer correctly implements the ddl behavior +// - We have a manual e2e test for the full behavior: TestVReplicationDDLHandling +func TestMoveTablesDDLFlag(t *testing.T) { + ctx := context.Background() + ms := &vtctldatapb.MaterializeSettings{ + Workflow: "workflow", + SourceKeyspace: "sourceks", + TargetKeyspace: "targetks", + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + }}, + } + + for onDDLAction := range binlogdatapb.OnDDLAction_value { + t.Run(fmt.Sprintf("OnDDL Flag:%v", onDDLAction), func(t *testing.T) { + env := newTestMaterializerEnv(t, ms, []string{"0"}, []string{"0"}) + 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{}) + + // TODO: we cannot test the actual query generated w/o having a + // TabletManager. Importing the tabletmanager package, however, causes + // a circular dependency. + // The TabletManager portion is tested in rpc_vreplication_test.go. + env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, getWorkflowQuery, getWorkflowRes) + env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzGetWorkflowStatusQuery, getWorkflowStatusRes) + env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) + + targetShard, err := env.topoServ.GetShardNames(ctx, ms.TargetKeyspace) + require.NoError(t, err) + sourceShard, err := env.topoServ.GetShardNames(ctx, ms.SourceKeyspace) + require.NoError(t, err) + want := fmt.Sprintf("shard_streams:{key:\"%s/%s\" value:{streams:{id:1 tablet:{cell:\"%s\" uid:200} source_shard:\"%s/%s\" position:\"MySQL56/9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97\" status:\"running\" info:\"VStream Lag: 0s\"}}}", + ms.TargetKeyspace, targetShard[0], env.cell, ms.SourceKeyspace, sourceShard[0]) + + res, err := env.ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ + Workflow: ms.Workflow, + SourceKeyspace: ms.SourceKeyspace, + TargetKeyspace: ms.TargetKeyspace, + IncludeTables: []string{"t1"}, + OnDdl: onDDLAction, + }) + require.NoError(t, err) + require.Equal(t, want, fmt.Sprintf("%+v", res)) + }) + } +} diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index b29851a3f8a..b0200a3c95a 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -20,30 +20,68 @@ import ( "context" "errors" "fmt" + "reflect" "sort" "strings" "sync" "time" + "golang.org/x/sync/semaphore" "google.golang.org/protobuf/encoding/prototext" + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/sets" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/trace" "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/schema" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vtctl/workflow/vexec" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/evalengine" + "vitess.io/vitess/go/vt/vtgate/vindexes" "vitess.io/vitess/go/vt/vttablet/tmclient" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" - "vitess.io/vitess/go/vt/proto/vttime" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + vttimepb "vitess.io/vitess/go/vt/proto/vttime" +) + +// TableCopyProgress stores the row counts and disk sizes of the source and target tables +type TableCopyProgress struct { + TargetRowCount, TargetTableSize int64 + SourceRowCount, SourceTableSize int64 +} + +// CopyProgress stores the TableCopyProgress for all tables still being copied +type CopyProgress map[string]*TableCopyProgress + +const ( + cannotSwitchError = "workflow has errors" + cannotSwitchCopyIncomplete = "copy is still in progress" + cannotSwitchHighLag = "replication lag %ds is higher than allowed lag %ds" + cannotSwitchFailedTabletRefresh = "could not refresh all of the tablets involved in the operation:\n%s" + cannotSwitchFrozen = "workflow is frozen" + + // Number of LOCK TABLES cycles to perform on the sources during SwitchWrites. + lockTablesCycles = 2 + // Time to wait between LOCK TABLES cycles on the sources during SwitchWrites. + lockTablesCycleDelay = time.Duration(100 * time.Millisecond) + + // Default duration used for lag, timeout, etc. + defaultDuration = 30 * time.Second ) var ( @@ -57,19 +95,18 @@ var ( // ErrMultipleTargetKeyspaces occurs when a workflow somehow has multiple // target keyspaces across different shard primaries. This should be // impossible. - ErrMultipleTargetKeyspaces = errors.New("multiple target keyspaces for a single workflow") + ErrMultipleTargetKeyspaces = errors.New("multiple target keyspaces for a single workflow") + ErrWorkflowNotFullySwitched = errors.New("cannot complete workflow because you have not yet switched all read and write traffic") + ErrWorkflowPartiallySwitched = errors.New("cannot cancel workflow because you have already switched some or all read and write traffic") ) // Server provides an API to work with Vitess workflows, like vreplication // workflows (MoveTables, Reshard, etc) and schema migration workflows. -// -// NB: This is in alpha, and you probably don't want to depend on it (yet!). -// Currently, it provides only a read-only API to vreplication workflows. Write -// actions on vreplication workflows, and schema migration workflows entirely, -// are not yet supported, but planned. type Server struct { ts *topo.Server tmc tmclient.TabletManagerClient + // Limt the number of concurrent background goroutines if needed. + sem *semaphore.Weighted } // NewServer returns a new server instance with the given topo.Server and @@ -264,6 +301,20 @@ func (s *Server) GetCellsWithTableReadsSwitched( return cellsSwitched, cellsNotSwitched, nil } +func (s *Server) GetWorkflow(ctx context.Context, keyspace, workflow string) (*vtctldatapb.Workflow, error) { + res, err := s.GetWorkflows(ctx, &vtctldatapb.GetWorkflowsRequest{ + Keyspace: keyspace, + Workflow: workflow, + }) + if err != nil { + return nil, err + } + if len(res.Workflows) != 1 { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected number of workflows returned; expected 1, got %d", len(res.Workflows)) + } + return res.Workflows[0], nil +} + // GetWorkflows returns a list of all workflows that exist in a given keyspace, // with some additional filtering depending on the request parameters (for // example, ActiveOnly=true restricts the search to only workflows that are @@ -279,8 +330,15 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows span.Annotate("active_only", req.ActiveOnly) where := "" + predicates := []string{} if req.ActiveOnly { - where = "WHERE state <> 'Stopped'" + predicates = append(predicates, "state <> 'Stopped'") + } + if req.Workflow != "" { + predicates = append(predicates, fmt.Sprintf("workflow = '%s'", req.Workflow)) + } + if len(predicates) > 0 { + where = fmt.Sprintf("WHERE %s", strings.Join(predicates, " AND ")) } query := fmt.Sprintf(` @@ -383,10 +441,10 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows StopPosition: stopPos, State: state, DbName: dbName, - TransactionTimestamp: &vttime.Time{ + TransactionTimestamp: &vttimepb.Time{ Seconds: transactionTimeSeconds, }, - TimeUpdated: &vttime.Time{ + TimeUpdated: &vttimepb.Time{ Seconds: timeUpdatedSeconds, }, Message: message, @@ -403,11 +461,11 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows switch { case strings.Contains(strings.ToLower(stream.Message), "error"): - stream.State = "Error" - case stream.State == "Running" && len(stream.CopyStates) > 0: - stream.State = "Copying" - case stream.State == "Running" && int64(time.Now().Second())-timeUpdatedSeconds > 10: - stream.State = "Lagging" + stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: + stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-timeUpdatedSeconds > 10: + stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() } // At this point, we're going to start modifying the maps defined @@ -618,10 +676,10 @@ ORDER BY StreamId: streamID, Type: typ, State: state, - CreatedAt: &vttime.Time{ + CreatedAt: &vttimepb.Time{ Seconds: createdAt.Unix(), }, - UpdatedAt: &vttime.Time{ + UpdatedAt: &vttimepb.Time{ Seconds: updatedAt.Unix(), }, Message: message, @@ -720,6 +778,120 @@ ORDER BY }, nil } +func (s *Server) getWorkflowState(ctx context.Context, targetKeyspace, workflowName string) (*trafficSwitcher, *State, error) { + ts, err := s.buildTrafficSwitcher(ctx, targetKeyspace, workflowName) + + if err != nil { + log.Errorf("buildTrafficSwitcher failed: %v", err) + return nil, nil, err + } + + state := &State{ + Workflow: workflowName, + SourceKeyspace: ts.SourceKeyspaceName(), + TargetKeyspace: targetKeyspace, + IsPartialMigration: ts.isPartialMigration, + } + + var ( + reverse bool + sourceKeyspace string + ) + + // We reverse writes by using the source_keyspace.workflowname_reverse workflow + // spec, so we need to use the source of the reverse workflow, which is the + // target of the workflow initiated by the user for checking routing rules. + // Similarly we use a target shard of the reverse workflow as the original + // source to check if writes have been switched. + if strings.HasSuffix(workflowName, "_reverse") { + reverse = true + // Flip the source and target keyspaces. + sourceKeyspace = state.TargetKeyspace + targetKeyspace = state.SourceKeyspace + workflowName = ReverseWorkflowName(workflowName) + } else { + sourceKeyspace = state.SourceKeyspace + } + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + state.WorkflowType = TypeMoveTables + + // We assume a consistent state, so only choose routing rule for one table. + if len(ts.Tables()) == 0 { + return nil, nil, fmt.Errorf("no tables in workflow %s.%s", targetKeyspace, workflowName) + + } + table := ts.Tables()[0] + + if ts.isPartialMigration { // shard level traffic switching is all or nothing + shardRoutingRules, err := s.ts.GetShardRoutingRules(ctx) + if err != nil { + return nil, nil, err + } + + rules := shardRoutingRules.Rules + for _, rule := range rules { + switch rule.ToKeyspace { + case sourceKeyspace: + state.ShardsNotYetSwitched = append(state.ShardsNotYetSwitched, rule.Shard) + case targetKeyspace: + state.ShardsAlreadySwitched = append(state.ShardsAlreadySwitched, rule.Shard) + default: + // Not a relevant rule. + } + } + } else { + state.RdonlyCellsSwitched, state.RdonlyCellsNotSwitched, err = s.GetCellsWithTableReadsSwitched(ctx, targetKeyspace, table, topodatapb.TabletType_RDONLY) + if err != nil { + return nil, nil, err + } + + state.ReplicaCellsSwitched, state.ReplicaCellsNotSwitched, err = s.GetCellsWithTableReadsSwitched(ctx, targetKeyspace, table, topodatapb.TabletType_REPLICA) + if err != nil { + return nil, nil, err + } + globalRules, err := topotools.GetRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return nil, nil, err + } + for _, table := range ts.Tables() { + rr := globalRules[table] + // If a rule exists for the table and points to the target keyspace, then + // writes have been switched. + if len(rr) > 0 && rr[0] == fmt.Sprintf("%s.%s", targetKeyspace, table) { + state.WritesSwitched = true + break + } + } + } + } else { + state.WorkflowType = TypeReshard + + // We assume a consistent state, so only choose one shard. + var shard *topo.ShardInfo + if reverse { + shard = ts.TargetShards()[0] + } else { + shard = ts.SourceShards()[0] + } + + state.RdonlyCellsSwitched, state.RdonlyCellsNotSwitched, err = s.GetCellsWithShardReadsSwitched(ctx, targetKeyspace, shard, topodatapb.TabletType_RDONLY) + if err != nil { + return nil, nil, err + } + + state.ReplicaCellsSwitched, state.ReplicaCellsNotSwitched, err = s.GetCellsWithShardReadsSwitched(ctx, targetKeyspace, shard, topodatapb.TabletType_REPLICA) + if err != nil { + return nil, nil, err + } + + if !shard.IsPrimaryServing { + state.WritesSwitched = true + } + } + + return ts, state, nil +} + func (s *Server) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, id int64) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { span, ctx := trace.NewSpan(ctx, "workflow.Server.getWorkflowCopyStates") defer span.Finish() @@ -752,49 +924,1708 @@ func (s *Server) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletI return copyStates, nil } -// WorkflowUpdate is part of the vtctlservicepb.VtctldServer interface. +// MoveTablesCreate is part of the vtctlservicepb.VtctldServer interface. // It passes the embedded TabletRequest object to the given keyspace's -// target primary tablets that are participating in the given workflow. -func (s *Server) WorkflowUpdate(ctx context.Context, req *vtctldatapb.WorkflowUpdateRequest) (*vtctldatapb.WorkflowUpdateResponse, error) { - span, ctx := trace.NewSpan(ctx, "workflow.Server.WorkflowUpdate") +// target primary tablets that will be executing the workflow. +func (s *Server) MoveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTablesCreateRequest) (*vtctldatapb.WorkflowStatusResponse, error) { + span, ctx := trace.NewSpan(ctx, "workflow.Server.MoveTablesCreate") + defer span.Finish() + + span.Annotate("keyspace", req.TargetKeyspace) + span.Annotate("workflow", req.Workflow) + span.Annotate("cells", req.Cells) + span.Annotate("tablet_types", req.TabletTypes) + span.Annotate("on_ddl", req.OnDdl) + + sourceKeyspace := req.SourceKeyspace + targetKeyspace := req.TargetKeyspace + //FIXME validate tableSpecs, allTables, excludeTables + var ( + tables = req.IncludeTables + externalTopo *topo.Server + sourceTopo *topo.Server = s.ts + err error + ) + + // When the source is an external cluster mounted using the Mount command. + if req.ExternalClusterName != "" { + externalTopo, err = s.ts.OpenExternalVitessClusterServer(ctx, req.ExternalClusterName) + if err != nil { + return nil, err + } + sourceTopo = externalTopo + log.Infof("Successfully opened external topo: %+v", externalTopo) + } + + var vschema *vschemapb.Keyspace + vschema, err = s.ts.GetVSchema(ctx, targetKeyspace) + if err != nil { + return nil, err + } + if vschema == nil { + return nil, fmt.Errorf("no vschema found for target keyspace %s", targetKeyspace) + } + ksTables, err := getTablesInKeyspace(ctx, sourceTopo, s.tmc, sourceKeyspace) + if err != nil { + return nil, err + } + if len(tables) > 0 { + err = s.validateSourceTablesExist(ctx, sourceKeyspace, ksTables, tables) + if err != nil { + return nil, err + } + } else { + if req.AllTables { + tables = ksTables + } else { + return nil, fmt.Errorf("no tables to move") + } + } + if len(req.ExcludeTables) > 0 { + err = s.validateSourceTablesExist(ctx, sourceKeyspace, ksTables, req.ExcludeTables) + if err != nil { + return nil, err + } + } + var tables2 []string + for _, t := range tables { + if shouldInclude(t, req.ExcludeTables) { + tables2 = append(tables2, t) + } + } + tables = tables2 + if len(tables) == 0 { + return nil, fmt.Errorf("no tables to move") + } + log.Infof("Found tables to move: %s", strings.Join(tables, ",")) + + if !vschema.Sharded { + if err := s.addTablesToVSchema(ctx, sourceKeyspace, vschema, tables, externalTopo == nil); err != nil { + return nil, err + } + } + if externalTopo == nil { + // Save routing rules before vschema. If we save vschema first, and routing rules + // fails to save, we may generate duplicate table errors. + 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 vschema != nil { + // We added to the vschema. + if err := s.ts.SaveVSchema(ctx, targetKeyspace, vschema); err != nil { + return nil, err + } + } + } + if err := s.ts.RebuildSrvVSchema(ctx, nil); err != nil { + return nil, err + } + ms := &vtctldatapb.MaterializeSettings{ + Workflow: req.Workflow, + MaterializationIntent: vtctldatapb.MaterializationIntent_MOVETABLES, + SourceKeyspace: sourceKeyspace, + TargetKeyspace: targetKeyspace, + Cell: strings.Join(req.Cells, ","), + TabletTypes: topoproto.MakeStringTypeCSV(req.TabletTypes), + TabletSelectionPreference: req.TabletSelectionPreference, + StopAfterCopy: req.StopAfterCopy, + ExternalCluster: req.ExternalClusterName, + SourceShards: req.SourceShards, + OnDdl: req.OnDdl, + DeferSecondaryKeys: req.DeferSecondaryKeys, + } + if req.SourceTimeZone != "" { + ms.SourceTimeZone = req.SourceTimeZone + ms.TargetTimeZone = "UTC" + } + createDDLMode := createDDLAsCopy + if req.DropForeignKeys { + createDDLMode = createDDLAsCopyDropForeignKeys + } + + for _, table := range tables { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select * from %v", sqlparser.NewIdentifierCS(table)) + ms.TableSettings = append(ms.TableSettings, &vtctldatapb.TableMaterializeSettings{ + TargetTable: table, + SourceExpression: buf.String(), + CreateDdl: createDDLMode, + }) + } + mz := &materializer{ + ctx: ctx, + ts: s.ts, + sourceTs: sourceTopo, + tmc: s.tmc, + ms: ms, + } + err = mz.prepareMaterializerStreams(req) + if err != nil { + return nil, err + } + + if ms.SourceTimeZone != "" { + if err := mz.checkTZConversion(ctx, ms.SourceTimeZone); err != nil { + return nil, err + } + } + + tabletShards, err := s.collectTargetStreams(ctx, mz) + if err != nil { + return nil, err + } + + migrationID, err := getMigrationID(targetKeyspace, tabletShards) + if err != nil { + return nil, err + } + + if mz.ms.ExternalCluster == "" { + exists, tablets, err := s.checkIfPreviousJournalExists(ctx, mz, migrationID) + if err != nil { + return nil, err + } + if exists { + log.Errorf("Found a previous journal entry for %d", migrationID) + msg := fmt.Sprintf("found an entry from a previous run for migration id %d in _vt.resharding_journal on tablets %s, ", + migrationID, strings.Join(tablets, ",")) + msg += fmt.Sprintf("please review and delete it before proceeding and then start the workflow using: MoveTables --workflow %s --target-keyspace %s start", + req.Workflow, req.TargetKeyspace) + return nil, fmt.Errorf(msg) + } + } + + if req.AutoStart { + if err := mz.startStreams(ctx); err != nil { + return nil, err + } + } + + return s.WorkflowStatus(ctx, &vtctldatapb.WorkflowStatusRequest{ + Keyspace: targetKeyspace, + Workflow: req.Workflow, + }) +} + +// MoveTablesComplete is part of the vtctlservicepb.VtctldServer interface. +// It cleans up a successful MoveTables workflow and its related artifacts. +func (s *Server) MoveTablesComplete(ctx context.Context, req *vtctldatapb.MoveTablesCompleteRequest) (*vtctldatapb.MoveTablesCompleteResponse, error) { + span, ctx := trace.NewSpan(ctx, "workflow.Server.MoveTablesComplete") + defer span.Finish() + + ts, state, err := s.getWorkflowState(ctx, req.TargetKeyspace, req.Workflow) + if err != nil { + return nil, err + } + + var summary string + if req.DryRun { + summary = fmt.Sprintf("Complete dry run results for workflow %s.%s at %v", req.TargetKeyspace, req.Workflow, time.Now().UTC().Format(time.RFC822)) + } else { + summary = fmt.Sprintf("Successfully completed the %s workflow in the %s keyspace", req.Workflow, req.TargetKeyspace) + } + var dryRunResults *[]string + + if state.WorkflowType == TypeMigrate { + dryRunResults, err = s.finalizeMigrateWorkflow(ctx, req.TargetKeyspace, req.Workflow, strings.Join(ts.tables, ","), + false, req.KeepData, req.KeepRoutingRules, req.DryRun) + if err != nil { + return nil, vterrors.Wrapf(err, "failed to finalize the %s workflow in the %s keyspace", + req.Workflow, req.TargetKeyspace) + } + resp := &vtctldatapb.MoveTablesCompleteResponse{ + Summary: summary, + } + if dryRunResults != nil { + resp.DryRunResults = *dryRunResults + } + return resp, nil + } + + if !state.WritesSwitched || len(state.ReplicaCellsNotSwitched) > 0 || len(state.RdonlyCellsNotSwitched) > 0 { + return nil, ErrWorkflowNotFullySwitched + } + var renameTable TableRemovalType + if req.RenameTables { + renameTable = RenameTable + } else { + renameTable = DropTable + } + if dryRunResults, err = s.dropSources(ctx, ts, renameTable, req.KeepData, req.KeepRoutingRules, false, req.DryRun); err != nil { + return nil, err + } + + resp := &vtctldatapb.MoveTablesCompleteResponse{ + Summary: summary, + } + if dryRunResults != nil { + resp.DryRunResults = *dryRunResults + } + + return resp, nil +} + +// WorkflowDelete is part of the vtctlservicepb.VtctldServer interface. +// It passes on the request to the target primary tablets that are +// participating in the given workflow. +func (s *Server) WorkflowDelete(ctx context.Context, req *vtctldatapb.WorkflowDeleteRequest) (*vtctldatapb.WorkflowDeleteResponse, error) { + span, ctx := trace.NewSpan(ctx, "workflow.Server.WorkflowDelete") defer span.Finish() span.Annotate("keyspace", req.Keyspace) - span.Annotate("workflow", req.TabletRequest.Workflow) - span.Annotate("cells", req.TabletRequest.Cells) - span.Annotate("tablet_types", req.TabletRequest.TabletTypes) - span.Annotate("on_ddl", req.TabletRequest.OnDdl) + span.Annotate("workflow", req.Workflow) - vx := vexec.NewVExec(req.Keyspace, req.TabletRequest.Workflow, s.ts, s.tmc) + // Cleanup related data and artifacts. + if _, err := s.DropTargets(ctx, req.Keyspace, req.Workflow, req.KeepData, req.KeepRoutingRules, false); err != nil { + if topo.IsErrType(err, topo.NoNode) { + return nil, vterrors.Wrapf(err, "%s keyspace does not exist", req.Keyspace) + } + return nil, err + } + + deleteReq := &tabletmanagerdatapb.DeleteVReplicationWorkflowRequest{ + Workflow: req.Workflow, + } + vx := vexec.NewVExec(req.Keyspace, req.Workflow, s.ts, s.tmc) callback := func(ctx context.Context, tablet *topo.TabletInfo) (*querypb.QueryResult, error) { - res, err := s.tmc.UpdateVRWorkflow(ctx, tablet.Tablet, req.TabletRequest) + res, err := s.tmc.DeleteVReplicationWorkflow(ctx, tablet.Tablet, deleteReq) if err != nil { return nil, err } + // Best effort cleanup and optimization of related data. + s.deleteWorkflowVDiffData(ctx, tablet.Tablet, req.Workflow) + s.optimizeCopyStateTable(tablet.Tablet) return res.Result, err } res, err := vx.CallbackContext(ctx, callback) if err != nil { - if topo.IsErrType(err, topo.NoNode) { - return nil, vterrors.Wrapf(err, "%s keyspace does not exist", req.Keyspace) - } return nil, err } if len(res) == 0 { - return nil, fmt.Errorf("the %s workflow does not exist in the %s keyspace", req.TabletRequest.Workflow, req.Keyspace) + return nil, fmt.Errorf("the %s workflow does not exist in the %s keyspace", req.Workflow, req.Keyspace) } - response := &vtctldatapb.WorkflowUpdateResponse{} - response.Summary = fmt.Sprintf("Successfully updated the %s workflow on (%d) target primary tablets in the %s keyspace", req.TabletRequest.Workflow, len(res), req.Keyspace) - details := make([]*vtctldatapb.WorkflowUpdateResponse_TabletInfo, 0, len(res)) + response := &vtctldatapb.WorkflowDeleteResponse{} + response.Summary = fmt.Sprintf("Successfully cancelled the %s workflow in the %s keyspace", req.Workflow, req.Keyspace) + details := make([]*vtctldatapb.WorkflowDeleteResponse_TabletInfo, 0, len(res)) for tinfo, tres := range res { - result := &vtctldatapb.WorkflowUpdateResponse_TabletInfo{ - Tablet: fmt.Sprintf("%s-%d (%s/%s)", tinfo.Alias.Cell, tinfo.Alias.Uid, tinfo.Keyspace, tinfo.Shard), - Changed: tres.RowsAffected > 0, // Can be more than one with shard merges + result := &vtctldatapb.WorkflowDeleteResponse_TabletInfo{ + Tablet: tinfo.Alias, + Deleted: tres.RowsAffected > 0, // Can be more than one with shard merges } details = append(details, result) } response.Details = details return response, nil } + +func (s *Server) WorkflowStatus(ctx context.Context, req *vtctldatapb.WorkflowStatusRequest) (*vtctldatapb.WorkflowStatusResponse, error) { + ts, state, err := s.getWorkflowState(ctx, req.Keyspace, req.Workflow) + if err != nil { + return nil, err + } + copyProgress, err := s.GetCopyProgress(ctx, ts, state) + if err != nil { + return nil, err + } + resp := &vtctldatapb.WorkflowStatusResponse{} + if copyProgress != nil { + resp.TableCopyState = make(map[string]*vtctldatapb.WorkflowStatusResponse_TableCopyState, len(*copyProgress)) + // We sort the tables for intuitive and consistent output. + var tables []string + for table := range *copyProgress { + tables = append(tables, table) + } + sort.Strings(tables) + var progress TableCopyProgress + for _, table := range tables { + var rowCountPct, tableSizePct float32 + resp.TableCopyState[table] = &vtctldatapb.WorkflowStatusResponse_TableCopyState{} + progress = *(*copyProgress)[table] + if progress.SourceRowCount > 0 { + rowCountPct = float32(100.0 * float64(progress.TargetRowCount) / float64(progress.SourceRowCount)) + } + if progress.SourceTableSize > 0 { + tableSizePct = float32(100.0 * float64(progress.TargetTableSize) / float64(progress.SourceTableSize)) + } + resp.TableCopyState[table].RowsCopied = progress.TargetRowCount + resp.TableCopyState[table].RowsTotal = progress.SourceRowCount + resp.TableCopyState[table].RowsPercentage = rowCountPct + resp.TableCopyState[table].BytesCopied = progress.TargetTableSize + resp.TableCopyState[table].BytesTotal = progress.SourceTableSize + resp.TableCopyState[table].BytesPercentage = tableSizePct + } + } + + workflow, err := s.GetWorkflow(ctx, req.Keyspace, req.Workflow) + if err != nil { + return nil, err + } + + // The stream key is target keyspace/tablet alias, e.g. 0/test-0000000100. + // We sort the keys for intuitive and consistent output. + streamKeys := make([]string, 0, len(workflow.ShardStreams)) + for streamKey := range workflow.ShardStreams { + streamKeys = append(streamKeys, streamKey) + } + sort.Strings(streamKeys) + resp.ShardStreams = make(map[string]*vtctldatapb.WorkflowStatusResponse_ShardStreams, len(streamKeys)) + for _, streamKey := range streamKeys { + streams := workflow.ShardStreams[streamKey].GetStreams() + keyParts := strings.Split(streamKey, "/") + if len(keyParts) != 2 { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected stream key format in: %s ; expect /", + streamKey) + } + // We want to use target keyspace/shard as the map key for the + // response, e.g. customer/-80. + ksShard := fmt.Sprintf("%s/%s", req.Keyspace, keyParts[0]) + resp.ShardStreams[ksShard] = &vtctldatapb.WorkflowStatusResponse_ShardStreams{} + resp.ShardStreams[ksShard].Streams = make([]*vtctldatapb.WorkflowStatusResponse_ShardStreamState, len(streams)) + for i, st := range streams { + info := []string{} + ts := &vtctldatapb.WorkflowStatusResponse_ShardStreamState{} + if st.State == binlogdatapb.VReplicationWorkflowState_Error.String() { + info = append(info, st.Message) + } else if st.Position == "" { + info = append(info, "VStream has not started") + } else { + now := time.Now().Nanosecond() + updateLag := int64(now) - st.TimeUpdated.Seconds + if updateLag > 0*1e9 { + info = append(info, "VStream may not be running") + } + txLag := int64(now) - st.TransactionTimestamp.Seconds + info = append(info, fmt.Sprintf("VStream Lag: %ds", txLag/1e9)) + if st.TransactionTimestamp.Seconds > 0 { // if no events occur after copy phase, TransactionTimeStamp can be 0 + info = append(info, fmt.Sprintf("; Tx time: %s.", time.Unix(st.TransactionTimestamp.Seconds, 0).Format(time.ANSIC))) + } + } + ts.Id = int32(st.Id) + ts.Tablet = st.Tablet + ts.SourceShard = fmt.Sprintf("%s/%s", st.BinlogSource.Keyspace, st.BinlogSource.Shard) + ts.Position = st.Position + ts.Status = st.State + ts.Info = strings.Join(info, "; ") + resp.ShardStreams[ksShard].Streams[i] = ts + } + } + + return resp, nil +} + +// GetCopyProgress returns the progress of all tables being copied in the +// workflow. +func (s *Server) GetCopyProgress(ctx context.Context, ts *trafficSwitcher, state *State) (*CopyProgress, error) { + getTablesQuery := "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = %d" + getRowCountQuery := "select table_name, table_rows, data_length from information_schema.tables where table_schema = %s and table_name in (%s)" + tables := make(map[string]bool) + const MaxRows = 1000 + sourcePrimaries := make(map[*topodatapb.TabletAlias]bool) + for _, target := range ts.targets { + for id, bls := range target.Sources { + query := fmt.Sprintf(getTablesQuery, id) + p3qr, err := s.tmc.ExecuteFetchAsDba(ctx, target.GetPrimary().Tablet, true, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(query), + MaxRows: MaxRows, + }) + if err != nil { + return nil, err + } + if len(p3qr.Rows) < 1 { + continue + } + qr := sqltypes.Proto3ToResult(p3qr) + for i := 0; i < len(p3qr.Rows); i++ { + tables[qr.Rows[i][0].ToString()] = true + } + sourcesi, err := s.ts.GetShard(ctx, bls.Keyspace, bls.Shard) + if err != nil { + return nil, err + } + found := false + for existingSource := range sourcePrimaries { + if existingSource.Uid == sourcesi.PrimaryAlias.Uid { + found = true + } + } + if !found { + sourcePrimaries[sourcesi.PrimaryAlias] = true + } + } + } + if len(tables) == 0 { + return nil, nil + } + var tableList []string + targetRowCounts := make(map[string]int64) + sourceRowCounts := make(map[string]int64) + targetTableSizes := make(map[string]int64) + sourceTableSizes := make(map[string]int64) + + for table := range tables { + tableList = append(tableList, encodeString(table)) + targetRowCounts[table] = 0 + sourceRowCounts[table] = 0 + targetTableSizes[table] = 0 + sourceTableSizes[table] = 0 + } + + var getTableMetrics = func(tablet *topodatapb.Tablet, query string, rowCounts *map[string]int64, tableSizes *map[string]int64) error { + p3qr, err := s.tmc.ExecuteFetchAsDba(ctx, tablet, true, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(query), + MaxRows: uint64(len(tables)), + }) + if err != nil { + return err + } + qr := sqltypes.Proto3ToResult(p3qr) + for i := 0; i < len(qr.Rows); i++ { + table := qr.Rows[i][0].ToString() + rowCount, err := evalengine.ToInt64(qr.Rows[i][1]) + if err != nil { + return err + } + tableSize, err := evalengine.ToInt64(qr.Rows[i][2]) + if err != nil { + return err + } + (*rowCounts)[table] += rowCount + (*tableSizes)[table] += tableSize + } + return nil + } + sourceDbName := "" + for _, tsSource := range ts.sources { + sourceDbName = tsSource.GetPrimary().DbName() + break + } + if sourceDbName == "" { + return nil, fmt.Errorf("no sources found for workflow %s.%s", state.TargetKeyspace, state.Workflow) + } + targetDbName := "" + for _, tsTarget := range ts.targets { + targetDbName = tsTarget.GetPrimary().DbName() + break + } + if sourceDbName == "" || targetDbName == "" { + return nil, fmt.Errorf("workflow %s.%s is incorrectly configured", state.TargetKeyspace, state.Workflow) + } + sort.Strings(tableList) // sort list for repeatability for mocking in tests + tablesStr := strings.Join(tableList, ",") + query := fmt.Sprintf(getRowCountQuery, encodeString(targetDbName), tablesStr) + for _, target := range ts.targets { + tablet := target.GetPrimary().Tablet + if err := getTableMetrics(tablet, query, &targetRowCounts, &targetTableSizes); err != nil { + return nil, err + } + } + + query = fmt.Sprintf(getRowCountQuery, encodeString(sourceDbName), tablesStr) + for source := range sourcePrimaries { + ti, err := s.ts.GetTablet(ctx, source) + tablet := ti.Tablet + if err != nil { + return nil, err + } + if err := getTableMetrics(tablet, query, &sourceRowCounts, &sourceTableSizes); err != nil { + return nil, err + } + } + + copyProgress := CopyProgress{} + for table, rowCount := range targetRowCounts { + copyProgress[table] = &TableCopyProgress{ + TargetRowCount: rowCount, + TargetTableSize: targetTableSizes[table], + SourceRowCount: sourceRowCounts[table], + SourceTableSize: sourceTableSizes[table], + } + } + return ©Progress, nil +} + +// WorkflowUpdate is part of the vtctlservicepb.VtctldServer interface. +// It passes the embedded TabletRequest object to the given keyspace's +// target primary tablets that are participating in the given workflow. +func (s *Server) WorkflowUpdate(ctx context.Context, req *vtctldatapb.WorkflowUpdateRequest) (*vtctldatapb.WorkflowUpdateResponse, error) { + span, ctx := trace.NewSpan(ctx, "workflow.Server.WorkflowUpdate") + defer span.Finish() + + span.Annotate("keyspace", req.Keyspace) + span.Annotate("workflow", req.TabletRequest.Workflow) + span.Annotate("cells", req.TabletRequest.Cells) + span.Annotate("tablet_types", req.TabletRequest.TabletTypes) + span.Annotate("on_ddl", req.TabletRequest.OnDdl) + + vx := vexec.NewVExec(req.Keyspace, req.TabletRequest.Workflow, s.ts, s.tmc) + callback := func(ctx context.Context, tablet *topo.TabletInfo) (*querypb.QueryResult, error) { + res, err := s.tmc.UpdateVReplicationWorkflow(ctx, tablet.Tablet, req.TabletRequest) + if err != nil { + return nil, err + } + return res.Result, err + } + res, err := vx.CallbackContext(ctx, callback) + if err != nil { + if topo.IsErrType(err, topo.NoNode) { + return nil, vterrors.Wrapf(err, "%s keyspace does not exist", req.Keyspace) + } + return nil, err + } + + if len(res) == 0 { + return nil, fmt.Errorf("the %s workflow does not exist in the %s keyspace", req.TabletRequest.Workflow, req.Keyspace) + } + + response := &vtctldatapb.WorkflowUpdateResponse{} + response.Summary = fmt.Sprintf("Successfully updated the %s workflow on (%d) target primary tablets in the %s keyspace", req.TabletRequest.Workflow, len(res), req.Keyspace) + details := make([]*vtctldatapb.WorkflowUpdateResponse_TabletInfo, 0, len(res)) + for tinfo, tres := range res { + result := &vtctldatapb.WorkflowUpdateResponse_TabletInfo{ + Tablet: tinfo.Alias, + Changed: tres.RowsAffected > 0, // Can be more than one with shard merges + } + details = append(details, result) + } + response.Details = details + return response, nil +} + +// validateSourceTablesExist validates that tables provided are present +// in the source keyspace. +func (s *Server) validateSourceTablesExist(ctx context.Context, sourceKeyspace string, ksTables, tables []string) error { + var missingTables []string + for _, table := range tables { + if schema.IsInternalOperationTableName(table) { + continue + } + found := false + + for _, ksTable := range ksTables { + if table == ksTable { + found = true + break + } + } + if !found { + missingTables = append(missingTables, table) + } + } + if len(missingTables) > 0 { + return fmt.Errorf("table(s) not found in source keyspace %s: %s", sourceKeyspace, strings.Join(missingTables, ",")) + } + return nil +} + +// addTablesToVSchema adds tables to an (unsharded) vschema if they are not already defined. +// If copyVSchema is true then we copy over the vschema table definitions from the source, +// otherwise we create empty ones. +// For a migrate workflow we do not copy the vschema since the source keyspace is just a +// proxy to import data into Vitess. +func (s *Server) addTablesToVSchema(ctx context.Context, sourceKeyspace string, targetVSchema *vschemapb.Keyspace, tables []string, copyVSchema bool) error { + if targetVSchema.Tables == nil { + targetVSchema.Tables = make(map[string]*vschemapb.Table) + } + if copyVSchema { + srcVSchema, err := s.ts.GetVSchema(ctx, sourceKeyspace) + if err != nil { + return vterrors.Wrapf(err, "failed to get vschema for source keyspace %s", sourceKeyspace) + } + for _, table := range tables { + srcTable, sok := srcVSchema.Tables[table] + if _, tok := targetVSchema.Tables[table]; sok && !tok { + targetVSchema.Tables[table] = srcTable + // If going from sharded to unsharded, then we need to remove the + // column vindexes as they are not valid for unsharded tables. + if srcVSchema.Sharded { + targetVSchema.Tables[table].ColumnVindexes = nil + } + } + } + } + // Ensure that each table at least has an empty definition on the target. + for _, table := range tables { + if _, tok := targetVSchema.Tables[table]; !tok { + targetVSchema.Tables[table] = &vschemapb.Table{} + } + } + return nil +} + +func (s *Server) collectTargetStreams(ctx context.Context, mz *materializer) ([]string, error) { + var shardTablets []string + var mu sync.Mutex + err := mz.forAllTargets(func(target *topo.ShardInfo) error { + var qrproto *querypb.QueryResult + var id int64 + var err error + targetPrimary, err := s.ts.GetTablet(ctx, target.PrimaryAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) + } + query := fmt.Sprintf("select id from _vt.vreplication where db_name=%s and workflow=%s", encodeString(targetPrimary.DbName()), encodeString(mz.ms.Workflow)) + if qrproto, err = s.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + } + qr := sqltypes.Proto3ToResult(qrproto) + for i := 0; i < len(qr.Rows); i++ { + id, err = evalengine.ToInt64(qr.Rows[i][0]) + if err != nil { + return err + } + mu.Lock() + shardTablets = append(shardTablets, fmt.Sprintf("%s:%d", target.ShardName(), id)) + mu.Unlock() + } + return nil + }) + if err != nil { + return nil, err + } + return shardTablets, nil +} + +func (s *Server) checkIfPreviousJournalExists(ctx context.Context, mz *materializer, migrationID int64) (bool, []string, error) { + forAllSources := func(f func(*topo.ShardInfo) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, sourceShard := range mz.sourceShards { + wg.Add(1) + go func(sourceShard *topo.ShardInfo) { + defer wg.Done() + + if err := f(sourceShard); err != nil { + allErrors.RecordError(err) + } + }(sourceShard) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) + } + + var ( + mu sync.Mutex + exists bool + tablets []string + ) + + err := forAllSources(func(si *topo.ShardInfo) error { + tablet, err := s.ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + return err + } + if tablet == nil { + return nil + } + _, exists, err = s.CheckReshardingJournalExistsOnTablet(ctx, tablet.Tablet, migrationID) + if err != nil { + return err + } + if exists { + mu.Lock() + defer mu.Unlock() + tablets = append(tablets, tablet.AliasString()) + } + return nil + }) + return exists, tablets, err +} + +// deleteWorkflowVDiffData cleans up any potential VDiff related data associated +// with the workflow on the given tablet. +func (s *Server) deleteWorkflowVDiffData(ctx context.Context, tablet *topodatapb.Tablet, workflow string) { + sqlDeleteVDiffs := `delete from vd, vdt, vdl using _vt.vdiff as vd inner join _vt.vdiff_table as vdt on (vd.id = vdt.vdiff_id) + inner join _vt.vdiff_log as vdl on (vd.id = vdl.vdiff_id) + where vd.keyspace = %s and vd.workflow = %s` + query := fmt.Sprintf(sqlDeleteVDiffs, encodeString(tablet.Keyspace), encodeString(workflow)) + rows := -1 + if _, err := s.tmc.ExecuteFetchAsAllPrivs(ctx, tablet, &tabletmanagerdatapb.ExecuteFetchAsAllPrivsRequest{ + Query: []byte(query), + MaxRows: uint64(rows), + }); err != nil { + if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Num != mysql.ERNoSuchTable { // the tables may not exist if no vdiffs have been run + log.Errorf("Error deleting vdiff data for %s.%s workflow: %v", tablet.Keyspace, workflow, err) + } + } +} + +// optimizeCopyStateTable rebuilds the copy_state table to ensure the on-disk +// structures are minimal and optimized and resets the auto-inc value for +// subsequent inserts. +// This helps to ensure that the size, storage, and performance related factors +// for the table remain optimal over time and that we don't ever exhaust the +// available auto-inc values for the table. +// Note: it's not critical that this executes successfully any given time, it's +// only important that we try to do this periodically so that things stay in an +// optimal state over long periods of time. For this reason, the work is done +// asynchronously in the background on the given tablet and any failures are +// logged as warnings. Because it's done in the background we use the AllPrivs +// account to be sure that we don't execute the writes if READ_ONLY is set on +// the MySQL instance. +func (s *Server) optimizeCopyStateTable(tablet *topodatapb.Tablet) { + if s.sem != nil { + if !s.sem.TryAcquire(1) { + log.Warningf("Deferring work to optimize the copy_state table on %q due to hitting the maximum concurrent background job limit.", + tablet.Alias.String()) + return + } + } + go func() { + defer func() { + if s.sem != nil { + s.sem.Release(1) + } + }() + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Minute) + defer cancel() + sqlOptimizeTable := "optimize table _vt.copy_state" + if _, err := s.tmc.ExecuteFetchAsAllPrivs(ctx, tablet, &tabletmanagerdatapb.ExecuteFetchAsAllPrivsRequest{ + Query: []byte(sqlOptimizeTable), + MaxRows: uint64(100), // always produces 1+rows with notes and status + }); err != nil { + if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Num == mysql.ERNoSuchTable { // the table may not exist + return + } + log.Warningf("Failed to optimize the copy_state table on %q: %v", tablet.Alias.String(), err) + } + // This will automatically set the value to 1 or the current max value in the + // table, whichever is greater. + sqlResetAutoInc := "alter table _vt.copy_state auto_increment = 1" + if _, err := s.tmc.ExecuteFetchAsAllPrivs(ctx, tablet, &tabletmanagerdatapb.ExecuteFetchAsAllPrivsRequest{ + Query: []byte(sqlResetAutoInc), + MaxRows: uint64(0), + }); err != nil { + log.Warningf("Failed to reset the auto_increment value for the copy_state table on %q: %v", + tablet.Alias.String(), err) + } + }() +} + +// DropTargets cleans up target tables, shards and denied tables if a MoveTables/Reshard +// is cancelled. +func (s *Server) DropTargets(ctx context.Context, targetKeyspace, workflow string, keepData, keepRoutingRules, dryRun bool) (*[]string, error) { + ts, state, err := s.getWorkflowState(ctx, targetKeyspace, workflow) + if err != nil { + log.Errorf("Failed to get VReplication workflow state for %s.%s: %v", targetKeyspace, workflow, err) + return nil, err + } + + // Return an error if the workflow traffic is partially switched. + if state.WritesSwitched || len(state.ReplicaCellsSwitched) > 0 || len(state.RdonlyCellsSwitched) > 0 { + return nil, ErrWorkflowPartiallySwitched + } + + if state.WorkflowType == TypeMigrate { + _, err := s.finalizeMigrateWorkflow(ctx, targetKeyspace, workflow, "", true, keepData, keepRoutingRules, dryRun) + return nil, err + } + + ts.keepRoutingRules = keepRoutingRules + var sw iswitcher + if dryRun { + sw = &switcherDryRun{ts: ts, drLog: NewLogRecorder()} + } else { + sw = &switcher{s: s, ts: ts} + } + var tctx context.Context + tctx, sourceUnlock, lockErr := sw.lockKeyspace(ctx, ts.SourceKeyspaceName(), "DropTargets") + if lockErr != nil { + ts.Logger().Errorf("Source LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer sourceUnlock(&err) + ctx = tctx + + if ts.TargetKeyspaceName() != ts.SourceKeyspaceName() { + tctx, targetUnlock, lockErr := sw.lockKeyspace(ctx, ts.TargetKeyspaceName(), "DropTargets") + if lockErr != nil { + ts.Logger().Errorf("Target LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer targetUnlock(&err) + ctx = tctx + } + if !keepData { + switch ts.MigrationType() { + case binlogdatapb.MigrationType_TABLES: + if err := sw.removeTargetTables(ctx); err != nil { + return nil, err + } + if err := sw.dropSourceDeniedTables(ctx); err != nil { + return nil, err + } + case binlogdatapb.MigrationType_SHARDS: + if err := sw.dropTargetShards(ctx); err != nil { + return nil, err + } + } + } + if err := s.dropRelatedArtifacts(ctx, keepRoutingRules, sw); err != nil { + return nil, err + } + if err := ts.TopoServer().RebuildSrvVSchema(ctx, nil); err != nil { + return nil, err + } + return sw.logs(), nil +} + +func (s *Server) buildTrafficSwitcher(ctx context.Context, targetKeyspace, workflowName string) (*trafficSwitcher, error) { + tgtInfo, err := BuildTargets(ctx, s.ts, s.tmc, targetKeyspace, workflowName) + if err != nil { + log.Infof("Error building targets: %s", err) + return nil, err + } + targets, frozen, optCells, optTabletTypes := tgtInfo.Targets, tgtInfo.Frozen, tgtInfo.OptCells, tgtInfo.OptTabletTypes + + ts := &trafficSwitcher{ + ws: s, + logger: logutil.NewConsoleLogger(), + workflow: workflowName, + reverseWorkflow: ReverseWorkflowName(workflowName), + id: HashStreams(targetKeyspace, targets), + targets: targets, + sources: make(map[string]*MigrationSource), + targetKeyspace: targetKeyspace, + frozen: frozen, + optCells: optCells, + optTabletTypes: optTabletTypes, + workflowType: tgtInfo.WorkflowType, + workflowSubType: tgtInfo.WorkflowSubType, + } + log.Infof("Migration ID for workflow %s: %d", workflowName, ts.id) + sourceTopo := s.ts + + // Build the sources. + for _, target := range targets { + for _, bls := range target.Sources { + if ts.sourceKeyspace == "" { + ts.sourceKeyspace = bls.Keyspace + ts.sourceTimeZone = bls.SourceTimeZone + ts.targetTimeZone = bls.TargetTimeZone + ts.externalCluster = bls.ExternalCluster + if ts.externalCluster != "" { + externalTopo, err := s.ts.OpenExternalVitessClusterServer(ctx, ts.externalCluster) + if err != nil { + return nil, err + } + sourceTopo = externalTopo + ts.externalTopo = externalTopo + } + } else if ts.sourceKeyspace != bls.Keyspace { + return nil, fmt.Errorf("source keyspaces are mismatched across streams: %v vs %v", ts.sourceKeyspace, bls.Keyspace) + } + + if ts.tables == nil { + for _, rule := range bls.Filter.Rules { + ts.tables = append(ts.tables, rule.Match) + } + sort.Strings(ts.tables) + } else { + var tables []string + for _, rule := range bls.Filter.Rules { + tables = append(tables, rule.Match) + } + sort.Strings(tables) + if !reflect.DeepEqual(ts.tables, tables) { + return nil, fmt.Errorf("table lists are mismatched across streams: %v vs %v", ts.tables, tables) + } + } + + if _, ok := ts.sources[bls.Shard]; ok { + continue + } + sourcesi, err := sourceTopo.GetShard(ctx, bls.Keyspace, bls.Shard) + if err != nil { + return nil, err + } + if sourcesi.PrimaryAlias == nil { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "source shard %s/%s currently has no primary tablet", + bls.Keyspace, bls.Shard) + } + sourcePrimary, err := sourceTopo.GetTablet(ctx, sourcesi.PrimaryAlias) + if err != nil { + return nil, err + } + ts.sources[bls.Shard] = NewMigrationSource(sourcesi, sourcePrimary) + } + } + if ts.sourceKeyspace != ts.targetKeyspace || ts.externalCluster != "" { + ts.migrationType = binlogdatapb.MigrationType_TABLES + } else { + // TODO(sougou): for shard migration, validate that source and target combined + // keyranges match. + ts.migrationType = binlogdatapb.MigrationType_SHARDS + for sourceShard := range ts.sources { + if _, ok := ts.targets[sourceShard]; ok { + // If shards are overlapping, then this is a table migration. + ts.migrationType = binlogdatapb.MigrationType_TABLES + break + } + } + } + vs, err := sourceTopo.GetVSchema(ctx, ts.sourceKeyspace) + if err != nil { + return nil, err + } + ts.sourceKSSchema, err = vindexes.BuildKeyspaceSchema(vs, ts.sourceKeyspace) + if err != nil { + return nil, err + } + + sourceShards, targetShards := ts.getSourceAndTargetShardsNames() + + ts.isPartialMigration, err = ts.isPartialMoveTables(sourceShards, targetShards) + if err != nil { + return nil, err + } + if ts.isPartialMigration { + log.Infof("Migration is partial, for shards %+v", sourceShards) + } + return ts, nil +} + +func (s *Server) dropRelatedArtifacts(ctx context.Context, keepRoutingRules bool, sw iswitcher) error { + if err := sw.dropSourceReverseVReplicationStreams(ctx); err != nil { + return err + } + if !keepRoutingRules { + if err := sw.deleteRoutingRules(ctx); err != nil { + return err + } + if err := sw.deleteShardRoutingRules(ctx); err != nil { + return err + } + } + + return nil +} + +// dropSources cleans up source tables, shards and denied tables after a +// MoveTables/Reshard is completed. +func (s *Server) dropSources(ctx context.Context, ts *trafficSwitcher, removalType TableRemovalType, keepData, keepRoutingRules, force, dryRun bool) (*[]string, error) { + var ( + sw iswitcher + err error + ) + if dryRun { + sw = &switcherDryRun{ts: ts, drLog: NewLogRecorder()} + } else { + sw = &switcher{ts: ts, s: s} + } + var tctx context.Context + tctx, sourceUnlock, lockErr := sw.lockKeyspace(ctx, ts.SourceKeyspaceName(), "DropSources") + if lockErr != nil { + ts.Logger().Errorf("Source LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer sourceUnlock(&err) + ctx = tctx + if ts.TargetKeyspaceName() != ts.SourceKeyspaceName() { + tctx, targetUnlock, lockErr := sw.lockKeyspace(ctx, ts.TargetKeyspaceName(), "DropSources") + if lockErr != nil { + ts.Logger().Errorf("Target LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer targetUnlock(&err) + ctx = tctx + } + if !force { + if err := sw.validateWorkflowHasCompleted(ctx); err != nil { + ts.Logger().Errorf("Workflow has not completed, cannot DropSources: %v", err) + return nil, err + } + } + if !keepData { + switch ts.MigrationType() { + case binlogdatapb.MigrationType_TABLES: + log.Infof("Deleting tables") + if err := sw.removeSourceTables(ctx, removalType); err != nil { + return nil, err + } + if err := sw.dropSourceDeniedTables(ctx); err != nil { + return nil, err + } + + case binlogdatapb.MigrationType_SHARDS: + log.Infof("Removing shards") + if err := sw.dropSourceShards(ctx); err != nil { + return nil, err + } + } + } + if err := s.dropArtifacts(ctx, keepRoutingRules, sw); err != nil { + return nil, err + } + if err := ts.TopoServer().RebuildSrvVSchema(ctx, nil); err != nil { + return nil, err + } + + return sw.logs(), nil +} + +func (s *Server) dropArtifacts(ctx context.Context, keepRoutingRules bool, sw iswitcher) error { + if err := sw.dropSourceReverseVReplicationStreams(ctx); err != nil { + return err + } + if err := sw.dropTargetVReplicationStreams(ctx); err != nil { + return err + } + if !keepRoutingRules { + if err := sw.deleteRoutingRules(ctx); err != nil { + return err + } + if err := sw.deleteShardRoutingRules(ctx); err != nil { + return err + } + } + + return nil +} + +// DeleteShard will do all the necessary changes in the topology server +// to entirely remove a shard. +func (s *Server) DeleteShard(ctx context.Context, keyspace, shard string, recursive, evenIfServing bool) error { + // Read the Shard object. If it's not there, try to clean up + // the topology anyway. + shardInfo, err := s.ts.GetShard(ctx, keyspace, shard) + if err != nil { + if topo.IsErrType(err, topo.NoNode) { + log.Infof("Shard %v/%v doesn't seem to exist, cleaning up any potential leftover", keyspace, shard) + return s.ts.DeleteShard(ctx, keyspace, shard) + } + return err + } + + servingCells, err := s.ts.GetShardServingCells(ctx, shardInfo) + if err != nil { + return err + } + // Check the Serving map for the shard, we don't want to + // remove a serving shard if not absolutely sure. + if !evenIfServing && len(servingCells) > 0 { + return fmt.Errorf("shard %v/%v is still serving, cannot delete it, use the even-if-serving flag if needed", keyspace, shard) + } + + cells, err := s.ts.GetCellInfoNames(ctx) + if err != nil { + return err + } + + // Go through all the cells. + for _, cell := range cells { + var aliases []*topodatapb.TabletAlias + + // Get the ShardReplication object for that cell. Try + // to find all tablets that may belong to our shard. + sri, err := s.ts.GetShardReplication(ctx, cell, keyspace, shard) + switch { + case topo.IsErrType(err, topo.NoNode): + // No ShardReplication object. It means the + // topo is inconsistent. Let's read all the + // tablets for that cell, and if we find any + // in our keyspace / shard, either abort or + // try to delete them. + aliases, err = s.ts.GetTabletAliasesByCell(ctx, cell) + if err != nil { + return fmt.Errorf("GetTabletsByCell(%v) failed: %v", cell, err) + } + case err == nil: + // We found a ShardReplication object. We + // trust it to have all tablet records. + aliases = make([]*topodatapb.TabletAlias, len(sri.Nodes)) + for i, n := range sri.Nodes { + aliases[i] = n.TabletAlias + } + default: + return fmt.Errorf("GetShardReplication(%v, %v, %v) failed: %v", cell, keyspace, shard, err) + } + + // Get the corresponding Tablet records. Note + // GetTabletMap ignores ErrNoNode, and it's good for + // our purpose, it means a tablet was deleted but is + // still referenced. + tabletMap, err := s.ts.GetTabletMap(ctx, aliases) + if err != nil { + return fmt.Errorf("GetTabletMap() failed: %v", err) + } + + // Remove the tablets that don't belong to our + // keyspace/shard from the map. + for a, ti := range tabletMap { + if ti.Keyspace != keyspace || ti.Shard != shard { + delete(tabletMap, a) + } + } + + // Now see if we need to DeleteTablet, and if we can, do it. + if len(tabletMap) > 0 { + if !recursive { + return fmt.Errorf("shard %v/%v still has %v tablets in cell %v; use --recursive or remove them manually", keyspace, shard, len(tabletMap), cell) + } + + log.Infof("Deleting all tablets in shard %v/%v cell %v", keyspace, shard, cell) + for tabletAlias, tabletInfo := range tabletMap { + // We don't care about scrapping or updating the replication graph, + // because we're about to delete the entire replication graph. + log.Infof("Deleting tablet %v", tabletAlias) + if err := s.ts.DeleteTablet(ctx, tabletInfo.Alias); err != nil && !topo.IsErrType(err, topo.NoNode) { + // We don't want to continue if a DeleteTablet fails for + // any good reason (other than missing tablet, in which + // case it's just a topology server inconsistency we can + // ignore). If we continue and delete the replication + // graph, the tablet record will be orphaned, since + // we'll no longer know it belongs to this shard. + // + // If the problem is temporary, or resolved externally, re-running + // DeleteShard will skip over tablets that were already deleted. + return fmt.Errorf("can't delete tablet %v: %v", tabletAlias, err) + } + } + } + } + + // Try to remove the replication graph and serving graph in each cell, + // regardless of its existence. + for _, cell := range cells { + if err := s.ts.DeleteShardReplication(ctx, cell, keyspace, shard); err != nil && !topo.IsErrType(err, topo.NoNode) { + log.Warningf("Cannot delete ShardReplication in cell %v for %v/%v: %v", cell, keyspace, shard, err) + } + } + + return s.ts.DeleteShard(ctx, keyspace, shard) +} + +// updateShardRecords updates the shard records based on 'from' or 'to' direction. +func (s *Server) updateShardRecords(ctx context.Context, keyspace string, shards []*topo.ShardInfo, cells []string, servedType topodatapb.TabletType, isFrom bool, clearSourceShards bool) (err error) { + return topotools.UpdateShardRecords(ctx, s.ts, s.tmc, keyspace, shards, cells, servedType, isFrom, clearSourceShards, nil) +} + +// refreshPrimaryTablets will just RPC-ping all the primary tablets with RefreshState +func (s *Server) refreshPrimaryTablets(ctx context.Context, shards []*topo.ShardInfo) error { + wg := sync.WaitGroup{} + rec := concurrency.AllErrorRecorder{} + for _, si := range shards { + wg.Add(1) + go func(si *topo.ShardInfo) { + defer wg.Done() + log.Infof("RefreshState primary %v", topoproto.TabletAliasString(si.PrimaryAlias)) + ti, err := s.ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + rec.RecordError(err) + return + } + + if err := s.tmc.RefreshState(ctx, ti.Tablet); err != nil { + rec.RecordError(err) + } else { + log.Infof("%v responded", topoproto.TabletAliasString(si.PrimaryAlias)) + } + }(si) + } + wg.Wait() + return rec.Error() +} + +// finalizeMigrateWorkflow deletes the streams for the Migrate workflow. +// We only cleanup the target for external sources. +func (s *Server) finalizeMigrateWorkflow(ctx context.Context, targetKeyspace, workflow, tableSpecs string, cancel, keepData, keepRoutingRules, dryRun bool) (*[]string, error) { + ts, err := s.buildTrafficSwitcher(ctx, targetKeyspace, workflow) + if err != nil { + ts.Logger().Errorf("buildTrafficSwitcher failed: %v", err) + return nil, err + } + var sw iswitcher + if dryRun { + sw = &switcherDryRun{ts: ts, drLog: NewLogRecorder()} + } else { + sw = &switcher{s: s, ts: ts} + } + var tctx context.Context + tctx, targetUnlock, lockErr := sw.lockKeyspace(ctx, ts.TargetKeyspaceName(), "completeMigrateWorkflow") + if lockErr != nil { + ts.Logger().Errorf("Target LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer targetUnlock(&err) + ctx = tctx + if err := sw.dropTargetVReplicationStreams(ctx); err != nil { + return nil, err + } + if !cancel { + if err := sw.addParticipatingTablesToKeyspace(ctx, targetKeyspace, tableSpecs); err != nil { + return nil, err + } + if err := ts.TopoServer().RebuildSrvVSchema(ctx, nil); err != nil { + return nil, err + } + } + log.Infof("cancel is %t, keepData %t", cancel, keepData) + if cancel && !keepData { + if err := sw.removeTargetTables(ctx); err != nil { + return nil, err + } + } + return sw.logs(), nil +} + +// WorkflowSwitchTraffic switches traffic in the direction passed for specified tablet types. +func (s *Server) WorkflowSwitchTraffic(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest) (*vtctldatapb.WorkflowSwitchTrafficResponse, error) { + var ( + dryRunResults []string + rdDryRunResults, wrDryRunResults *[]string + hasReplica, hasRdonly, hasPrimary bool + ) + + timeout, set, err := protoutil.DurationFromProto(req.Timeout) + if err != nil { + err = vterrors.Wrapf(err, "unable to parse Timeout into a valid duration") + return nil, err + } + if !set { + timeout = defaultDuration + } + + ts, startState, err := s.getWorkflowState(ctx, req.Keyspace, req.Workflow) + if err != nil { + return nil, err + } + + if startState.WorkflowType == TypeMigrate { + return nil, fmt.Errorf("invalid action for Migrate workflow: SwitchTraffic") + } + + maxReplicationLagAllowed, set, err := protoutil.DurationFromProto(req.MaxReplicationLagAllowed) + if err != nil { + err = vterrors.Wrapf(err, "unable to parse MaxReplicationLagAllowed into a valid duration") + return nil, err + } + if !set { + maxReplicationLagAllowed = defaultDuration + } + + direction := TrafficSwitchDirection(req.Direction) + if direction == DirectionBackward { + ts, startState, err = s.getWorkflowState(ctx, startState.SourceKeyspace, ts.reverseWorkflow) + if err != nil { + return nil, err + } + } + reason, err := s.canSwitch(ctx, ts, startState, direction, int64(maxReplicationLagAllowed.Seconds())) + if err != nil { + return nil, err + } + if reason != "" { + return nil, fmt.Errorf("cannot switch traffic for workflow %s at this time: %s", startState.Workflow, reason) + } + + hasReplica, hasRdonly, hasPrimary, err = parseTabletTypes(req.TabletTypes) + if err != nil { + return nil, err + } + if hasReplica || hasRdonly { + if rdDryRunResults, err = s.switchReads(ctx, req, ts, startState, timeout, false, direction); err != nil { + return nil, err + } + } + if rdDryRunResults != nil { + dryRunResults = append(dryRunResults, *rdDryRunResults...) + } + if hasPrimary { + if _, wrDryRunResults, err = s.switchWrites(ctx, req, ts, timeout, false, req.EnableReverseReplication); err != nil { + return nil, err + } + } + + if wrDryRunResults != nil { + dryRunResults = append(dryRunResults, *wrDryRunResults...) + } + if req.DryRun && len(dryRunResults) == 0 { + dryRunResults = append(dryRunResults, "No changes required") + } + cmd := "SwitchTraffic" + if direction == DirectionBackward { + cmd = "ReverseTraffic" + } + resp := &vtctldatapb.WorkflowSwitchTrafficResponse{} + if req.DryRun { + resp.Summary = fmt.Sprintf("%s dry run results for workflow %s.%s at %v", cmd, req.Keyspace, req.Workflow, time.Now().UTC().Format(time.RFC822)) + resp.DryRunResults = dryRunResults + } else { + resp.Summary = fmt.Sprintf("%s was successful for workflow %s.%s", cmd, req.Keyspace, req.Workflow) + // Reload the state after the SwitchTraffic operation + // and return that as a string. + keyspace := req.Keyspace + workflow := req.Workflow + if direction == DirectionBackward { + keyspace = startState.SourceKeyspace + workflow = ts.reverseWorkflow + } + resp.StartState = startState.String() + _, currentState, err := s.getWorkflowState(ctx, keyspace, workflow) + if err != nil { + resp.CurrentState = fmt.Sprintf("Error reloading workflow state after switching traffic: %v", err) + } else { + resp.CurrentState = currentState.String() + } + } + return resp, nil +} + +// switchReads is a generic way of switching read traffic for a workflow. +func (s *Server) switchReads(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest, ts *trafficSwitcher, state *State, timeout time.Duration, cancel bool, direction TrafficSwitchDirection) (*[]string, error) { + roTypesToSwitchStr := topoproto.MakeStringTypeCSV(req.TabletTypes) + var hasReplica, hasRdonly bool + for _, roType := range req.TabletTypes { + switch roType { + case topodatapb.TabletType_REPLICA: + hasReplica = true + case topodatapb.TabletType_RDONLY: + hasRdonly = true + } + } + + log.Infof("Switching reads: %s.%s tablet types: %s, cells: %s, workflow state: %s", ts.targetKeyspace, ts.workflow, roTypesToSwitchStr, ts.optCells, state.String()) + if !hasReplica && !hasRdonly { + return nil, fmt.Errorf("tablet types must be REPLICA or RDONLY: %s", roTypesToSwitchStr) + } + if !ts.isPartialMigration { // shard level traffic switching is all or nothing + if direction == DirectionBackward && hasReplica && len(state.ReplicaCellsSwitched) == 0 { + return nil, fmt.Errorf("requesting reversal of read traffic for REPLICAs but REPLICA reads have not been switched") + } + if direction == DirectionBackward && hasRdonly && len(state.RdonlyCellsSwitched) == 0 { + return nil, fmt.Errorf("requesting reversal of SwitchReads for RDONLYs but RDONLY reads have not been switched") + } + } + var cells []string = req.Cells + // If no cells were provided in the command then use the value from the workflow. + if len(cells) == 0 && ts.optCells != "" { + cells = strings.Split(strings.TrimSpace(ts.optCells), ",") + } + + // If there are no rdonly tablets in the cells ask to switch rdonly tablets as well so that routing rules + // are updated for rdonly as well. Otherwise vitess will not know that the workflow has completed and will + // incorrectly report that not all reads have been switched. User currently is forced to switch non-existent + // rdonly tablets. + if hasReplica && !hasRdonly { + var err error + rdonlyTabletsExist, err := topotools.DoCellsHaveRdonlyTablets(ctx, s.ts, cells) + if err != nil { + return nil, err + } + if rdonlyTabletsExist { + return nil, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "requesting reversal of SwitchReads for REPLICAs but RDONLY tablets also exist in the cells") + } + } + + // If journals exist notify user and fail. + journalsExist, _, err := ts.checkJournals(ctx) + if err != nil { + ts.Logger().Errorf("checkJournals failed: %v", err) + return nil, err + } + if journalsExist { + log.Infof("Found a previous journal entry for %d", ts.id) + } + var sw iswitcher + if req.DryRun { + sw = &switcherDryRun{ts: ts, drLog: NewLogRecorder()} + } else { + sw = &switcher{ts: ts, s: s} + } + + if err := ts.validate(ctx); err != nil { + ts.Logger().Errorf("validate failed: %v", err) + return nil, err + } + + // For reads, locking the source keyspace is sufficient. + ctx, unlock, lockErr := sw.lockKeyspace(ctx, ts.SourceKeyspaceName(), "SwitchReads") + if lockErr != nil { + ts.Logger().Errorf("LockKeyspace failed: %v", lockErr) + return nil, lockErr + } + defer unlock(&err) + + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + if 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, cells, req.TabletTypes, direction); err != nil { + ts.Logger().Errorf("switchTableReads failed: %v", err) + return nil, err + } + return sw.logs(), nil + } + ts.Logger().Infof("About to switchShardReads: %+v, %+s, %+v", cells, roTypesToSwitchStr, direction) + if err := sw.switchShardReads(ctx, cells, req.TabletTypes, direction); err != nil { + ts.Logger().Errorf("switchShardReads failed: %v", err) + return nil, err + } + + ts.Logger().Infof("switchShardReads Completed: %+v, %+s, %+v", cells, roTypesToSwitchStr, direction) + if err := s.ts.ValidateSrvKeyspace(ctx, ts.targetKeyspace, strings.Join(cells, ",")); err != nil { + err2 := vterrors.Wrapf(err, "After switching shard reads, found SrvKeyspace for %s is corrupt in cell %s", + ts.targetKeyspace, strings.Join(cells, ",")) + ts.Logger().Errorf("%w", err2) + return nil, err2 + } + return sw.logs(), nil +} + +// switchWrites is a generic way of migrating write traffic for a workflow. +func (s *Server) switchWrites(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest, ts *trafficSwitcher, timeout time.Duration, + cancel, reverseReplication bool) (journalID int64, dryRunResults *[]string, err error) { + + var sw iswitcher + if req.DryRun { + sw = &switcherDryRun{ts: ts, drLog: NewLogRecorder()} + } else { + sw = &switcher{ts: ts, s: s} + } + + if ts.frozen { + ts.Logger().Warningf("Writes have already been switched for workflow %s, nothing to do here", ts.WorkflowName()) + return 0, sw.logs(), nil + } + + if err := ts.validate(ctx); err != nil { + ts.Logger().Errorf("validate failed: %v", err) + return 0, nil, err + } + + if reverseReplication { + err := areTabletsAvailableToStreamFrom(ctx, req, ts, ts.TargetKeyspaceName(), ts.TargetShards()) + if err != nil { + return 0, nil, err + } + } + + // Need to lock both source and target keyspaces. + tctx, sourceUnlock, lockErr := sw.lockKeyspace(ctx, ts.SourceKeyspaceName(), "SwitchWrites") + if lockErr != nil { + ts.Logger().Errorf("LockKeyspace failed: %v", lockErr) + return 0, nil, lockErr + } + ctx = tctx + defer sourceUnlock(&err) + if ts.TargetKeyspaceName() != ts.SourceKeyspaceName() { + tctx, targetUnlock, lockErr := sw.lockKeyspace(ctx, ts.TargetKeyspaceName(), "SwitchWrites") + if lockErr != nil { + ts.Logger().Errorf("LockKeyspace failed: %v", lockErr) + return 0, nil, lockErr + } + ctx = tctx + defer targetUnlock(&err) + } + + // If no journals exist, sourceWorkflows will be initialized by sm.MigrateStreams. + journalsExist, sourceWorkflows, err := ts.checkJournals(ctx) + if err != nil { + ts.Logger().Errorf("checkJournals failed: %v", err) + return 0, nil, err + } + if !journalsExist { + ts.Logger().Infof("No previous journals were found. Proceeding normally.") + sm, err := BuildStreamMigrator(ctx, ts, cancel) + if err != nil { + ts.Logger().Errorf("buildStreamMigrater failed: %v", err) + return 0, nil, err + } + if cancel { + sw.cancelMigration(ctx, sm) + return 0, sw.logs(), nil + } + + ts.Logger().Infof("Stopping streams") + sourceWorkflows, err = sw.stopStreams(ctx, sm) + if err != nil { + ts.Logger().Errorf("stopStreams failed: %v", err) + for key, streams := range sm.Streams() { + for _, stream := range streams { + ts.Logger().Errorf("stream in stopStreams: key %s shard %s stream %+v", key, stream.BinlogSource.Shard, stream.BinlogSource) + } + } + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + + ts.Logger().Infof("Stopping source writes") + if err := sw.stopSourceWrites(ctx); err != nil { + ts.Logger().Errorf("stopSourceWrites failed: %v", err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + ts.Logger().Infof("Executing LOCK TABLES on source tables %d times", lockTablesCycles) + // Doing this twice with a pause in-between to catch any writes that may have raced in between + // the tablet's deny list check and the first mysqld side table lock. + for cnt := 1; cnt <= lockTablesCycles; cnt++ { + if err := ts.executeLockTablesOnSource(ctx); err != nil { + ts.Logger().Errorf("Failed to execute LOCK TABLES (attempt %d of %d) on sources: %v", cnt, lockTablesCycles, err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + // No need to UNLOCK the tables as the connection was closed once the locks were acquired + // and thus the locks released. + time.Sleep(lockTablesCycleDelay) + } + } + + ts.Logger().Infof("Waiting for streams to catchup") + if err := sw.waitForCatchup(ctx, timeout); err != nil { + ts.Logger().Errorf("waitForCatchup failed: %v", err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + + ts.Logger().Infof("Migrating streams") + if err := sw.migrateStreams(ctx, sm); err != nil { + ts.Logger().Errorf("migrateStreams failed: %v", err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + + ts.Logger().Infof("Resetting sequences") + if err := sw.resetSequences(ctx); err != nil { + ts.Logger().Errorf("resetSequences failed: %v", err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + + ts.Logger().Infof("Creating reverse streams") + if err := sw.createReverseVReplication(ctx); err != nil { + ts.Logger().Errorf("createReverseVReplication failed: %v", err) + sw.cancelMigration(ctx, sm) + return 0, nil, err + } + } else { + if cancel { + err := fmt.Errorf("traffic switching has reached the point of no return, cannot cancel") + ts.Logger().Errorf("%v", err) + return 0, nil, err + } + ts.Logger().Infof("Journals were found. Completing the left over steps.") + // Need to gather positions in case all journals were not created. + if err := ts.gatherPositions(ctx); err != nil { + ts.Logger().Errorf("gatherPositions failed: %v", err) + return 0, nil, err + } + } + + // This is the point of no return. Once a journal is created, + // traffic can be redirected to target shards. + if err := sw.createJournals(ctx, sourceWorkflows); err != nil { + ts.Logger().Errorf("createJournals failed: %v", err) + return 0, nil, err + } + if err := sw.allowTargetWrites(ctx); err != nil { + ts.Logger().Errorf("allowTargetWrites failed: %v", err) + return 0, nil, err + } + if err := sw.changeRouting(ctx); err != nil { + ts.Logger().Errorf("changeRouting failed: %v", err) + return 0, nil, err + } + if err := sw.streamMigraterfinalize(ctx, ts, sourceWorkflows); err != nil { + ts.Logger().Errorf("finalize failed: %v", err) + return 0, nil, err + } + if reverseReplication { + if err := sw.startReverseVReplication(ctx); err != nil { + ts.Logger().Errorf("startReverseVReplication failed: %v", err) + return 0, nil, err + } + } + + if err := sw.freezeTargetVReplication(ctx); err != nil { + ts.Logger().Errorf("deleteTargetVReplication failed: %v", err) + return 0, nil, err + } + + return ts.id, sw.logs(), nil +} + +func (s *Server) canSwitch(ctx context.Context, ts *trafficSwitcher, state *State, direction TrafficSwitchDirection, maxAllowedReplLagSecs int64) (reason string, err error) { + if direction == DirectionForward && state.WritesSwitched || + direction == DirectionBackward && !state.WritesSwitched { + log.Infof("writes already switched no need to check lag") + return "", nil + } + wf, err := s.GetWorkflow(ctx, state.TargetKeyspace, state.Workflow) + if err != nil { + return "", err + } + for _, stream := range wf.ShardStreams { + for _, st := range stream.GetStreams() { + if st.Message == Frozen { + return cannotSwitchFrozen, nil + } + // If no new events have been replicated after the copy phase then it will be 0. + if vreplLag := time.Now().Unix() - st.TimeUpdated.Seconds; vreplLag > maxAllowedReplLagSecs { + return fmt.Sprintf(cannotSwitchHighLag, vreplLag, maxAllowedReplLagSecs), nil + } + switch st.State { + case binlogdatapb.VReplicationWorkflowState_Copying.String(): + return cannotSwitchCopyIncomplete, nil + case binlogdatapb.VReplicationWorkflowState_Error.String(): + return cannotSwitchError, nil + } + } + } + + // Ensure that the tablets on both sides are in good shape as we make this same call in the + // process and an error will cause us to backout. + refreshErrors := strings.Builder{} + var m sync.Mutex + var wg sync.WaitGroup + rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout) + defer cancel() + refreshTablets := func(shards []*topo.ShardInfo, stype string) { + defer wg.Done() + for _, si := range shards { + if partial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, s.ts, s.tmc, si, nil, ts.Logger()); err != nil || partial { + m.Lock() + refreshErrors.WriteString(fmt.Sprintf("failed to successfully refresh all tablets in the %s/%s %s shard (%v):\n %v\n", + si.Keyspace(), si.ShardName(), stype, err, partialDetails)) + m.Unlock() + } + } + } + wg.Add(1) + go refreshTablets(ts.SourceShards(), "source") + wg.Add(1) + go refreshTablets(ts.TargetShards(), "target") + wg.Wait() + if refreshErrors.Len() > 0 { + return fmt.Sprintf(cannotSwitchFailedTabletRefresh, refreshErrors.String()), nil + } + return "", nil +} + +// VReplicationExec executes a query remotely using the DBA pool. +func (s *Server) VReplicationExec(ctx context.Context, tabletAlias *topodatapb.TabletAlias, query string) (*querypb.QueryResult, error) { + ti, err := s.ts.GetTablet(ctx, tabletAlias) + if err != nil { + return nil, err + } + return s.tmc.VReplicationExec(ctx, ti.Tablet, query) +} diff --git a/go/vt/vtctl/workflow/state.go b/go/vt/vtctl/workflow/state.go index 613f82d0b43..927f5a9db56 100644 --- a/go/vt/vtctl/workflow/state.go +++ b/go/vt/vtctl/workflow/state.go @@ -16,15 +16,45 @@ limitations under the License. package workflow -// Type is the type of a workflow. +import ( + "fmt" + "strings" +) + +// VReplicationWorkflowType specifies whether workflow is +// MoveTables or Reshard and maps directly to what is stored +// in the backend database. +type VReplicationWorkflowType int + +// VReplicationWorkflowType enums. +const ( + MoveTablesWorkflow = VReplicationWorkflowType(iota) + ReshardWorkflow + MigrateWorkflow +) + +// Type is the type of a workflow as a string and maps directly +// to what is provided and presented to the user. type Type string -// Workflow types. +// Workflow string types. const ( - TypeReshard Type = "Reshard" TypeMoveTables Type = "MoveTables" + TypeReshard Type = "Reshard" + TypeMigrate Type = "Migrate" ) +var TypeStrMap = map[VReplicationWorkflowType]Type{ + MoveTablesWorkflow: TypeMoveTables, + ReshardWorkflow: TypeReshard, + MigrateWorkflow: TypeMigrate, +} +var TypeIntMap = map[Type]VReplicationWorkflowType{ + TypeMoveTables: MoveTablesWorkflow, + TypeReshard: ReshardWorkflow, + TypeMigrate: MigrateWorkflow, +} + // State represents the state of a workflow. type State struct { Workflow string @@ -45,3 +75,52 @@ type State struct { ShardsAlreadySwitched []string ShardsNotYetSwitched []string } + +func (s *State) String() string { + var stateInfo []string + if !s.IsPartialMigration { // shard level traffic switching is all or nothing + if len(s.RdonlyCellsNotSwitched) == 0 && len(s.ReplicaCellsNotSwitched) == 0 && len(s.ReplicaCellsSwitched) > 0 { + stateInfo = append(stateInfo, "All Reads Switched") + } else if len(s.RdonlyCellsSwitched) == 0 && len(s.ReplicaCellsSwitched) == 0 { + stateInfo = append(stateInfo, "Reads Not Switched") + } else { + stateInfo = append(stateInfo, "Reads partially switched") + if len(s.ReplicaCellsNotSwitched) == 0 { + stateInfo = append(stateInfo, "All Replica Reads Switched") + } else if len(s.ReplicaCellsSwitched) == 0 { + stateInfo = append(stateInfo, "Replica not switched") + } else { + stateInfo = append(stateInfo, "Replica switched in cells: "+strings.Join(s.ReplicaCellsSwitched, ",")) + } + if len(s.RdonlyCellsNotSwitched) == 0 { + stateInfo = append(stateInfo, "All Rdonly Reads Switched") + } else if len(s.RdonlyCellsSwitched) == 0 { + stateInfo = append(stateInfo, "Rdonly not switched") + } else { + stateInfo = append(stateInfo, "Rdonly switched in cells: "+strings.Join(s.RdonlyCellsSwitched, ",")) + } + } + } + if s.WritesSwitched { + stateInfo = append(stateInfo, "Writes Switched") + } else if s.IsPartialMigration { + // For partial migrations, the traffic switching is all or nothing + // at the shard level, so reads are effectively switched on the + // shard when writes are switched. + if len(s.ShardsAlreadySwitched) > 0 && len(s.ShardsNotYetSwitched) > 0 { + stateInfo = append(stateInfo, fmt.Sprintf("Reads partially switched, for shards: %s", strings.Join(s.ShardsAlreadySwitched, ","))) + stateInfo = append(stateInfo, fmt.Sprintf("Writes partially switched, for shards: %s", strings.Join(s.ShardsAlreadySwitched, ","))) + } else { + if len(s.ShardsAlreadySwitched) == 0 { + stateInfo = append(stateInfo, "Reads Not Switched") + stateInfo = append(stateInfo, "Writes Not Switched") + } else { + stateInfo = append(stateInfo, "All Reads Switched") + stateInfo = append(stateInfo, "All Writes Switched") + } + } + } else { + stateInfo = append(stateInfo, "Writes Not Switched") + } + return strings.Join(stateInfo, ". ") +} diff --git a/go/vt/vtctl/workflow/stream_migrator.go b/go/vt/vtctl/workflow/stream_migrator.go index 6d6929a9b4a..aa004b0bd23 100644 --- a/go/vt/vtctl/workflow/stream_migrator.go +++ b/go/vt/vtctl/workflow/stream_migrator.go @@ -27,7 +27,6 @@ import ( "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/logutil" @@ -564,7 +563,7 @@ func (sm *StreamMigrator) createTargetStreams(ctx context.Context, tmpl []*VRepl } return sm.ts.ForAllTargets(func(target *MigrationTarget) error { - ig := vreplication.NewInsertGenerator(binlogplayer.BlpStopped, target.GetPrimary().DbName()) + ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, target.GetPrimary().DbName()) tabletStreams := VReplicationStreams(tmpl).Copy().ToSlice() for _, vrs := range tabletStreams { diff --git a/go/vt/vtctl/workflow/stream_migrator_test.go b/go/vt/vtctl/workflow/stream_migrator_test.go index 903e873a130..04f787eb4d4 100644 --- a/go/vt/vtctl/workflow/stream_migrator_test.go +++ b/go/vt/vtctl/workflow/stream_migrator_test.go @@ -24,7 +24,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "vitess.io/vitess/go/vt/proto/vschema" "vitess.io/vitess/go/vt/vtgate/vindexes" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" @@ -282,20 +281,20 @@ func TestTemplatize(t *testing.T) { }} vs := &vschemapb.Keyspace{ Sharded: true, - Vindexes: map[string]*vschema.Vindex{ + Vindexes: map[string]*vschemapb.Vindex{ "thash": { Type: "hash", }, }, - Tables: map[string]*vschema.Table{ + Tables: map[string]*vschemapb.Table{ "t1": { - ColumnVindexes: []*vschema.ColumnVindex{{ + ColumnVindexes: []*vschemapb.ColumnVindex{{ Columns: []string{"c1"}, Name: "thash", }}, }, "t2": { - ColumnVindexes: []*vschema.ColumnVindex{{ + ColumnVindexes: []*vschemapb.ColumnVindex{{ Columns: []string{"c1"}, Name: "thash", }}, diff --git a/go/vt/vtctl/workflow/switcher.go b/go/vt/vtctl/workflow/switcher.go new file mode 100644 index 00000000000..2199cbe3641 --- /dev/null +++ b/go/vt/vtctl/workflow/switcher.go @@ -0,0 +1,143 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "time" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +var _ iswitcher = (*switcher)(nil) + +type switcher struct { + s *Server + ts *trafficSwitcher +} + +func (r *switcher) addParticipatingTablesToKeyspace(ctx context.Context, keyspace, tableSpecs string) error { + return r.ts.addParticipatingTablesToKeyspace(ctx, keyspace, tableSpecs) +} + +func (r *switcher) deleteRoutingRules(ctx context.Context) error { + return r.ts.deleteRoutingRules(ctx) +} + +func (r *switcher) deleteShardRoutingRules(ctx context.Context) error { + return r.ts.deleteShardRoutingRules(ctx) +} + +func (r *switcher) dropSourceDeniedTables(ctx context.Context) error { + return r.ts.dropSourceDeniedTables(ctx) +} + +func (r *switcher) validateWorkflowHasCompleted(ctx context.Context) error { + return r.ts.validateWorkflowHasCompleted(ctx) +} + +func (r *switcher) removeSourceTables(ctx context.Context, removalType TableRemovalType) error { + return r.ts.removeSourceTables(ctx, removalType) +} + +func (r *switcher) dropSourceShards(ctx context.Context) error { + return r.ts.dropSourceShards(ctx) +} + +func (r *switcher) switchShardReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { + return r.ts.switchShardReads(ctx, cells, servedTypes, direction) +} + +func (r *switcher) switchTableReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { + return r.ts.switchTableReads(ctx, cells, servedTypes, direction) +} + +func (r *switcher) startReverseVReplication(ctx context.Context) error { + return r.ts.startReverseVReplication(ctx) +} + +func (r *switcher) createJournals(ctx context.Context, sourceWorkflows []string) error { + return r.ts.createJournals(ctx, sourceWorkflows) +} + +func (r *switcher) allowTargetWrites(ctx context.Context) error { + return r.ts.allowTargetWrites(ctx) +} + +func (r *switcher) changeRouting(ctx context.Context) error { + return r.ts.changeRouting(ctx) +} + +func (r *switcher) streamMigraterfinalize(ctx context.Context, ts *trafficSwitcher, workflows []string) error { + return StreamMigratorFinalize(ctx, ts, workflows) +} + +func (r *switcher) createReverseVReplication(ctx context.Context) error { + return r.ts.createReverseVReplication(ctx) +} + +func (r *switcher) migrateStreams(ctx context.Context, sm *StreamMigrator) error { + return sm.MigrateStreams(ctx) +} + +func (r *switcher) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error { + return r.ts.waitForCatchup(ctx, filteredReplicationWaitTime) +} + +func (r *switcher) stopSourceWrites(ctx context.Context) error { + return r.ts.stopSourceWrites(ctx) +} + +func (r *switcher) stopStreams(ctx context.Context, sm *StreamMigrator) ([]string, error) { + return sm.StopStreams(ctx) +} + +func (r *switcher) cancelMigration(ctx context.Context, sm *StreamMigrator) { + r.ts.cancelMigration(ctx, sm) +} + +func (r *switcher) lockKeyspace(ctx context.Context, keyspace, action string) (context.Context, func(*error), error) { + return r.s.ts.LockKeyspace(ctx, keyspace, action) +} + +func (r *switcher) freezeTargetVReplication(ctx context.Context) error { + return r.ts.freezeTargetVReplication(ctx) +} + +func (r *switcher) dropTargetVReplicationStreams(ctx context.Context) error { + return r.ts.dropTargetVReplicationStreams(ctx) +} + +func (r *switcher) dropSourceReverseVReplicationStreams(ctx context.Context) error { + return r.ts.dropSourceReverseVReplicationStreams(ctx) +} + +func (r *switcher) removeTargetTables(ctx context.Context) error { + return r.ts.removeTargetTables(ctx) +} + +func (r *switcher) dropTargetShards(ctx context.Context) error { + return r.ts.dropTargetShards(ctx) +} + +func (r *switcher) logs() *[]string { + return nil +} + +func (r *switcher) resetSequences(ctx context.Context) error { + return r.ts.resetSequences(ctx) +} diff --git a/go/vt/vtctl/workflow/switcher_dry_run.go b/go/vt/vtctl/workflow/switcher_dry_run.go new file mode 100644 index 00000000000..05bee1246b9 --- /dev/null +++ b/go/vt/vtctl/workflow/switcher_dry_run.go @@ -0,0 +1,367 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "fmt" + "sort" + "strings" + "time" + + "vitess.io/vitess/go/mysql" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +var _ iswitcher = (*switcherDryRun)(nil) + +type switcherDryRun struct { + drLog *LogRecorder + ts *trafficSwitcher +} + +func (dr *switcherDryRun) addParticipatingTablesToKeyspace(ctx context.Context, keyspace, tableSpecs string) error { + dr.drLog.Log("All source tables will be added to the target keyspace vschema") + return nil +} + +func (dr *switcherDryRun) deleteRoutingRules(ctx context.Context) error { + dr.drLog.Log("Routing rules for participating tables will be deleted") + return nil +} + +func (dr *switcherDryRun) deleteShardRoutingRules(ctx context.Context) error { + if dr.ts.isPartialMigration { + dr.drLog.Log("Shard routing rules for participating shards 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) + for _, source := range dr.ts.Sources() { + sourceShards = append(sourceShards, source.GetShard().ShardName()) + } + for _, target := range dr.ts.Targets() { + targetShards = append(targetShards, target.GetShard().ShardName()) + } + sort.Strings(sourceShards) + sort.Strings(targetShards) + if direction == DirectionForward { + dr.drLog.Logf("Switch reads from keyspace %s to keyspace %s for shards [%s] to shards [%s]", + dr.ts.SourceKeyspaceName(), dr.ts.TargetKeyspaceName(), strings.Join(sourceShards, ","), strings.Join(targetShards, ",")) + } else { + dr.drLog.Logf("Switch reads from keyspace %s to keyspace %s for shards [%s] to shards [%s]", + dr.ts.TargetKeyspaceName(), dr.ts.SourceKeyspaceName(), strings.Join(targetShards, ","), strings.Join(sourceShards, ",")) + } + return nil +} + +func (dr *switcherDryRun) switchTableReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { + ks := dr.ts.TargetKeyspaceName() + if direction == DirectionBackward { + ks = dr.ts.SourceKeyspaceName() + } + var tabletTypes []string + for _, servedType := range servedTypes { + tabletTypes = append(tabletTypes, servedType.String()) + } + tables := strings.Join(dr.ts.Tables(), ",") + dr.drLog.Logf("Switch reads for tables [%s] to keyspace %s for tablet types [%s]", tables, ks, strings.Join(tabletTypes, ",")) + dr.drLog.Logf("Routing rules for tables [%s] will be updated", tables) + return nil +} + +func (dr *switcherDryRun) createJournals(ctx context.Context, sourceWorkflows []string) error { + dr.drLog.Log("Create journal entries on source databases") + if len(sourceWorkflows) > 0 { + dr.drLog.Logf("Source workflows found: [%s]", strings.Join(sourceWorkflows, ",")) + } + return nil +} + +func (dr *switcherDryRun) allowTargetWrites(ctx context.Context) error { + dr.drLog.Logf("Enable writes on keyspace %s for tables [%s]", dr.ts.TargetKeyspaceName(), strings.Join(dr.ts.Tables(), ",")) + return nil +} + +func (dr *switcherDryRun) changeRouting(ctx context.Context) error { + dr.drLog.Logf("Switch routing from keyspace %s to keyspace %s", dr.ts.SourceKeyspaceName(), dr.ts.TargetKeyspaceName()) + var deleteLogs, addLogs []string + if dr.ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + tables := strings.Join(dr.ts.Tables(), ",") + dr.drLog.Logf("Routing rules for tables [%s] will be updated", tables) + return nil + } + deleteLogs = nil + addLogs = nil + for _, source := range dr.ts.Sources() { + deleteLogs = append(deleteLogs, fmt.Sprintf("shard:%s;tablet:%d", source.GetShard().ShardName(), source.GetShard().PrimaryAlias.Uid)) + } + for _, target := range dr.ts.Targets() { + addLogs = append(addLogs, fmt.Sprintf("shard:%s;tablet:%d", target.GetShard().ShardName(), target.GetShard().PrimaryAlias.Uid)) + } + if len(deleteLogs) > 0 { + dr.drLog.Logf("IsPrimaryServing will be set to false for: [%s]", strings.Join(deleteLogs, ",")) + dr.drLog.Logf("IsPrimaryServing will be set to true for: [%s]", strings.Join(addLogs, ",")) + } + return nil +} + +func (dr *switcherDryRun) streamMigraterfinalize(ctx context.Context, ts *trafficSwitcher, workflows []string) error { + logs := make([]string, 0) + for _, t := range ts.Targets() { + logs = append(logs, fmt.Sprintf("tablet:%d", t.GetPrimary().Alias.Uid)) + } + dr.drLog.Logf("Switch writes completed, freeze and delete vreplication streams on: [%s]", strings.Join(logs, ",")) + return nil +} + +func (dr *switcherDryRun) startReverseVReplication(ctx context.Context) error { + logs := make([]string, 0) + for _, t := range dr.ts.Sources() { + logs = append(logs, fmt.Sprintf("tablet:%d", t.GetPrimary().Alias.Uid)) + } + dr.drLog.Logf("Start reverse vreplication streams on: [%s]", strings.Join(logs, ",")) + return nil +} + +func (dr *switcherDryRun) createReverseVReplication(ctx context.Context) error { + dr.drLog.Logf("Create reverse vreplication workflow %s", dr.ts.ReverseWorkflowName()) + return nil +} + +func (dr *switcherDryRun) migrateStreams(ctx context.Context, sm *StreamMigrator) error { + templates := sm.Templates() + + if len(templates) == 0 { + return nil + } + logs := make([]string, 0) + + dr.drLog.Logf("Migrate streams to %s:", dr.ts.TargetKeyspaceName()) + for key, streams := range sm.Streams() { + for _, stream := range streams { + logs = append(logs, fmt.Sprintf("shard:%s;id:%d;workflow:%s;position:%s;binlogsource:%v", key, stream.ID, stream.Workflow, mysql.EncodePosition(stream.Position), stream.BinlogSource)) + } + } + if len(logs) > 0 { + dr.drLog.Logf("Migrate source streams: [%s]", strings.Join(logs, ",")) + logs = nil + } + for _, target := range dr.ts.Targets() { + tabletStreams := templates + for _, vrs := range tabletStreams { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;tablet:%d;workflow:%s;id:%d,position:%v;binlogsource:%s", + vrs.BinlogSource.Keyspace, vrs.BinlogSource.Shard, target.GetPrimary().Alias.Uid, vrs.Workflow, vrs.ID, mysql.EncodePosition(vrs.Position), vrs.BinlogSource)) + } + } + if len(logs) > 0 { + dr.drLog.Logf("Create target streams (as stopped): [%s]", strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error { + dr.drLog.Logf("Wait for vreplication on stopped streams to catchup for up to %v", filteredReplicationWaitTime) + return nil +} + +func (dr *switcherDryRun) stopSourceWrites(ctx context.Context) error { + logs := make([]string, 0) + for _, source := range dr.ts.Sources() { + position, _ := dr.ts.TabletManagerClient().PrimaryPosition(ctx, source.GetPrimary().Tablet) + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;position:%s", dr.ts.SourceKeyspaceName(), source.GetShard().ShardName(), position)) + } + if len(logs) > 0 { + dr.drLog.Logf("Stop writes on keyspace %s for tables [%s]: [%s]", dr.ts.SourceKeyspaceName(), + strings.Join(dr.ts.Tables(), ","), strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) stopStreams(ctx context.Context, sm *StreamMigrator) ([]string, error) { + logs := make([]string, 0) + for _, streams := range sm.Streams() { + for _, stream := range streams { + logs = append(logs, fmt.Sprintf("id:%d;keyspace:%s;shard:%s;rules:%s;position:%v", + stream.ID, stream.BinlogSource.Keyspace, stream.BinlogSource.Shard, stream.BinlogSource.Filter, stream.Position)) + } + } + if len(logs) > 0 { + dr.drLog.Logf("Stop streams on keyspace %s: [%s]", dr.ts.SourceKeyspaceName(), strings.Join(logs, ",")) + } + return nil, nil +} + +func (dr *switcherDryRun) cancelMigration(ctx context.Context, sm *StreamMigrator) { + dr.drLog.Log("Cancel stream migrations as requested") +} + +func (dr *switcherDryRun) lockKeyspace(ctx context.Context, keyspace, _ string) (context.Context, func(*error), error) { + dr.drLog.Logf("Lock keyspace %s", keyspace) + return ctx, func(e *error) { + dr.drLog.Logf("Unlock keyspace %s", keyspace) + }, nil +} + +func (dr *switcherDryRun) removeSourceTables(ctx context.Context, removalType TableRemovalType) error { + logs := make([]string, 0) + for _, source := range dr.ts.Sources() { + for _, tableName := range dr.ts.Tables() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;dbname:%s;tablet:%d;table:%s", + source.GetPrimary().Keyspace, source.GetPrimary().Shard, source.GetPrimary().DbName(), source.GetPrimary().Alias.Uid, tableName)) + } + } + action := "Dropping" + if removalType == RenameTable { + action = "Renaming" + } + if len(logs) > 0 { + dr.drLog.Logf("%s these tables from the database and removing them from the vschema for keyspace %s: [%s]", + action, dr.ts.SourceKeyspaceName(), strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) dropSourceShards(ctx context.Context) error { + logs := make([]string, 0) + tabletsList := make(map[string][]string) + for _, si := range dr.ts.SourceShards() { + tabletAliases, err := dr.ts.TopoServer().FindAllTabletAliasesInShard(ctx, si.Keyspace(), si.ShardName()) + if err != nil { + return err + } + tabletsList[si.ShardName()] = make([]string, 0) + for _, t := range tabletAliases { + tabletsList[si.ShardName()] = append(tabletsList[si.ShardName()], fmt.Sprintf("%d", t.Uid)) + } + sort.Strings(tabletsList[si.ShardName()]) + logs = append(logs, fmt.Sprintf("cell:%s;keyspace:%s;shards:[%s]", + si.Shard.PrimaryAlias.Cell, si.Keyspace(), si.ShardName()), strings.Join(tabletsList[si.ShardName()], ",")) + } + if len(logs) > 0 { + dr.drLog.Logf("Delete shards (and all related tablets): [%s]", strings.Join(logs, ",")) + } + + return nil +} + +func (dr *switcherDryRun) validateWorkflowHasCompleted(ctx context.Context) error { + return doValidateWorkflowHasCompleted(ctx, dr.ts) +} + +func (dr *switcherDryRun) dropTargetVReplicationStreams(ctx context.Context) error { + logs := make([]string, 0) + for _, t := range dr.ts.Targets() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;workflow:%s;dbname:%s;tablet:%d", + t.GetShard().Keyspace(), t.GetShard().ShardName(), dr.ts.WorkflowName(), t.GetPrimary().DbName(), t.GetPrimary().Alias.Uid)) + } + dr.drLog.Logf("Delete vreplication streams on targets: [%s]", strings.Join(logs, ",")) + return nil +} + +func (dr *switcherDryRun) dropSourceReverseVReplicationStreams(ctx context.Context) error { + logs := make([]string, 0) + for _, t := range dr.ts.Sources() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;workflow:%s;dbname:%s;tablet:%d", + t.GetShard().Keyspace(), t.GetShard().ShardName(), ReverseWorkflowName(dr.ts.WorkflowName()), t.GetPrimary().DbName(), t.GetPrimary().Alias.Uid)) + } + dr.drLog.Logf("Delete reverse vreplication streams on sources: [%s]", strings.Join(logs, ",")) + return nil +} + +func (dr *switcherDryRun) freezeTargetVReplication(ctx context.Context) error { + logs := make([]string, 0) + for _, target := range dr.ts.Targets() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;tablet:%d;workflow:%s;dbname:%s", + target.GetPrimary().Keyspace, target.GetPrimary().Shard, target.GetPrimary().Alias.Uid, dr.ts.WorkflowName(), target.GetPrimary().DbName())) + } + if len(logs) > 0 { + dr.drLog.Logf("Mark vreplication streams frozen on: [%s]", strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) dropSourceDeniedTables(ctx context.Context) error { + logs := make([]string, 0) + for _, si := range dr.ts.SourceShards() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;tablet:%d", si.Keyspace(), si.ShardName(), si.PrimaryAlias.Uid)) + } + if len(logs) > 0 { + dr.drLog.Logf("Denied tables records on [%s] will be removed from: [%s]", strings.Join(dr.ts.Tables(), ","), strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) logs() *[]string { + return &dr.drLog.logs +} + +func (dr *switcherDryRun) removeTargetTables(ctx context.Context) error { + logs := make([]string, 0) + for _, target := range dr.ts.Targets() { + for _, tableName := range dr.ts.Tables() { + logs = append(logs, fmt.Sprintf("keyspace:%s;shard:%s;dbname:%s;tablet:%d;table:%s", + target.GetPrimary().Keyspace, target.GetPrimary().Shard, target.GetPrimary().DbName(), target.GetPrimary().Alias.Uid, tableName)) + } + } + if len(logs) > 0 { + dr.drLog.Logf("Dropping these tables from the database and removing from the vschema for keyspace %s: [%s]", + dr.ts.TargetKeyspaceName(), strings.Join(logs, ",")) + } + return nil +} + +func (dr *switcherDryRun) dropTargetShards(ctx context.Context) error { + logs := make([]string, 0) + tabletsList := make(map[string][]string) + for _, si := range dr.ts.TargetShards() { + tabletAliases, err := dr.ts.TopoServer().FindAllTabletAliasesInShard(ctx, si.Keyspace(), si.ShardName()) + if err != nil { + return err + } + tabletsList[si.ShardName()] = make([]string, 0) + for _, t := range tabletAliases { + tabletsList[si.ShardName()] = append(tabletsList[si.ShardName()], fmt.Sprintf("%d", t.Uid)) + } + sort.Strings(tabletsList[si.ShardName()]) + logs = append(logs, fmt.Sprintf("cell:%s;keyspace:%s;shards:[%s]", + si.Shard.PrimaryAlias.Cell, si.Keyspace(), si.ShardName()), strings.Join(tabletsList[si.ShardName()], ",")) + } + if len(logs) > 0 { + dr.drLog.Logf("Delete shards (and all related tablets): [%s]", strings.Join(logs, ",")) + } + + return nil +} + +func (dr *switcherDryRun) resetSequences(ctx context.Context) error { + var err error + mustReset := false + if mustReset, err = dr.ts.mustResetSequences(ctx); err != nil { + return err + } + if !mustReset { + return nil + } + dr.drLog.Log("The sequence caches will be reset on the source since sequence tables are being moved") + return nil +} diff --git a/go/vt/vtctl/workflow/switcher_interface.go b/go/vt/vtctl/workflow/switcher_interface.go new file mode 100644 index 00000000000..c0b5265e63f --- /dev/null +++ b/go/vt/vtctl/workflow/switcher_interface.go @@ -0,0 +1,55 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "context" + "time" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +type iswitcher interface { + lockKeyspace(ctx context.Context, keyspace, action string) (context.Context, func(*error), error) + cancelMigration(ctx context.Context, sm *StreamMigrator) + stopStreams(ctx context.Context, sm *StreamMigrator) ([]string, error) + stopSourceWrites(ctx context.Context) error + waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error + migrateStreams(ctx context.Context, sm *StreamMigrator) error + createReverseVReplication(ctx context.Context) error + createJournals(ctx context.Context, sourceWorkflows []string) error + allowTargetWrites(ctx context.Context) error + changeRouting(ctx context.Context) error + streamMigraterfinalize(ctx context.Context, ts *trafficSwitcher, workflows []string) error + startReverseVReplication(ctx context.Context) error + switchTableReads(ctx context.Context, cells []string, servedType []topodatapb.TabletType, direction TrafficSwitchDirection) error + switchShardReads(ctx context.Context, cells []string, servedType []topodatapb.TabletType, direction TrafficSwitchDirection) error + validateWorkflowHasCompleted(ctx context.Context) error + removeSourceTables(ctx context.Context, removalType TableRemovalType) error + dropSourceShards(ctx context.Context) error + dropSourceDeniedTables(ctx context.Context) error + freezeTargetVReplication(ctx context.Context) error + dropSourceReverseVReplicationStreams(ctx context.Context) error + dropTargetVReplicationStreams(ctx context.Context) error + removeTargetTables(ctx context.Context) error + dropTargetShards(ctx context.Context) error + deleteRoutingRules(ctx context.Context) error + deleteShardRoutingRules(ctx context.Context) error + addParticipatingTablesToKeyspace(ctx context.Context, keyspace, tableSpecs string) error + logs() *[]string + resetSequences(ctx context.Context) error +} diff --git a/go/vt/vtctl/workflow/traffic_switcher.go b/go/vt/vtctl/workflow/traffic_switcher.go index 53a6e0ede9d..e0482dedeb0 100644 --- a/go/vt/vtctl/workflow/traffic_switcher.go +++ b/go/vt/vtctl/workflow/traffic_switcher.go @@ -17,43 +17,67 @@ limitations under the License. package workflow import ( - "bytes" "context" "errors" "fmt" - "hash/fnv" - "math" "sort" "strings" + "sync" + "time" - "google.golang.org/protobuf/encoding/prototext" - - "vitess.io/vitess/go/sets" - "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/json2" + "vitess.io/vitess/go/sqlescape" + "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/vindexes" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tmclient" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) const ( // Frozen is the message value of frozen vreplication streams. Frozen = "FROZEN" + // Running is the state value of a vreplication stream in the + // replicating state. + Running = "RUNNING" + + // How long to wait when refreshing the state of each tablet in a shard. Note that these + // are refreshed in parallel, non-topo errors are ignored (in the error handling) and we + // may only do a partial refresh. Because in some cases it's unsafe to switch the traffic + // if some tablets do not refresh, we may need to look for partial results and produce + // an error (with the provided details of WHY) if we see them. + // Side note: the default lock/lease TTL in etcd is 60s so the default tablet refresh + // timeout of 60s can cause us to lose our keyspace lock before completing the + // operation too. + shardTabletRefreshTimeout = time.Duration(30 * time.Second) + + // Use pt-osc's naming convention, this format also ensures vstreamer ignores such tables. + renameTableTemplate = "_%.59s_old" // limit table name to 64 characters + + sqlDeleteWorkflow = "delete from _vt.vreplication where db_name = %s and workflow = %s" ) -var ( - // ErrNoStreams occurs when no target streams are found for a workflow in a - // target keyspace. - ErrNoStreams = errors.New("no streams found") -) +// accessType specifies the type of access for a shard (allow/disallow writes). +type accessType int -// TrafficSwitchDirection specifies the switching direction. -type TrafficSwitchDirection int +const ( + allowWrites = accessType(iota) + disallowWrites +) // The following constants define the switching direction. const ( @@ -61,20 +85,29 @@ const ( DirectionBackward ) -// TableRemovalType specifies the way the a table will be removed during a -// DropSource for a MoveTables workflow. -type TableRemovalType int - // The following consts define if DropSource will drop or rename the table. const ( DropTable = TableRemovalType(iota) RenameTable ) -var tableRemovalTypeStrs = [...]string{ - "DROP TABLE", - "RENAME TABLE", -} +// TrafficSwitchDirection specifies the switching direction. +type TrafficSwitchDirection int + +// TableRemovalType specifies the way the a table will be removed during a +// DropSource for a MoveTables workflow. +type TableRemovalType int + +var ( + // ErrNoStreams occurs when no target streams are found for a workflow in a + // target keyspace. + ErrNoStreams = errors.New("no streams found") + + tableRemovalTypeStrs = []string{ + "DROP TABLE", + "RENAME TABLE", + } +) // String returns a string representation of a TableRemovalType func (trt TableRemovalType) String() string { @@ -85,13 +118,13 @@ func (trt TableRemovalType) String() string { return tableRemovalTypeStrs[trt] } -// ITrafficSwitcher is a temporary hack to allow us to move streamMigrater out -// of package wrangler without also needing to move trafficSwitcher in the same -// changeset. +// ITrafficSwitcher is a hack to allow us to maintain the legacy wrangler +// package for vtctl/vtctlclient while migrating most of the TrafficSwitcher +// related code to the workflow package for vtctldclient usage. // -// After moving TrafficSwitcher to this package, this type should be removed, -// and StreamMigrator should be updated to contain a field of type -// *TrafficSwitcher instead of ITrafficSwitcher. +// After moving TrafficSwitcher to this package and removing the implementation +// in wrangler, this type should be removed, and StreamMigrator should be updated +// to contain a field of type *TrafficSwitcher instead of ITrafficSwitcher. type ITrafficSwitcher interface { /* Functions that expose types and behavior contained in *wrangler.Wrangler */ @@ -165,6 +198,96 @@ func (source *MigrationSource) GetPrimary() *topo.TabletInfo { return source.primary } +// trafficSwitcher contains the metadata for switching read and write traffic +// for vreplication streams. +type trafficSwitcher struct { + ws *Server + logger logutil.Logger + + migrationType binlogdatapb.MigrationType + isPartialMigration bool + workflow string + + // if frozen is true, the rest of the fields are not set. + frozen bool + reverseWorkflow string + id int64 + sources map[string]*MigrationSource + targets map[string]*MigrationTarget + sourceKeyspace string + targetKeyspace string + tables []string + keepRoutingRules bool + sourceKSSchema *vindexes.KeyspaceSchema + optCells string // cells option passed to MoveTables/Reshard Create + optTabletTypes string // tabletTypes option passed to MoveTables/Reshard Create + externalCluster string + externalTopo *topo.Server + sourceTimeZone string + targetTimeZone string + workflowType binlogdatapb.VReplicationWorkflowType + workflowSubType binlogdatapb.VReplicationWorkflowSubType +} + +func (ts *trafficSwitcher) TopoServer() *topo.Server { return ts.ws.ts } +func (ts *trafficSwitcher) TabletManagerClient() tmclient.TabletManagerClient { return ts.ws.tmc } +func (ts *trafficSwitcher) Logger() logutil.Logger { + if ts.logger == nil { + ts.logger = logutil.NewConsoleLogger() + } + return ts.logger +} +func (ts *trafficSwitcher) VReplicationExec(ctx context.Context, alias *topodatapb.TabletAlias, query string) (*querypb.QueryResult, error) { + return ts.ws.VReplicationExec(ctx, alias, query) +} +func (ts *trafficSwitcher) ExternalTopo() *topo.Server { return ts.externalTopo } +func (ts *trafficSwitcher) MigrationType() binlogdatapb.MigrationType { return ts.migrationType } +func (ts *trafficSwitcher) IsPartialMigration() bool { return ts.isPartialMigration } +func (ts *trafficSwitcher) ReverseWorkflowName() string { return ts.reverseWorkflow } +func (ts *trafficSwitcher) SourceKeyspaceName() string { return ts.sourceKSSchema.Keyspace.Name } +func (ts *trafficSwitcher) SourceKeyspaceSchema() *vindexes.KeyspaceSchema { return ts.sourceKSSchema } +func (ts *trafficSwitcher) Sources() map[string]*MigrationSource { return ts.sources } +func (ts *trafficSwitcher) Tables() []string { return ts.tables } +func (ts *trafficSwitcher) TargetKeyspaceName() string { return ts.targetKeyspace } +func (ts *trafficSwitcher) Targets() map[string]*MigrationTarget { return ts.targets } +func (ts *trafficSwitcher) WorkflowName() string { return ts.workflow } +func (ts *trafficSwitcher) SourceTimeZone() string { return ts.sourceTimeZone } +func (ts *trafficSwitcher) TargetTimeZone() string { return ts.targetTimeZone } + +func (ts *trafficSwitcher) ForAllSources(f func(source *MigrationSource) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, source := range ts.sources { + wg.Add(1) + go func(source *MigrationSource) { + defer wg.Done() + + if err := f(source); err != nil { + allErrors.RecordError(err) + } + }(source) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} + +func (ts *trafficSwitcher) ForAllTargets(f func(source *MigrationTarget) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, target := range ts.targets { + wg.Add(1) + go func(target *MigrationTarget) { + defer wg.Done() + + if err := f(target); err != nil { + allErrors.RecordError(err) + } + }(target) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} + // MigrationTarget contains the metadata for each migration target. type MigrationTarget struct { si *topo.ShardInfo @@ -184,204 +307,916 @@ func (target *MigrationTarget) GetPrimary() *topo.TabletInfo { return target.primary } -// BuildTargets collects MigrationTargets and other metadata (see TargetInfo) -// from a workflow in the target keyspace. -// -// It returns ErrNoStreams if there are no targets found for the workflow. -func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, targetKeyspace string, workflow string) (*TargetInfo, error) { - targetShards, err := ts.GetShardNames(ctx, targetKeyspace) +func (ts *trafficSwitcher) SourceShards() []*topo.ShardInfo { + shards := make([]*topo.ShardInfo, 0, len(ts.Sources())) + for _, source := range ts.Sources() { + shards = append(shards, source.GetShard()) + } + return shards +} + +func (ts *trafficSwitcher) TargetShards() []*topo.ShardInfo { + shards := make([]*topo.ShardInfo, 0, len(ts.Targets())) + for _, target := range ts.Targets() { + shards = append(shards, target.GetShard()) + } + return shards +} + +func (ts *trafficSwitcher) getSourceAndTargetShardsNames() ([]string, []string) { + var sourceShards, targetShards []string + for _, si := range ts.SourceShards() { + sourceShards = append(sourceShards, si.ShardName()) + } + for _, si := range ts.TargetShards() { + targetShards = append(targetShards, si.ShardName()) + } + return sourceShards, targetShards +} + +// isPartialMoveTables returns true if whe workflow is MoveTables, has the same +// number of shards, is not covering the entire shard range, and has one-to-one +// shards in source and target. +func (ts *trafficSwitcher) isPartialMoveTables(sourceShards, targetShards []string) (bool, error) { + if ts.MigrationType() != binlogdatapb.MigrationType_TABLES { + return false, nil + } + + skr, tkr, err := getSourceAndTargetKeyRanges(sourceShards, targetShards) + if err != nil { + return false, err + } + + if key.KeyRangeIsComplete(skr) || key.KeyRangeIsComplete(tkr) || len(sourceShards) != len(targetShards) { + return false, nil + } + + return key.KeyRangeEqual(skr, tkr), nil +} + +// addParticipatingTablesToKeyspace updates the vschema with the new tables that +// were created as part of the Migrate flow. It is called when the Migrate flow +// is Completed. +func (ts *trafficSwitcher) addParticipatingTablesToKeyspace(ctx context.Context, keyspace, tableSpecs string) error { + vschema, err := ts.TopoServer().GetVSchema(ctx, keyspace) if err != nil { - return nil, err - } - - var ( - frozen bool - optCells string - optTabletTypes string - targets = make(map[string]*MigrationTarget, len(targetShards)) - workflowType binlogdatapb.VReplicationWorkflowType - workflowSubType binlogdatapb.VReplicationWorkflowSubType - ) - - // We check all shards in the target keyspace. Not all of them may have a - // stream. For example, if we're splitting -80 to [-40,40-80], only those - // two target shards will have vreplication streams, and the other shards in - // the target keyspace will not. - for _, targetShard := range targetShards { - si, err := ts.GetShard(ctx, targetKeyspace, targetShard) + return err + } + if vschema == nil { + return fmt.Errorf("no vschema found for keyspace %s", keyspace) + } + if vschema.Tables == nil { + vschema.Tables = make(map[string]*vschemapb.Table) + } + if strings.HasPrefix(tableSpecs, "{") { // user defined the vschema snippet, typically for a sharded target + wrap := fmt.Sprintf(`{"tables": %s}`, tableSpecs) + ks := &vschemapb.Keyspace{} + if err := json2.Unmarshal([]byte(wrap), ks); err != nil { + return err + } if err != nil { - return nil, err + return err } + for table, vtab := range ks.Tables { + vschema.Tables[table] = vtab + } + } else { + if vschema.Sharded { + return fmt.Errorf("no sharded vschema was provided, so you will need to update the vschema of the target manually for the moved tables") + } + for _, table := range ts.tables { + vschema.Tables[table] = &vschemapb.Table{} + } + } + return ts.TopoServer().SaveVSchema(ctx, keyspace, vschema) +} + +func (ts *trafficSwitcher) deleteRoutingRules(ctx context.Context) error { + rules, err := topotools.GetRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + log.Errorf("DEBUG: rules: %v", rules) + for _, table := range ts.Tables() { + delete(rules, table) + delete(rules, table+"@replica") + delete(rules, table+"@rdonly") + delete(rules, ts.TargetKeyspaceName()+"."+table) + delete(rules, ts.TargetKeyspaceName()+"."+table+"@replica") + delete(rules, ts.TargetKeyspaceName()+"."+table+"@rdonly") + delete(rules, ts.SourceKeyspaceName()+"."+table) + delete(rules, ts.SourceKeyspaceName()+"."+table+"@replica") + delete(rules, ts.SourceKeyspaceName()+"."+table+"@rdonly") + } + log.Errorf("DEBUG: new rules: %v", rules) + if err := topotools.SaveRoutingRules(ctx, ts.TopoServer(), rules); err != nil { + return err + } + return nil +} + +func (ts *trafficSwitcher) deleteShardRoutingRules(ctx context.Context) error { + if !ts.isPartialMigration { + return nil + } + srr, err := topotools.GetShardRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + for _, si := range ts.TargetShards() { + delete(srr, fmt.Sprintf("%s.%s", ts.targetKeyspace, si.ShardName())) + } + if err := topotools.SaveShardRoutingRules(ctx, ts.TopoServer(), srr); err != nil { + return err + } + return nil +} - if si.PrimaryAlias == nil { - // This can happen if bad inputs are given. - return nil, fmt.Errorf("shard %v/%v doesn't have a primary set", targetKeyspace, targetShard) +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 { + return si.UpdateSourceDeniedTables(ctx, topodatapb.TabletType_PRIMARY, nil, true, ts.Tables()) + }); err != nil { + return err } + rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout) + defer cancel() + _, _, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), source.GetShard(), nil, ts.Logger()) + return err + }) +} + +func (ts *trafficSwitcher) validateWorkflowHasCompleted(ctx context.Context) error { + return doValidateWorkflowHasCompleted(ctx, ts) +} + +func (ts *trafficSwitcher) dropParticipatingTablesFromKeyspace(ctx context.Context, keyspace string) error { + vschema, err := ts.TopoServer().GetVSchema(ctx, keyspace) + if err != nil { + return err + } + // VReplication does NOT create the vschema entries in SHARDED + // TARGET keyspaces -- as we cannot know the proper vindex + // definitions to use -- and we should not delete them either + // (on workflow Cancel) as the user must create them separately + // and they contain information about the vindex definitions, etc. + if vschema.Sharded && keyspace == ts.TargetKeyspaceName() { + return nil + } + for _, tableName := range ts.Tables() { + delete(vschema.Tables, tableName) + } + return ts.TopoServer().SaveVSchema(ctx, keyspace, vschema) +} + +func (ts *trafficSwitcher) removeSourceTables(ctx context.Context, removalType TableRemovalType) error { + err := ts.ForAllSources(func(source *MigrationSource) error { + for _, tableName := range ts.Tables() { + query := fmt.Sprintf("drop table %s.%s", + sqlescape.EscapeID(sqlescape.UnescapeID(source.GetPrimary().DbName())), + sqlescape.EscapeID(sqlescape.UnescapeID(tableName))) + if removalType == DropTable { + ts.Logger().Infof("%s: Dropping table %s.%s\n", + source.GetPrimary().String(), source.GetPrimary().DbName(), tableName) + } else { + renameName := getRenameFileName(tableName) + ts.Logger().Infof("%s: Renaming table %s.%s to %s.%s\n", + source.GetPrimary().String(), source.GetPrimary().DbName(), tableName, source.GetPrimary().DbName(), renameName) + query = fmt.Sprintf("rename table %s.%s TO %s.%s", + sqlescape.EscapeID(sqlescape.UnescapeID(source.GetPrimary().DbName())), + sqlescape.EscapeID(sqlescape.UnescapeID(tableName)), + sqlescape.EscapeID(sqlescape.UnescapeID(source.GetPrimary().DbName())), + sqlescape.EscapeID(sqlescape.UnescapeID(renameName))) + } + _, err := ts.ws.tmc.ExecuteFetchAsDba(ctx, source.GetPrimary().Tablet, false, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(query), + MaxRows: 1, + ReloadSchema: true, + }) + if err != nil { + ts.Logger().Errorf("%s: Error removing table %s: %v", source.GetPrimary().String(), tableName, err) + return err + } + ts.Logger().Infof("%s: Removed table %s.%s\n", source.GetPrimary().String(), source.GetPrimary().DbName(), tableName) - primary, err := ts.GetTablet(ctx, si.PrimaryAlias) - if err != nil { - return nil, err } + return nil + }) + if err != nil { + return err + } + + return ts.dropParticipatingTablesFromKeyspace(ctx, ts.SourceKeyspaceName()) +} - // NB: changing the whitespace of this query breaks tests for now. - // (TODO:@ajm188) extend FakeDBClient to be less whitespace-sensitive on - // expected queries. - query := fmt.Sprintf("select id, source, message, cell, tablet_types, workflow_type, workflow_sub_type, defer_secondary_keys from _vt.vreplication where workflow=%s and db_name=%s", encodeString(workflow), encodeString(primary.DbName())) - p3qr, err := tmc.VReplicationExec(ctx, primary.Tablet, query) +// FIXME: even after dropSourceShards there are still entries in the topo, need to research and fix +func (ts *trafficSwitcher) dropSourceShards(ctx context.Context) error { + return ts.ForAllSources(func(source *MigrationSource) error { + ts.Logger().Infof("Deleting shard %s.%s\n", source.GetShard().Keyspace(), source.GetShard().ShardName()) + err := ts.ws.DeleteShard(ctx, source.GetShard().Keyspace(), source.GetShard().ShardName(), true, false) if err != nil { - return nil, err + ts.Logger().Errorf("Error deleting shard %s: %v", source.GetShard().ShardName(), err) + return err } + ts.Logger().Infof("Deleted shard %s.%s\n", source.GetShard().Keyspace(), source.GetShard().ShardName()) + return nil + }) +} - if len(p3qr.Rows) < 1 { - continue +func (ts *trafficSwitcher) switchShardReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { + var fromShards, toShards []*topo.ShardInfo + if direction == DirectionForward { + fromShards, toShards = ts.SourceShards(), ts.TargetShards() + } else { + fromShards, toShards = ts.TargetShards(), ts.SourceShards() + } + if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), strings.Join(cells, ",")); err != nil { + err2 := vterrors.Wrapf(err, "Before switching shard reads, found SrvKeyspace for %s is corrupt in cell %s", + ts.TargetKeyspaceName(), strings.Join(cells, ",")) + log.Errorf("%w", err2) + return err2 + } + for _, servedType := range servedTypes { + if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), fromShards, cells, servedType, true /* isFrom */, false /* clearSourceShards */); err != nil { + return err } - - target := &MigrationTarget{ - si: si, - primary: primary, - Sources: make(map[int32]*binlogdatapb.BinlogSource), + if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), toShards, cells, servedType, false, false); err != nil { + return err + } + err := ts.TopoServer().MigrateServedType(ctx, ts.SourceKeyspaceName(), toShards, fromShards, servedType, cells) + if err != nil { + return err } + } + if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), strings.Join(cells, ",")); err != nil { + err2 := vterrors.Wrapf(err, "after switching shard reads, found SrvKeyspace for %s is corrupt in cell %s", + ts.TargetKeyspaceName(), strings.Join(cells, ",")) + log.Errorf("%w", err2) + return err2 + } + return nil +} - qr := sqltypes.Proto3ToResult(p3qr) - for _, row := range qr.Named().Rows { - id, err := row["id"].ToInt32() - if err != nil { - return nil, err +func (ts *trafficSwitcher) switchTableReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { + log.Infof("switchTableReads: servedTypes: %+v, direction %t", servedTypes, direction) + rules, err := topotools.GetRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + // We assume that the following rules were setup when the targets were created: + // table -> sourceKeyspace.table + // targetKeyspace.table -> sourceKeyspace.table + // For forward migration, we add tablet type specific rules to redirect traffic to the target. + // For backward, we redirect to source. + for _, servedType := range servedTypes { + tt := strings.ToLower(servedType.String()) + for _, table := range ts.Tables() { + if direction == DirectionForward { + log.Infof("Route direction forward") + } else { + log.Infof("Route direction backwards") } + toTarget := []string{ts.TargetKeyspaceName() + "." + table} + rules[table+"@"+tt] = toTarget + rules[ts.TargetKeyspaceName()+"."+table+"@"+tt] = toTarget + rules[ts.SourceKeyspaceName()+"."+table+"@"+tt] = toTarget + } + } + if err := topotools.SaveRoutingRules(ctx, ts.TopoServer(), rules); err != nil { + return err + } + return ts.TopoServer().RebuildSrvVSchema(ctx, cells) +} - var bls binlogdatapb.BinlogSource - rowBytes, err := row["source"].ToBytes() - if err != nil { - return nil, err - } - if err := prototext.Unmarshal(rowBytes, &bls); err != nil { - return nil, err - } +func (ts *trafficSwitcher) startReverseVReplication(ctx context.Context) error { + return ts.ForAllSources(func(source *MigrationSource) error { + query := fmt.Sprintf("update _vt.vreplication set state='Running', message='' where db_name=%s", encodeString(source.GetPrimary().DbName())) + _, err := ts.VReplicationExec(ctx, source.GetPrimary().Alias, query) + return err + }) +} - if row["message"].ToString() == Frozen { - frozen = true +func (ts *trafficSwitcher) createJournals(ctx context.Context, sourceWorkflows []string) error { + log.Infof("In createJournals for source workflows %+v", sourceWorkflows) + return ts.ForAllSources(func(source *MigrationSource) error { + if source.Journaled { + return nil + } + participants := make([]*binlogdatapb.KeyspaceShard, 0) + participantMap := make(map[string]bool) + journal := &binlogdatapb.Journal{ + Id: ts.id, + MigrationType: ts.MigrationType(), + Tables: ts.Tables(), + LocalPosition: source.Position, + Participants: participants, + SourceWorkflows: sourceWorkflows, + } + for targetShard, target := range ts.Targets() { + for _, tsource := range target.Sources { + participantMap[tsource.Shard] = true } + journal.ShardGtids = append(journal.ShardGtids, &binlogdatapb.ShardGtid{ + Keyspace: ts.TargetKeyspaceName(), + Shard: targetShard, + Gtid: target.Position, + }) + } + shards := make([]string, 0) + for shard := range participantMap { + shards = append(shards, shard) + } + sort.Sort(vreplication.ShardSorter(shards)) + for _, shard := range shards { + journal.Participants = append(journal.Participants, &binlogdatapb.KeyspaceShard{ + Keyspace: source.GetShard().Keyspace(), + Shard: shard, + }) + + } + log.Infof("Creating journal %v", journal) + ts.Logger().Infof("Creating journal: %v", journal) + statement := fmt.Sprintf("insert into _vt.resharding_journal "+ + "(id, db_name, val) "+ + "values (%v, %v, %v)", + ts.id, encodeString(source.GetPrimary().DbName()), encodeString(journal.String())) + if _, err := ts.TabletManagerClient().VReplicationExec(ctx, source.GetPrimary().Tablet, statement); err != nil { + return err + } + return nil + }) +} - target.Sources[id] = &bls - optCells = row["cell"].ToString() - optTabletTypes = row["tablet_types"].ToString() +func (ts *trafficSwitcher) changeShardsAccess(ctx context.Context, keyspace string, shards []*topo.ShardInfo, access accessType) error { + if err := ts.TopoServer().UpdateDisableQueryService(ctx, keyspace, shards, topodatapb.TabletType_PRIMARY, nil, access == disallowWrites /* disable */); err != nil { + return err + } + return ts.ws.refreshPrimaryTablets(ctx, shards) +} - workflowType = getVReplicationWorkflowType(row) - workflowSubType = getVReplicationWorkflowSubType(row) +func (ts *trafficSwitcher) allowTargetWrites(ctx context.Context) error { + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + return ts.allowTableTargetWrites(ctx) + } + return ts.changeShardsAccess(ctx, ts.TargetKeyspaceName(), ts.TargetShards(), allowWrites) +} +func (ts *trafficSwitcher) allowTableTargetWrites(ctx context.Context) error { + return ts.ForAllTargets(func(target *MigrationTarget) error { + if _, err := ts.TopoServer().UpdateShardFields(ctx, ts.TargetKeyspaceName(), target.GetShard().ShardName(), func(si *topo.ShardInfo) error { + return si.UpdateSourceDeniedTables(ctx, topodatapb.TabletType_PRIMARY, nil, true, ts.Tables()) + }); err != nil { + return err } + rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout) + defer cancel() + _, _, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), target.GetShard(), nil, ts.Logger()) + return err + }) +} - targets[targetShard] = target +func (ts *trafficSwitcher) changeRouting(ctx context.Context) error { + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + return ts.changeWriteRoute(ctx) } + return ts.changeShardRouting(ctx) +} - if len(targets) == 0 { - return nil, fmt.Errorf("%w in keyspace %s for %s", ErrNoStreams, targetKeyspace, workflow) +func (ts *trafficSwitcher) changeWriteRoute(ctx context.Context) error { + if ts.isPartialMigration { + srr, err := topotools.GetShardRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + for _, si := range ts.SourceShards() { + delete(srr, fmt.Sprintf("%s.%s", ts.TargetKeyspaceName(), si.ShardName())) + ts.Logger().Infof("Deleted shard routing: %v:%v", ts.TargetKeyspaceName(), si.ShardName()) + srr[fmt.Sprintf("%s.%s", ts.SourceKeyspaceName(), si.ShardName())] = ts.TargetKeyspaceName() + ts.Logger().Infof("Added shard routing: %v:%v", ts.SourceKeyspaceName(), si.ShardName()) + } + if err := topotools.SaveShardRoutingRules(ctx, ts.TopoServer(), srr); err != nil { + return err + } + } else { + rules, err := topotools.GetRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + for _, table := range ts.Tables() { + targetKsTable := fmt.Sprintf("%s.%s", ts.TargetKeyspaceName(), table) + sourceKsTable := fmt.Sprintf("%s.%s", ts.SourceKeyspaceName(), table) + delete(rules, targetKsTable) + ts.Logger().Infof("Deleted routing: %s", targetKsTable) + rules[table] = []string{targetKsTable} + rules[sourceKsTable] = []string{targetKsTable} + ts.Logger().Infof("Added routing: %v %v", table, sourceKsTable) + } + if err := topotools.SaveRoutingRules(ctx, ts.TopoServer(), rules); err != nil { + return err + } } - return &TargetInfo{ - Targets: targets, - Frozen: frozen, - OptCells: optCells, - OptTabletTypes: optTabletTypes, - WorkflowType: workflowType, - WorkflowSubType: workflowSubType, - }, nil + return ts.TopoServer().RebuildSrvVSchema(ctx, nil) +} + +func (ts *trafficSwitcher) changeShardRouting(ctx context.Context) error { + if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), ""); err != nil { + err2 := vterrors.Wrapf(err, "Before changing shard routes, found SrvKeyspace for %s is corrupt", ts.TargetKeyspaceName()) + log.Errorf("%w", err2) + return err2 + } + err := ts.ForAllSources(func(source *MigrationSource) error { + _, err := ts.TopoServer().UpdateShardFields(ctx, ts.SourceKeyspaceName(), source.GetShard().ShardName(), func(si *topo.ShardInfo) error { + si.IsPrimaryServing = false + return nil + }) + return err + }) + if err != nil { + return err + } + err = ts.ForAllTargets(func(target *MigrationTarget) error { + _, err := ts.TopoServer().UpdateShardFields(ctx, ts.TargetKeyspaceName(), target.GetShard().ShardName(), func(si *topo.ShardInfo) error { + si.IsPrimaryServing = true + return nil + }) + return err + }) + if err != nil { + return err + } + err = ts.TopoServer().MigrateServedType(ctx, ts.TargetKeyspaceName(), ts.TargetShards(), ts.SourceShards(), topodatapb.TabletType_PRIMARY, nil) + if err != nil { + return err + } + if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), ""); err != nil { + err2 := vterrors.Wrapf(err, "after changing shard routes, found SrvKeyspace for %s is corrupt", ts.TargetKeyspaceName()) + log.Errorf("%w", err2) + return err2 + } + return nil } -func getVReplicationWorkflowType(row sqltypes.RowNamedValues) binlogdatapb.VReplicationWorkflowType { - i, _ := row["workflow_type"].ToInt32() - return binlogdatapb.VReplicationWorkflowType(i) +func (ts *trafficSwitcher) getReverseVReplicationUpdateQuery(targetCell string, sourceCell string, dbname 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. + if ts.optCells != "" && targetCell != sourceCell && strings.Contains(ts.optCells+",", targetCell+",") && + !strings.Contains(ts.optCells+",", sourceCell+",") { + ts.optCells = strings.Replace(ts.optCells, targetCell, sourceCell, 1) + } + + 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) + return query + } + return "" } -func getVReplicationWorkflowSubType(row sqltypes.RowNamedValues) binlogdatapb.VReplicationWorkflowSubType { - i, _ := row["workflow_sub_type"].ToInt32() - return binlogdatapb.VReplicationWorkflowSubType(i) +func (ts *trafficSwitcher) deleteReverseVReplication(ctx context.Context) error { + return ts.ForAllSources(func(source *MigrationSource) error { + query := fmt.Sprintf(sqlDeleteWorkflow, encodeString(source.GetPrimary().DbName()), encodeString(ts.reverseWorkflow)) + if _, err := ts.TabletManagerClient().VReplicationExec(ctx, source.GetPrimary().Tablet, query); err != nil { + return err + } + ts.ws.deleteWorkflowVDiffData(ctx, source.GetPrimary().Tablet, ts.reverseWorkflow) + ts.ws.optimizeCopyStateTable(source.GetPrimary().Tablet) + return nil + }) } -// CompareShards compares the list of shards in a workflow with the shards in -// that keyspace according to the topo. It returns an error if they do not match. -// -// This function is used to validate MoveTables workflows. -// -// (TODO|@ajm188): This function is temporarily-exported until *wrangler.trafficSwitcher -// has been fully moved over to this package. Once that refactor is finished, -// this function should be unexported. Consequently, YOU SHOULD NOT DEPEND ON -// THIS FUNCTION EXTERNALLY. -func CompareShards(ctx context.Context, keyspace string, shards []*topo.ShardInfo, ts *topo.Server) error { - shardSet := sets.New[string]() - for _, si := range shards { - shardSet.Insert(si.ShardName()) +func (ts *trafficSwitcher) ForAllUIDs(f func(target *MigrationTarget, uid int32) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, target := range ts.Targets() { + for uid := range target.Sources { + wg.Add(1) + go func(target *MigrationTarget, uid int32) { + defer wg.Done() + + if err := f(target, uid); err != nil { + allErrors.RecordError(err) + } + }(target, uid) + } } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} - topoShards, err := ts.GetShardNames(ctx, keyspace) - if err != nil { +func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error { + if err := ts.deleteReverseVReplication(ctx); err != nil { return err } + err := ts.ForAllUIDs(func(target *MigrationTarget, uid int32) error { + bls := target.Sources[uid] + source := ts.Sources()[bls.Shard] + reverseBls := &binlogdatapb.BinlogSource{ + Keyspace: ts.TargetKeyspaceName(), + Shard: target.GetShard().ShardName(), + TabletType: bls.TabletType, + Filter: &binlogdatapb.Filter{}, + OnDdl: bls.OnDdl, + SourceTimeZone: bls.TargetTimeZone, + TargetTimeZone: bls.SourceTimeZone, + } - topoShardSet := sets.New[string](topoShards...) - if !shardSet.Equal(topoShardSet) { - wfExtra := shardSet.Difference(topoShardSet) - topoExtra := topoShardSet.Difference(shardSet) + for _, rule := range bls.Filter.Rules { + if rule.Filter == "exclude" { + reverseBls.Filter.Rules = append(reverseBls.Filter.Rules, rule) + continue + } + var filter string + if strings.HasPrefix(rule.Match, "/") { + if ts.SourceKeyspaceSchema().Keyspace.Sharded { + filter = key.KeyRangeString(source.GetShard().KeyRange) + } + } else { + var inKeyrange string + if ts.SourceKeyspaceSchema().Keyspace.Sharded { + vtable, ok := ts.SourceKeyspaceSchema().Tables[rule.Match] + if !ok { + return fmt.Errorf("table %s not found in vschema1", rule.Match) + } + // TODO(sougou): handle degenerate cases like sequence, etc. + // We currently assume the primary vindex is the best way to filter, which may not be true. + inKeyrange = fmt.Sprintf(" where in_keyrange(%s, '%s.%s', '%s')", sqlparser.String(vtable.ColumnVindexes[0].Columns[0]), + ts.SourceKeyspaceName(), vtable.ColumnVindexes[0].Name, key.KeyRangeString(source.GetShard().KeyRange)) + } + filter = fmt.Sprintf("select * from %s%s", sqlescape.EscapeID(rule.Match), inKeyrange) + } + reverseBls.Filter.Rules = append(reverseBls.Filter.Rules, &binlogdatapb.Rule{ + Match: rule.Match, + Filter: filter, + }) + } + 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, + binlogplayer.CreateVReplicationState(ts.ReverseWorkflowName(), reverseBls, target.Position, + binlogdatapb.VReplicationWorkflowState_Stopped, source.GetPrimary().DbName(), ts.workflowType, ts.workflowSubType)) + if err != nil { + return err + } - var rec concurrency.AllErrorRecorder - if wfExtra.Len() > 0 { - wfExtraSorted := sets.List(wfExtra) - rec.RecordError(fmt.Errorf("switch command shards not in topo: %v", wfExtraSorted)) + // 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()) + if updateQuery != "" { + log.Infof("Updating vreplication stream entry on %s with: %s", source.GetPrimary().Alias, updateQuery) + _, err = ts.VReplicationExec(ctx, source.GetPrimary().Alias, updateQuery) + return err } + return nil + }) + return err +} - if topoExtra.Len() > 0 { - topoExtraSorted := sets.List(topoExtra) - rec.RecordError(fmt.Errorf("topo shards not in switch command: %v", topoExtraSorted)) +func (ts *trafficSwitcher) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error { + ctx, cancel := context.WithTimeout(ctx, filteredReplicationWaitTime) + defer cancel() + // Source writes have been stopped, wait for all streams on targets to catch up. + if err := ts.ForAllUIDs(func(target *MigrationTarget, uid int32) error { + ts.Logger().Infof("Before Catchup: uid: %d, target primary %s, target position %s, shard %s", uid, + target.GetPrimary().AliasString(), target.Position, target.GetShard().String()) + bls := target.Sources[uid] + source := ts.Sources()[bls.Shard] + ts.Logger().Infof("Before Catchup: waiting for keyspace:shard: %v:%v to reach source position %v, uid %d", + ts.TargetKeyspaceName(), target.GetShard().ShardName(), source.Position, uid) + if err := ts.TabletManagerClient().VReplicationWaitForPos(ctx, target.GetPrimary().Tablet, uid, source.Position); err != nil { + return err } + log.Infof("After catchup: target keyspace:shard: %v:%v, source position %v, uid %d", + ts.TargetKeyspaceName(), target.GetShard().ShardName(), source.Position, uid) + ts.Logger().Infof("After catchup: position for keyspace:shard: %v:%v reached, uid %d", + ts.TargetKeyspaceName(), target.GetShard().ShardName(), uid) + if _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, binlogplayer.StopVReplication(uid, "stopped for cutover")); err != nil { + log.Infof("Error marking stopped for cutover on %s, uid %d", target.GetPrimary().AliasString(), uid) + return err + } + return nil + }); err != nil { + return err + } + // all targets have caught up, record their positions for setting up reverse workflows + return ts.ForAllTargets(func(target *MigrationTarget) error { + var err error + target.Position, err = ts.TabletManagerClient().PrimaryPosition(ctx, target.GetPrimary().Tablet) + ts.Logger().Infof("After catchup, position for target primary %s, %v", target.GetPrimary().AliasString(), target.Position) + return err + }) +} - return fmt.Errorf("mismatched shards for keyspace %s: %s", keyspace, strings.Join(rec.ErrorStrings(), "; ")) +func (ts *trafficSwitcher) stopSourceWrites(ctx context.Context) error { + var err error + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + err = ts.changeTableSourceWrites(ctx, disallowWrites) + } else { + err = ts.changeShardsAccess(ctx, ts.SourceKeyspaceName(), ts.SourceShards(), disallowWrites) } + if err != nil { + log.Warningf("Error: %s", err) + return err + } + return ts.ForAllSources(func(source *MigrationSource) error { + var err error + source.Position, err = ts.TabletManagerClient().PrimaryPosition(ctx, source.GetPrimary().Tablet) + log.Infof("Stopped Source Writes. Position for source %v:%v: %v", + ts.SourceKeyspaceName(), source.GetShard().ShardName(), source.Position) + if err != nil { + log.Warningf("Error: %s", err) + } + return err + }) +} +func (ts *trafficSwitcher) changeTableSourceWrites(ctx context.Context, access accessType) error { + return ts.ForAllSources(func(source *MigrationSource) error { + if _, err := ts.TopoServer().UpdateShardFields(ctx, ts.SourceKeyspaceName(), source.GetShard().ShardName(), func(si *topo.ShardInfo) error { + return si.UpdateSourceDeniedTables(ctx, topodatapb.TabletType_PRIMARY, nil, access == allowWrites /* remove */, ts.Tables()) + }); err != nil { + return err + } + rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout) + defer cancel() + isPartial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), source.GetShard(), nil, ts.Logger()) + if isPartial { + err = fmt.Errorf("failed to successfully refresh all tablets in the %s/%s source shard (%v):\n %v", + source.GetShard().Keyspace(), source.GetShard().ShardName(), err, partialDetails) + } + return err + }) +} + +func (ts *trafficSwitcher) cancelMigration(ctx context.Context, sm *StreamMigrator) { + var err error + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + err = ts.changeTableSourceWrites(ctx, allowWrites) + } else { + err = ts.changeShardsAccess(ctx, ts.SourceKeyspaceName(), ts.SourceShards(), allowWrites) + } + if err != nil { + ts.Logger().Errorf("Cancel migration failed:", err) + } + + sm.CancelMigration(ctx) + + err = ts.ForAllTargets(func(target *MigrationTarget) error { + query := fmt.Sprintf("update _vt.vreplication set state='Running', message='' where db_name=%s and workflow=%s", + encodeString(target.GetPrimary().DbName()), encodeString(ts.WorkflowName())) + _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, query) + return err + }) + if err != nil { + ts.Logger().Errorf("Cancel migration failed: could not restart vreplication: %v", err) + } + + err = ts.deleteReverseVReplication(ctx) + if err != nil { + ts.Logger().Errorf("Cancel migration failed: could not delete revers vreplication entries: %v", err) + } +} + +func (ts *trafficSwitcher) freezeTargetVReplication(ctx context.Context) error { + // Mark target streams as frozen before deleting. If SwitchWrites gets + // re-invoked after a freeze, it will skip all the previous steps + err := ts.ForAllTargets(func(target *MigrationTarget) error { + ts.Logger().Infof("Marking target streams frozen for workflow %s db_name %s", ts.WorkflowName(), target.GetPrimary().DbName()) + query := fmt.Sprintf("update _vt.vreplication set message = '%s' where db_name=%s and workflow=%s", Frozen, + encodeString(target.GetPrimary().DbName()), encodeString(ts.WorkflowName())) + _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, query) + return err + }) + if err != nil { + return err + } return nil } -// HashStreams produces a stable hash based on the target keyspace and migration -// targets. -func HashStreams(targetKeyspace string, targets map[string]*MigrationTarget) int64 { - var expanded []string - for shard, target := range targets { - for uid := range target.Sources { - expanded = append(expanded, fmt.Sprintf("%s:%d", shard, uid)) +func (ts *trafficSwitcher) dropTargetVReplicationStreams(ctx context.Context) error { + return ts.ForAllTargets(func(target *MigrationTarget) error { + ts.Logger().Infof("Deleting target streams and related data for workflow %s db_name %s", ts.WorkflowName(), target.GetPrimary().DbName()) + query := fmt.Sprintf(sqlDeleteWorkflow, encodeString(target.GetPrimary().DbName()), encodeString(ts.WorkflowName())) + if _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, query); err != nil { + return err + } + ts.ws.deleteWorkflowVDiffData(ctx, target.GetPrimary().Tablet, ts.WorkflowName()) + ts.ws.optimizeCopyStateTable(target.GetPrimary().Tablet) + return nil + }) +} + +func (ts *trafficSwitcher) dropSourceReverseVReplicationStreams(ctx context.Context) error { + return ts.ForAllSources(func(source *MigrationSource) error { + ts.Logger().Infof("Deleting reverse streams and related data for workflow %s db_name %s", ts.WorkflowName(), source.GetPrimary().DbName()) + query := fmt.Sprintf(sqlDeleteWorkflow, encodeString(source.GetPrimary().DbName()), encodeString(ReverseWorkflowName(ts.WorkflowName()))) + if _, err := ts.TabletManagerClient().VReplicationExec(ctx, source.GetPrimary().Tablet, query); err != nil { + return err } + ts.ws.deleteWorkflowVDiffData(ctx, source.GetPrimary().Tablet, ReverseWorkflowName(ts.WorkflowName())) + ts.ws.optimizeCopyStateTable(source.GetPrimary().Tablet) + return nil + }) +} + +func (ts *trafficSwitcher) removeTargetTables(ctx context.Context) error { + log.Flush() + err := ts.ForAllTargets(func(target *MigrationTarget) error { + log.Infof("ForAllTargets: %+v", target) + for _, tableName := range ts.Tables() { + query := fmt.Sprintf("drop table %s.%s", + sqlescape.EscapeID(sqlescape.UnescapeID(target.GetPrimary().DbName())), + sqlescape.EscapeID(sqlescape.UnescapeID(tableName))) + ts.Logger().Infof("%s: Dropping table %s.%s\n", + target.GetPrimary().String(), target.GetPrimary().DbName(), tableName) + res, err := ts.ws.tmc.ExecuteFetchAsDba(ctx, target.GetPrimary().Tablet, false, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(query), + MaxRows: 1, + ReloadSchema: true, + }) + log.Infof("Removed target table with result: %+v", res) + log.Flush() + if err != nil { + ts.Logger().Errorf("%s: Error removing table %s: %v", + target.GetPrimary().String(), tableName, err) + return err + } + ts.Logger().Infof("%s: Removed table %s.%s\n", + target.GetPrimary().String(), target.GetPrimary().DbName(), tableName) + + } + return nil + }) + if err != nil { + return err } - sort.Strings(expanded) + return ts.dropParticipatingTablesFromKeyspace(ctx, ts.TargetKeyspaceName()) +} + +func (ts *trafficSwitcher) dropTargetShards(ctx context.Context) error { + return ts.ForAllTargets(func(target *MigrationTarget) error { + ts.Logger().Infof("Deleting shard %s.%s\n", target.GetShard().Keyspace(), target.GetShard().ShardName()) + err := ts.ws.DeleteShard(ctx, target.GetShard().Keyspace(), target.GetShard().ShardName(), true, false) + if err != nil { + ts.Logger().Errorf("Error deleting shard %s: %v", target.GetShard().ShardName(), err) + return err + } + ts.Logger().Infof("Deleted shard %s.%s\n", target.GetShard().Keyspace(), target.GetShard().ShardName()) + return nil + }) +} - hasher := fnv.New64() - hasher.Write([]byte(targetKeyspace)) +func (ts *trafficSwitcher) validate(ctx context.Context) error { + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + if ts.isPartialMigration { + return nil + } + sourceTopo := ts.ws.ts + if ts.externalTopo != nil { + sourceTopo = ts.externalTopo + } - for _, s := range expanded { - hasher.Write([]byte(s)) + // All shards must be present. + if err := CompareShards(ctx, ts.SourceKeyspaceName(), ts.SourceShards(), sourceTopo); err != nil { + return err + } + if err := CompareShards(ctx, ts.TargetKeyspaceName(), ts.TargetShards(), ts.ws.ts); err != nil { + return err + } + // Wildcard table names not allowed. + for _, table := range ts.tables { + if strings.HasPrefix(table, "/") { + return fmt.Errorf("cannot migrate streams with wild card table names: %v", table) + } + } } + return nil +} - // Convert to int64 after dropping the highest bit. - return int64(hasher.Sum64() & math.MaxInt64) +// checkJournals returns true if at least one journal has been created. +// If so, it also returns the list of sourceWorkflows that need to be switched. +func (ts *trafficSwitcher) checkJournals(ctx context.Context) (journalsExist bool, sourceWorkflows []string, err error) { + var mu sync.Mutex + + err = ts.ForAllSources(func(source *MigrationSource) error { + mu.Lock() + defer mu.Unlock() + journal, exists, err := ts.ws.CheckReshardingJournalExistsOnTablet(ctx, source.GetPrimary().Tablet, ts.id) + if err != nil { + return err + } + if exists { + if journal.Id != 0 { + sourceWorkflows = journal.SourceWorkflows + } + source.Journaled = true + journalsExist = true + } + return nil + }) + return journalsExist, sourceWorkflows, err } -const reverseSuffix = "_reverse" +// executeLockTablesOnSource executes a LOCK TABLES tb1 READ, tbl2 READ,... statement on each +// source shard's primary tablet using a non-pooled connection as the DBA user. The connection +// is closed when the LOCK TABLES statement returns, so we immediately release the LOCKs. +func (ts *trafficSwitcher) executeLockTablesOnSource(ctx context.Context) error { + ts.Logger().Infof("Locking (and then immediately unlocking) the following tables on source keyspace %v: %v", ts.SourceKeyspaceName(), ts.Tables()) + if len(ts.Tables()) == 0 { + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "no tables found in the source keyspace %v associated with the %s workflow", ts.SourceKeyspaceName(), ts.WorkflowName()) + } -// ReverseWorkflowName returns the "reversed" name of a workflow. For a -// "forward" workflow, this is the workflow name with "_reversed" appended, and -// for a "reversed" workflow, this is the workflow name with the "_reversed" -// suffix removed. -func ReverseWorkflowName(workflow string) string { - if strings.HasSuffix(workflow, reverseSuffix) { - return workflow[:len(workflow)-len(reverseSuffix)] + sb := strings.Builder{} + sb.WriteString("LOCK TABLES ") + for _, tableName := range ts.Tables() { + sb.WriteString(fmt.Sprintf("%s READ,", sqlescape.EscapeID(tableName))) } + // trim extra trailing comma + lockStmt := sb.String()[:sb.Len()-1] - return workflow + reverseSuffix + return ts.ForAllSources(func(source *MigrationSource) error { + primary := source.GetPrimary() + if primary == nil { + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "no primary found for source shard %s", source.GetShard()) + } + tablet := primary.Tablet + _, err := ts.ws.tmc.ExecuteFetchAsDba(ctx, tablet, true, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(lockStmt), + MaxRows: uint64(1), + DisableBinlogs: false, + ReloadSchema: true, + }) + if err != nil { + ts.Logger().Errorf("Error executing %s on source tablet %v: %v", lockStmt, tablet, err) + return err + } + return err + }) } -// Straight copy-paste of encodeString from wrangler/keyspace.go. I want to make -// this public, but it doesn't belong in package workflow. Maybe package sqltypes, -// or maybe package sqlescape? -func encodeString(in string) string { - buf := bytes.NewBuffer(nil) - sqltypes.NewVarChar(in).EncodeSQL(buf) - return buf.String() +func (ts *trafficSwitcher) gatherPositions(ctx context.Context) error { + err := ts.ForAllSources(func(source *MigrationSource) error { + var err error + source.Position, err = ts.ws.tmc.PrimaryPosition(ctx, source.GetPrimary().Tablet) + ts.Logger().Infof("Position for source %v:%v: %v", ts.SourceKeyspaceName(), source.GetShard().ShardName(), source.Position) + return err + }) + if err != nil { + return err + } + return ts.ForAllTargets(func(target *MigrationTarget) error { + var err error + target.Position, err = ts.ws.tmc.PrimaryPosition(ctx, target.GetPrimary().Tablet) + ts.Logger().Infof("Position for target %v:%v: %v", ts.TargetKeyspaceName(), target.GetShard().ShardName(), target.Position) + return err + }) +} + +func (ts *trafficSwitcher) isSequenceParticipating(ctx context.Context) (bool, error) { + vschema, err := ts.TopoServer().GetVSchema(ctx, ts.targetKeyspace) + if err != nil { + return false, err + } + if vschema == nil || vschema.Tables == nil { + return false, nil + } + sequenceFound := false + for _, table := range ts.Tables() { + vs, ok := vschema.Tables[table] + if !ok || vs == nil { + continue + } + if vs.Type == vindexes.TypeSequence { + sequenceFound = true + break + } + } + return sequenceFound, nil +} + +func (ts *trafficSwitcher) mustResetSequences(ctx context.Context) (bool, error) { + switch ts.workflowType { + case binlogdatapb.VReplicationWorkflowType_Migrate, + binlogdatapb.VReplicationWorkflowType_MoveTables: + return ts.isSequenceParticipating(ctx) + default: + return false, nil + } +} + +func (ts *trafficSwitcher) resetSequences(ctx context.Context) error { + var err error + mustReset := false + if mustReset, err = ts.mustResetSequences(ctx); err != nil { + return err + } + if !mustReset { + return nil + } + return ts.ForAllSources(func(source *MigrationSource) error { + ts.Logger().Infof("Resetting sequences for source shard %s.%s on tablet %s", + source.GetShard().Keyspace(), source.GetShard().ShardName(), source.GetPrimary().String()) + return ts.TabletManagerClient().ResetSequences(ctx, source.GetPrimary().Tablet, ts.Tables()) + }) } diff --git a/go/vt/vtctl/workflow/traffic_switcher_test.go b/go/vt/vtctl/workflow/traffic_switcher_test.go index 447e47d7490..c416baa18f9 100644 --- a/go/vt/vtctl/workflow/traffic_switcher_test.go +++ b/go/vt/vtctl/workflow/traffic_switcher_test.go @@ -25,7 +25,7 @@ import ( ) type testTrafficSwitcher struct { - ITrafficSwitcher + trafficSwitcher sourceKeyspaceSchema *vindexes.KeyspaceSchema } diff --git a/go/vt/vtctl/workflow/utils.go b/go/vt/vtctl/workflow/utils.go new file mode 100644 index 00000000000..bb8416414f8 --- /dev/null +++ b/go/vt/vtctl/workflow/utils.go @@ -0,0 +1,768 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package workflow + +import ( + "bytes" + "context" + "fmt" + "hash/fnv" + "math" + "sort" + "strings" + "sync" + + "google.golang.org/protobuf/encoding/prototext" + + "vitess.io/vitess/go/sets" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/discovery" + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/schema" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vttablet/tmclient" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" +) + +const reverseSuffix = "_reverse" + +func getTablesInKeyspace(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, keyspace string) ([]string, error) { + shards, err := ts.GetServingShards(ctx, keyspace) + if err != nil { + return nil, err + } + if len(shards) == 0 { + return nil, fmt.Errorf("keyspace %s has no shards", keyspace) + } + primary := shards[0].PrimaryAlias + if primary == nil { + return nil, fmt.Errorf("shard does not have a primary: %v", shards[0].ShardName()) + } + allTables := []string{"/.*/"} + + ti, err := ts.GetTablet(ctx, primary) + if err != nil { + return nil, err + } + req := &tabletmanagerdatapb.GetSchemaRequest{Tables: allTables} + schema, err := tmc.GetSchema(ctx, ti.Tablet, req) + if err != nil { + return nil, err + } + log.Infof("got table schemas: %+v from source primary %v.", schema, primary) + + var sourceTables []string + for _, td := range schema.TableDefinitions { + sourceTables = append(sourceTables, td.Name) + } + return sourceTables, nil +} + +// validateNewWorkflow ensures that the specified workflow doesn't already exist +// in the keyspace. +func validateNewWorkflow(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, keyspace, workflow string) error { + allshards, err := ts.FindAllShardsInKeyspace(ctx, keyspace) + if err != nil { + return err + } + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, si := range allshards { + if si.PrimaryAlias == nil { + allErrors.RecordError(fmt.Errorf("shard has no primary: %v", si.ShardName())) + continue + } + wg.Add(1) + go func(si *topo.ShardInfo) { + defer wg.Done() + + primary, err := ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + allErrors.RecordError(vterrors.Wrap(err, "validateWorkflowName.GetTablet")) + return + } + validations := []struct { + query string + msg string + }{{ + fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and workflow=%s", encodeString(primary.DbName()), encodeString(workflow)), + fmt.Sprintf("workflow %s already exists in keyspace %s on tablet %d", workflow, keyspace, primary.Alias.Uid), + }, { + fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and message='FROZEN' and workflow_sub_type != %d", encodeString(primary.DbName()), binlogdatapb.VReplicationWorkflowSubType_Partial), + fmt.Sprintf("found previous frozen workflow on tablet %d, please review and delete it first before creating a new workflow", + primary.Alias.Uid), + }} + for _, validation := range validations { + p3qr, err := tmc.VReplicationExec(ctx, primary.Tablet, validation.query) + if err != nil { + allErrors.RecordError(vterrors.Wrap(err, "validateWorkflowName.VReplicationExec")) + return + } + if p3qr != nil && len(p3qr.Rows) != 0 { + allErrors.RecordError(vterrors.Wrap(fmt.Errorf(validation.msg), "validateWorkflowName.VReplicationExec")) + return + } + } + }(si) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} + +// createDefaultShardRoutingRules creates a reverse routing rule for +// each shard in a new partial keyspace migration workflow that does +// not already have an existing routing rule in place. +func createDefaultShardRoutingRules(ctx context.Context, ms *vtctldatapb.MaterializeSettings, ts *topo.Server) error { + srr, err := topotools.GetShardRoutingRules(ctx, ts) + if err != nil { + return err + } + allShards, err := ts.GetServingShards(ctx, ms.SourceKeyspace) + if err != nil { + return err + } + changed := false + for _, si := range allShards { + fromSource := fmt.Sprintf("%s.%s", ms.SourceKeyspace, si.ShardName()) + fromTarget := fmt.Sprintf("%s.%s", ms.TargetKeyspace, si.ShardName()) + if srr[fromSource] == "" && srr[fromTarget] == "" { + srr[fromTarget] = ms.SourceKeyspace + changed = true + log.Infof("Added default shard routing rule from %q to %q", fromTarget, fromSource) + } + } + if changed { + if err := topotools.SaveShardRoutingRules(ctx, ts, srr); err != nil { + return err + } + if err := ts.RebuildSrvVSchema(ctx, nil); err != nil { + return err + } + } + return nil +} + +func stripTableConstraints(ddl string) (string, error) { + ast, err := sqlparser.ParseStrictDDL(ddl) + if err != nil { + return "", err + } + + stripConstraints := func(cursor *sqlparser.Cursor) bool { + switch node := cursor.Node().(type) { + case sqlparser.DDLStatement: + if node.GetTableSpec() != nil { + node.GetTableSpec().Constraints = nil + } + } + return true + } + + noConstraintAST := sqlparser.Rewrite(ast, stripConstraints, nil) + newDDL := sqlparser.String(noConstraintAST) + + return newDDL, nil +} + +func stripTableForeignKeys(ddl string) (string, error) { + ast, err := sqlparser.ParseStrictDDL(ddl) + if err != nil { + return "", err + } + + stripFKConstraints := func(cursor *sqlparser.Cursor) bool { + switch node := cursor.Node().(type) { + case sqlparser.DDLStatement: + if node.GetTableSpec() != nil { + var noFKConstraints []*sqlparser.ConstraintDefinition + for _, constraint := range node.GetTableSpec().Constraints { + if constraint.Details != nil { + if _, ok := constraint.Details.(*sqlparser.ForeignKeyDefinition); !ok { + noFKConstraints = append(noFKConstraints, constraint) + } + } + } + node.GetTableSpec().Constraints = noFKConstraints + } + } + return true + } + + noFKConstraintAST := sqlparser.Rewrite(ast, stripFKConstraints, nil) + newDDL := sqlparser.String(noFKConstraintAST) + return newDDL, nil +} + +func getSourceTableDDLs(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, shards []*topo.ShardInfo) (map[string]string, error) { + sourceDDLs := make(map[string]string) + allTables := []string{"/.*/"} + + sourcePrimary := shards[0].PrimaryAlias + if sourcePrimary == nil { + return nil, fmt.Errorf("shard must have a primary for copying schema: %v", shards[0].ShardName()) + } + + ti, err := ts.GetTablet(ctx, sourcePrimary) + if err != nil { + return nil, err + } + req := &tabletmanagerdatapb.GetSchemaRequest{Tables: allTables} + sourceSchema, err := tmc.GetSchema(ctx, ti.Tablet, req) + if err != nil { + return nil, err + } + + for _, td := range sourceSchema.TableDefinitions { + sourceDDLs[td.Name] = td.Schema + } + return sourceDDLs, nil +} + +func forAllShards(shards []*topo.ShardInfo, f func(*topo.ShardInfo) error) error { + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, target := range shards { + wg.Add(1) + go func(target *topo.ShardInfo) { + defer wg.Done() + + if err := f(target); err != nil { + allErrors.RecordError(err) + } + }(target) + } + wg.Wait() + return allErrors.AggrError(vterrors.Aggregate) +} + +func matchColInSelect(col sqlparser.IdentifierCI, sel *sqlparser.Select) (*sqlparser.ColName, error) { + for _, selExpr := range sel.SelectExprs { + switch selExpr := selExpr.(type) { + case *sqlparser.StarExpr: + return &sqlparser.ColName{Name: col}, nil + case *sqlparser.AliasedExpr: + match := selExpr.As + if match.IsEmpty() { + if colExpr, ok := selExpr.Expr.(*sqlparser.ColName); ok { + match = colExpr.Name + } else { + // Cannot match against a complex expression. + continue + } + } + if match.Equal(col) { + colExpr, ok := selExpr.Expr.(*sqlparser.ColName) + if !ok { + return nil, fmt.Errorf("vindex column cannot be a complex expression: %v", sqlparser.String(selExpr)) + } + return colExpr, nil + } + default: + return nil, fmt.Errorf("unsupported select expression: %v", sqlparser.String(selExpr)) + } + } + return nil, fmt.Errorf("could not find vindex column %v", sqlparser.String(col)) +} + +func shouldInclude(table string, excludes []string) bool { + // We filter out internal tables elsewhere when processing SchemaDefinition + // structures built from the GetSchema database related API calls. In this + // case, however, the table list comes from the user via the -tables flag + // so we need to filter out internal table names here in case a user has + // explicitly specified some. + // This could happen if there's some automated tooling that creates the list of + // tables to explicitly specify. + // But given that this should never be done in practice, we ignore the request. + if schema.IsInternalOperationTableName(table) { + return false + } + for _, t := range excludes { + if t == table { + return false + } + } + return true +} + +// getMigrationID produces a reproducible hash based on the input parameters. +func getMigrationID(targetKeyspace string, shardTablets []string) (int64, error) { + sort.Strings(shardTablets) + hasher := fnv.New64() + hasher.Write([]byte(targetKeyspace)) + for _, str := range shardTablets { + hasher.Write([]byte(str)) + } + // Convert to int64 after dropping the highest bit. + return int64(hasher.Sum64() & math.MaxInt64), nil +} + +// BuildTargets collects MigrationTargets and other metadata (see TargetInfo) +// from a workflow in the target keyspace. +// +// It returns ErrNoStreams if there are no targets found for the workflow. +func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, targetKeyspace string, workflow string) (*TargetInfo, error) { + targetShards, err := ts.GetShardNames(ctx, targetKeyspace) + if err != nil { + return nil, err + } + + var ( + frozen bool + optCells string + optTabletTypes string + targets = make(map[string]*MigrationTarget, len(targetShards)) + workflowType binlogdatapb.VReplicationWorkflowType + workflowSubType binlogdatapb.VReplicationWorkflowSubType + ) + + // We check all shards in the target keyspace. Not all of them may have a + // stream. For example, if we're splitting -80 to [-40,40-80], only those + // two target shards will have vreplication streams, and the other shards in + // the target keyspace will not. + for _, targetShard := range targetShards { + si, err := ts.GetShard(ctx, targetKeyspace, targetShard) + if err != nil { + return nil, err + } + + if si.PrimaryAlias == nil { + // This can happen if bad inputs are given. + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "shard %v/%v doesn't have a primary set", targetKeyspace, targetShard) + } + + primary, err := ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + return nil, err + } + + wf, err := tmc.ReadVReplicationWorkflow(ctx, primary.Tablet, &tabletmanagerdatapb.ReadVReplicationWorkflowRequest{ + Workflow: workflow, + }) + if err != nil { + return nil, err + } + + if len(wf.Streams) < 1 { + continue + } + + target := &MigrationTarget{ + si: si, + primary: primary, + Sources: make(map[int32]*binlogdatapb.BinlogSource), + } + + optCells = wf.Cells + optTabletTypes = topoproto.MakeStringTypeCSV(wf.TabletTypes) + workflowType = wf.WorkflowType + workflowSubType = wf.WorkflowSubType + + for _, stream := range wf.Streams { + if stream.Message == Frozen { + frozen = true + } + target.Sources[stream.Id] = stream.Bls + } + + targets[targetShard] = target + } + + if len(targets) == 0 { + return nil, fmt.Errorf("%w in keyspace %s for %s", ErrNoStreams, targetKeyspace, workflow) + } + + return &TargetInfo{ + Targets: targets, + Frozen: frozen, + OptCells: optCells, + OptTabletTypes: optTabletTypes, + WorkflowType: workflowType, + WorkflowSubType: workflowSubType, + }, nil +} + +func getSourceAndTargetKeyRanges(sourceShards, targetShards []string) (*topodatapb.KeyRange, *topodatapb.KeyRange, error) { + if len(sourceShards) == 0 || len(targetShards) == 0 { + return nil, nil, fmt.Errorf("either source or target shards are missing") + } + + getKeyRange := func(shard string) (*topodatapb.KeyRange, error) { + krs, err := key.ParseShardingSpec(shard) + if err != nil { + return nil, err + } + return krs[0], nil + } + + // Happily string sorting of shards also sorts them in the ascending order of key + // ranges in vitess. + sort.Strings(sourceShards) + sort.Strings(targetShards) + getFullKeyRange := func(shards []string) (*topodatapb.KeyRange, error) { + // Expect sorted shards. + kr1, err := getKeyRange(sourceShards[0]) + if err != nil { + return nil, err + } + kr2, err := getKeyRange(sourceShards[len(sourceShards)-1]) + if err != nil { + return nil, err + } + return &topodatapb.KeyRange{ + Start: kr1.Start, + End: kr2.End, + }, nil + } + + skr, err := getFullKeyRange(sourceShards) + if err != nil { + return nil, nil, err + } + tkr, err := getFullKeyRange(targetShards) + if err != nil { + return nil, nil, err + } + + return skr, tkr, nil +} + +// CompareShards compares the list of shards in a workflow with the shards in +// that keyspace according to the topo. It returns an error if they do not match. +// +// This function is used to validate MoveTables workflows. +// +// (TODO|@ajm188): This function is temporarily-exported until *wrangler.trafficSwitcher +// has been fully moved over to this package. Once that refactor is finished, +// this function should be unexported. Consequently, YOU SHOULD NOT DEPEND ON +// THIS FUNCTION EXTERNALLY. +func CompareShards(ctx context.Context, keyspace string, shards []*topo.ShardInfo, ts *topo.Server) error { + shardSet := sets.New[string]() + for _, si := range shards { + shardSet.Insert(si.ShardName()) + } + + topoShards, err := ts.GetShardNames(ctx, keyspace) + if err != nil { + return err + } + + topoShardSet := sets.New[string](topoShards...) + if !shardSet.Equal(topoShardSet) { + wfExtra := shardSet.Difference(topoShardSet) + topoExtra := topoShardSet.Difference(shardSet) + + var rec concurrency.AllErrorRecorder + if wfExtra.Len() > 0 { + wfExtraSorted := sets.List(wfExtra) + rec.RecordError(fmt.Errorf("switch command shards not in topo: %v", wfExtraSorted)) + } + + if topoExtra.Len() > 0 { + topoExtraSorted := sets.List(topoExtra) + rec.RecordError(fmt.Errorf("topo shards not in switch command: %v", topoExtraSorted)) + } + + return fmt.Errorf("mismatched shards for keyspace %s: %s", keyspace, strings.Join(rec.ErrorStrings(), "; ")) + } + + return nil +} + +// HashStreams produces a stable hash based on the target keyspace and migration +// targets. +func HashStreams(targetKeyspace string, targets map[string]*MigrationTarget) int64 { + var expanded []string + for shard, target := range targets { + for uid := range target.Sources { + expanded = append(expanded, fmt.Sprintf("%s:%d", shard, uid)) + } + } + + sort.Strings(expanded) + + hasher := fnv.New64() + hasher.Write([]byte(targetKeyspace)) + + for _, s := range expanded { + hasher.Write([]byte(s)) + } + + // Convert to int64 after dropping the highest bit. + return int64(hasher.Sum64() & math.MaxInt64) +} + +func doValidateWorkflowHasCompleted(ctx context.Context, ts *trafficSwitcher) error { + wg := sync.WaitGroup{} + rec := concurrency.AllErrorRecorder{} + if ts.MigrationType() == binlogdatapb.MigrationType_SHARDS { + _ = ts.ForAllSources(func(source *MigrationSource) error { + wg.Add(1) + if source.GetShard().IsPrimaryServing { + rec.RecordError(fmt.Errorf(fmt.Sprintf("Shard %s is still serving", source.GetShard().ShardName()))) + } + wg.Done() + return nil + }) + } else { + _ = ts.ForAllTargets(func(target *MigrationTarget) error { + wg.Add(1) + query := fmt.Sprintf("select 1 from _vt.vreplication where db_name='%s' and workflow='%s' and message!='FROZEN'", target.GetPrimary().DbName(), ts.WorkflowName()) + rs, _ := ts.VReplicationExec(ctx, target.GetPrimary().Alias, query) + if len(rs.Rows) > 0 { + rec.RecordError(fmt.Errorf("vreplication streams are not frozen on tablet %d", target.GetPrimary().Alias.Uid)) + } + wg.Done() + return nil + }) + } + wg.Wait() + + if !ts.keepRoutingRules { + // Check if table is routable. + if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + rules, err := topotools.GetRoutingRules(ctx, ts.TopoServer()) + if err != nil { + rec.RecordError(fmt.Errorf("could not get RoutingRules")) + } + for fromTable, toTables := range rules { + for _, toTable := range toTables { + for _, table := range ts.Tables() { + if toTable == fmt.Sprintf("%s.%s", ts.SourceKeyspaceName(), table) { + rec.RecordError(fmt.Errorf("routing still exists from keyspace %s table %s to %s", ts.SourceKeyspaceName(), table, fromTable)) + } + } + } + } + } + } + if rec.HasErrors() { + return fmt.Errorf("%s", strings.Join(rec.ErrorStrings(), "\n")) + } + return nil + +} + +// ReverseWorkflowName returns the "reversed" name of a workflow. For a +// "forward" workflow, this is the workflow name with "_reverse" appended, and +// for a "reversed" workflow, this is the workflow name with the "_reverse" +// suffix removed. +func ReverseWorkflowName(workflow string) string { + if strings.HasSuffix(workflow, reverseSuffix) { + return workflow[:len(workflow)-len(reverseSuffix)] + } + + return workflow + reverseSuffix +} + +// Straight copy-paste of encodeString from wrangler/keyspace.go. I want to make +// this public, but it doesn't belong in package workflow. Maybe package sqltypes, +// or maybe package sqlescape? +func encodeString(in string) string { + buf := bytes.NewBuffer(nil) + sqltypes.NewVarChar(in).EncodeSQL(buf) + return buf.String() +} + +func getRenameFileName(tableName string) string { + return fmt.Sprintf(renameTableTemplate, tableName) +} + +func parseTabletTypes(tabletTypes []topodatapb.TabletType) (hasReplica, hasRdonly, hasPrimary bool, err error) { + for _, tabletType := range tabletTypes { + switch { + case tabletType == topodatapb.TabletType_REPLICA: + hasReplica = true + case tabletType == topodatapb.TabletType_RDONLY: + hasRdonly = true + case tabletType == topodatapb.TabletType_PRIMARY: + hasPrimary = true + default: + return false, false, false, fmt.Errorf("invalid tablet type passed %s", tabletType) + } + } + return hasReplica, hasRdonly, hasPrimary, nil +} + +func areTabletsAvailableToStreamFrom(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest, ts *trafficSwitcher, keyspace string, shards []*topo.ShardInfo) error { + // We use the value from the workflow for the TabletPicker. + tabletTypesStr := ts.optTabletTypes + cells := req.Cells + // If no cells were provided in the command then use the value from the workflow. + if len(cells) == 0 && ts.optCells != "" { + cells = strings.Split(strings.TrimSpace(ts.optCells), ",") + } + + var wg sync.WaitGroup + allErrors := &concurrency.AllErrorRecorder{} + for _, shard := range shards { + wg.Add(1) + go func(cells []string, keyspace string, shard *topo.ShardInfo) { + defer wg.Done() + if cells == nil { + cells = append(cells, shard.PrimaryAlias.Cell) + } + tp, err := discovery.NewTabletPicker(ctx, ts.ws.ts, cells, shard.PrimaryAlias.Cell, keyspace, shard.ShardName(), tabletTypesStr, discovery.TabletPickerOptions{}) + if err != nil { + allErrors.RecordError(err) + return + } + tablets := tp.GetMatchingTablets(ctx) + if len(tablets) == 0 { + allErrors.RecordError(fmt.Errorf("no tablet found to source data in keyspace %s, shard %s", keyspace, shard.ShardName())) + return + } + }(cells, keyspace, shard) + } + + wg.Wait() + if allErrors.HasErrors() { + log.Errorf("%s", allErrors.Error()) + return allErrors.Error() + } + return nil +} + +// LegacyBuildTargets collects MigrationTargets and other metadata (see TargetInfo) +// from a workflow in the target keyspace. It uses VReplicationExec to get the workflow +// details rather than the new TabletManager ReadVReplicationWorkflow RPC. This is +// being used to slowly transition all of the older code, including unit tests, over to +// the new RPC and limit the impact of the new implementation to vtctldclient. You can see +// how the unit tests were being migrated here: https://gist.github.com/mattlord/738c12befe951f8d09304ff7fdc47c46 +// +// New callers should instead use the new BuildTargets function. +// +// It returns ErrNoStreams if there are no targets found for the workflow. +func LegacyBuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManagerClient, targetKeyspace string, workflow string) (*TargetInfo, error) { + targetShards, err := ts.GetShardNames(ctx, targetKeyspace) + if err != nil { + return nil, err + } + + var ( + frozen bool + optCells string + optTabletTypes string + targets = make(map[string]*MigrationTarget, len(targetShards)) + workflowType binlogdatapb.VReplicationWorkflowType + workflowSubType binlogdatapb.VReplicationWorkflowSubType + ) + + getVReplicationWorkflowType := func(row sqltypes.RowNamedValues) binlogdatapb.VReplicationWorkflowType { + i, _ := row["workflow_type"].ToInt32() + return binlogdatapb.VReplicationWorkflowType(i) + } + + getVReplicationWorkflowSubType := func(row sqltypes.RowNamedValues) binlogdatapb.VReplicationWorkflowSubType { + i, _ := row["workflow_sub_type"].ToInt32() + return binlogdatapb.VReplicationWorkflowSubType(i) + } + + // We check all shards in the target keyspace. Not all of them may have a + // stream. For example, if we're splitting -80 to [-40,40-80], only those + // two target shards will have vreplication streams, and the other shards in + // the target keyspace will not. + for _, targetShard := range targetShards { + si, err := ts.GetShard(ctx, targetKeyspace, targetShard) + if err != nil { + return nil, err + } + + if si.PrimaryAlias == nil { + // This can happen if bad inputs are given. + return nil, fmt.Errorf("shard %v/%v doesn't have a primary set", targetKeyspace, targetShard) + } + + primary, err := ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + return nil, err + } + + // NB: changing the whitespace of this query breaks tests for now. + // (TODO:@ajm188) extend FakeDBClient to be less whitespace-sensitive on + // expected queries. + query := fmt.Sprintf("select id, source, message, cell, tablet_types, workflow_type, workflow_sub_type, defer_secondary_keys from _vt.vreplication where workflow=%s and db_name=%s", encodeString(workflow), encodeString(primary.DbName())) + p3qr, err := tmc.VReplicationExec(ctx, primary.Tablet, query) + if err != nil { + return nil, err + } + + if len(p3qr.Rows) < 1 { + continue + } + + target := &MigrationTarget{ + si: si, + primary: primary, + Sources: make(map[int32]*binlogdatapb.BinlogSource), + } + + qr := sqltypes.Proto3ToResult(p3qr) + for _, row := range qr.Named().Rows { + id, err := row["id"].ToInt32() + if err != nil { + return nil, err + } + + var bls binlogdatapb.BinlogSource + rowBytes, err := row["source"].ToBytes() + if err != nil { + return nil, err + } + if err := prototext.Unmarshal(rowBytes, &bls); err != nil { + return nil, err + } + + if row["message"].ToString() == Frozen { + frozen = true + } + + target.Sources[id] = &bls + optCells = row["cell"].ToString() + optTabletTypes = row["tablet_types"].ToString() + + workflowType = getVReplicationWorkflowType(row) + workflowSubType = getVReplicationWorkflowSubType(row) + + } + + targets[targetShard] = target + } + + if len(targets) == 0 { + return nil, fmt.Errorf("%w in keyspace %s for %s", ErrNoStreams, targetKeyspace, workflow) + } + + return &TargetInfo{ + Targets: targets, + Frozen: frozen, + OptCells: optCells, + OptTabletTypes: optTabletTypes, + WorkflowType: workflowType, + WorkflowSubType: workflowSubType, + }, nil +} diff --git a/go/vt/vttablet/faketmclient/fake_client.go b/go/vt/vttablet/faketmclient/fake_client.go index ce4aa55acbb..56bd5e5d29f 100644 --- a/go/vt/vttablet/faketmclient/fake_client.go +++ b/go/vt/vttablet/faketmclient/fake_client.go @@ -62,11 +62,23 @@ type FakeTabletManagerClient struct { tmc tmclient.TabletManagerClient } +func (client *FakeTabletManagerClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + return nil, nil +} + +func (client *FakeTabletManagerClient) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) { + return nil, nil +} + +func (client *FakeTabletManagerClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + return nil, nil +} + func (client *FakeTabletManagerClient) ResetSequences(ctx context.Context, tablet *topodatapb.Tablet, tables []string) error { return nil } -func (client *FakeTabletManagerClient) UpdateVRWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) { +func (client *FakeTabletManagerClient) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { return nil, nil } diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index cee531d4b8f..37c12ef5c1c 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -690,6 +690,49 @@ func (client *Client) GetReplicas(ctx context.Context, tablet *topodatapb.Tablet return response.Addrs, nil } +// +// VReplication related methods +// + +func (client *Client) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.CreateVReplicationWorkflow(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + +func (client *Client) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.DeleteVReplicationWorkflow(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + +func (client *Client) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.ReadVReplicationWorkflow(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + // VReplicationExec is part of the tmclient.TabletManagerClient interface. func (client *Client) VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) { c, closer, err := client.dialer.dial(ctx, tablet) @@ -717,13 +760,13 @@ func (client *Client) VReplicationWaitForPos(ctx context.Context, tablet *topoda return nil } -func (client *Client) UpdateVRWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) { +func (client *Client) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { c, closer, err := client.dialer.dial(ctx, tablet) if err != nil { return nil, err } defer closer.Close() - response, err := c.UpdateVRWorkflow(ctx, request) + response, err := c.UpdateVReplicationWorkflow(ctx, request) if err != nil { return nil, err } diff --git a/go/vt/vttablet/grpctmserver/server.go b/go/vt/vttablet/grpctmserver/server.go index 3be6c17291b..1ec6a8993d7 100644 --- a/go/vt/vttablet/grpctmserver/server.go +++ b/go/vt/vttablet/grpctmserver/server.go @@ -20,11 +20,9 @@ import ( "context" "time" - "vitess.io/vitess/go/vt/callerid" - querypb "vitess.io/vitess/go/vt/proto/query" - "google.golang.org/grpc" + "vitess.io/vitess/go/vt/callerid" "vitess.io/vitess/go/vt/callinfo" "vitess.io/vitess/go/vt/hook" "vitess.io/vitess/go/vt/logutil" @@ -34,6 +32,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletmanager" logutilpb "vitess.io/vitess/go/vt/proto/logutil" + querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" tabletmanagerservicepb "vitess.io/vitess/go/vt/proto/tabletmanagerservice" ) @@ -350,6 +349,31 @@ func (s *server) GetReplicas(ctx context.Context, request *tabletmanagerdatapb.G return response, err } +// +// VReplication related methods +// + +func (s *server) CreateVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (response *tabletmanagerdatapb.CreateVReplicationWorkflowResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "CreateVReplicationWorkflow", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.CreateVReplicationWorkflowResponse{} + return s.tm.CreateVReplicationWorkflow(ctx, request) +} + +func (s *server) DeleteVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (response *tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "DeleteVReplicationWorkflow", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.DeleteVReplicationWorkflowResponse{} + return s.tm.DeleteVReplicationWorkflow(ctx, request) +} + +func (s *server) ReadVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (response *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "ReadVReplicationWorkflow", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{} + return s.tm.ReadVReplicationWorkflow(ctx, request) +} + func (s *server) VReplicationExec(ctx context.Context, request *tabletmanagerdatapb.VReplicationExecRequest) (response *tabletmanagerdatapb.VReplicationExecResponse, err error) { defer s.tm.HandleRPCPanic(ctx, "VReplicationExec", request, response, true /*verbose*/, &err) ctx = callinfo.GRPCCallInfo(ctx) @@ -365,11 +389,11 @@ func (s *server) VReplicationWaitForPos(ctx context.Context, request *tabletmana return &tabletmanagerdatapb.VReplicationWaitForPosResponse{}, err } -func (s *server) UpdateVRWorkflow(ctx context.Context, request *tabletmanagerdatapb.UpdateVRWorkflowRequest) (response *tabletmanagerdatapb.UpdateVRWorkflowResponse, err error) { - defer s.tm.HandleRPCPanic(ctx, "UpdateVRWorkflow", request, response, true /*verbose*/, &err) +func (s *server) UpdateVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (response *tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "UpdateVReplicationWorkflow", request, response, true /*verbose*/, &err) ctx = callinfo.GRPCCallInfo(ctx) - response = &tabletmanagerdatapb.UpdateVRWorkflowResponse{} - return s.tm.UpdateVRWorkflow(ctx, request) + response = &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{} + return s.tm.UpdateVReplicationWorkflow(ctx, request) } func (s *server) VDiff(ctx context.Context, request *tabletmanagerdatapb.VDiffRequest) (response *tabletmanagerdatapb.VDiffResponse, err error) { diff --git a/go/vt/vttablet/onlineddl/executor.go b/go/vt/vttablet/onlineddl/executor.go index 37ff614479f..cbf287ac090 100644 --- a/go/vt/vttablet/onlineddl/executor.go +++ b/go/vt/vttablet/onlineddl/executor.go @@ -3371,7 +3371,7 @@ func (e *Executor) readVReplStream(ctx context.Context, uuid string, okIfMissing timeThrottled: row.AsInt64("time_throttled", 0), componentThrottled: row.AsString("component_throttled", ""), transactionTimestamp: row.AsInt64("transaction_timestamp", 0), - state: row.AsString("state", ""), + state: binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[row.AsString("state", "")]), message: row.AsString("message", ""), rowsCopied: row.AsInt64("rows_copied", 0), bls: &binlogdatapb.BinlogSource{}, @@ -3450,9 +3450,9 @@ func (e *Executor) isVReplMigrationRunning(ctx context.Context, uuid string) (is return false, s, nil } switch s.state { - case binlogplayer.BlpError: + case binlogdatapb.VReplicationWorkflowState_Error: return false, s, nil - case binlogplayer.VReplicationInit, binlogplayer.VReplicationCopying, binlogplayer.BlpRunning: + case binlogdatapb.VReplicationWorkflowState_Init, binlogdatapb.VReplicationWorkflowState_Copying, binlogdatapb.VReplicationWorkflowState_Running: return true, s, nil } if strings.Contains(strings.ToLower(s.message), "error") { diff --git a/go/vt/vttablet/onlineddl/vrepl.go b/go/vt/vttablet/onlineddl/vrepl.go index 1264465d4c3..24711df0f14 100644 --- a/go/vt/vttablet/onlineddl/vrepl.go +++ b/go/vt/vttablet/onlineddl/vrepl.go @@ -35,15 +35,15 @@ import ( "vitess.io/vitess/go/mysql/collations/charset" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" - "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/dbconnpool" - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/onlineddl/vrepl" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) // VReplStream represents a row in _vt.vreplication table @@ -57,7 +57,7 @@ type VReplStream struct { timeThrottled int64 componentThrottled string transactionTimestamp int64 - state string + state binlogdatapb.VReplicationWorkflowState message string rowsCopied int64 bls *binlogdatapb.BinlogSource @@ -76,7 +76,7 @@ func (v *VReplStream) livenessTimeIndicator() int64 { // isRunning() returns true when the workflow is actively running func (v *VReplStream) isRunning() bool { switch v.state { - case binlogplayer.VReplicationInit, binlogplayer.VReplicationCopying, binlogplayer.BlpRunning: + case binlogdatapb.VReplicationWorkflowState_Init, binlogdatapb.VReplicationWorkflowState_Copying, binlogdatapb.VReplicationWorkflowState_Running: return true } return false @@ -85,7 +85,7 @@ func (v *VReplStream) isRunning() bool { // hasError() returns true when the workflow has failed and will not retry func (v *VReplStream) hasError() (isTerminal bool, vreplError error) { switch { - case v.state == binlogplayer.BlpError: + case v.state == binlogdatapb.VReplicationWorkflowState_Error: return true, errors.New(v.message) case strings.Contains(strings.ToLower(v.message), "error"): return false, errors.New(v.message) @@ -589,7 +589,7 @@ func (v *VRepl) analyze(ctx context.Context, conn *dbconnpool.DBConnection) erro // generateInsertStatement generates the INSERT INTO _vt.replication stataement that creates the vreplication workflow func (v *VRepl) generateInsertStatement(ctx context.Context) (string, error) { - ig := vreplication.NewInsertGenerator(binlogplayer.BlpStopped, v.dbName) + ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, v.dbName) ig.AddRow(v.workflow, v.bls, v.pos, "", "in_order:REPLICA,PRIMARY", binlogdatapb.VReplicationWorkflowType_OnlineDDL, binlogdatapb.VReplicationWorkflowSubType_None, false) diff --git a/go/vt/vttablet/tabletmanager/framework_test.go b/go/vt/vttablet/tabletmanager/framework_test.go new file mode 100644 index 00000000000..9d8f1c9a6ee --- /dev/null +++ b/go/vt/vttablet/tabletmanager/framework_test.go @@ -0,0 +1,430 @@ +/* +Copyright 2023 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package tabletmanager + +import ( + "context" + "fmt" + "regexp" + "strings" + "sync" + "testing" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/mysql/fakesqldb" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/grpcclient" + "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/vttablet/queryservice" + "vitess.io/vitess/go/vt/vttablet/tabletconn" + "vitess.io/vitess/go/vt/vttablet/tabletconntest" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" + "vitess.io/vitess/go/vt/vttablet/tmclient" + "vitess.io/vitess/go/vt/vttablet/tmclienttest" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +func init() { + tabletconn.RegisterDialer("grpc", func(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (queryservice.QueryService, error) { + return &tabletconntest.FakeQueryService{}, nil + }) +} + +type testEnv struct { + mu sync.Mutex + ctx context.Context + vrengine *vreplication.Engine + vrdbClient *binlogplayer.MockDBClient + ts *topo.Server + cells []string + tablets map[int]*fakeTabletConn + mysqld *mysqlctl.FakeMysqlDaemon + tmc *fakeTMClient + dbName string + protoName string +} + +func newTestEnv(t *testing.T, keyspace string, shards []string) *testEnv { + tenv := &testEnv{ + ctx: context.Background(), + vrdbClient: binlogplayer.NewMockDBClient(t), + tablets: make(map[int]*fakeTabletConn), + tmc: newFakeTMClient(), + cells: []string{"zone1"}, + dbName: "tmtestdb", + protoName: t.Name(), + } + tenv.mu.Lock() + defer tenv.mu.Unlock() + tenv.ts = memorytopo.NewServer(tenv.cells...) + tenv.tmc.keyspace = keyspace + tenv.tmc.shards = shards + + tabletconn.RegisterDialer(t.Name(), func(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (queryservice.QueryService, error) { + tenv.mu.Lock() + defer tenv.mu.Unlock() + if qs, ok := tenv.tablets[int(tablet.Alias.Uid)]; ok { + return qs, nil + } + return nil, fmt.Errorf("tablet %d not found", tablet.Alias.Uid) + }) + tabletconntest.SetProtocol(fmt.Sprintf("go.vt.vttablet.tabletmanager.framework_test_%s", t.Name()), tenv.protoName) + tmclient.RegisterTabletManagerClientFactory(t.Name(), func() tmclient.TabletManagerClient { + return tenv.tmc + }) + tmclienttest.SetProtocol(fmt.Sprintf("go.vt.vttablet.tabletmanager.framework_test_%s", t.Name()), tenv.protoName) + + dbClientFactory := func() binlogplayer.DBClient { + return tenv.vrdbClient + } + tenv.mysqld = mysqlctl.NewFakeMysqlDaemon(fakesqldb.New(t)) + tenv.vrengine = vreplication.NewTestEngine(tenv.ts, tenv.cells[0], tenv.mysqld, dbClientFactory, dbClientFactory, tenv.dbName, nil) + tenv.vrdbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vreplication where db_name='%s'", tenv.dbName), &sqltypes.Result{}, nil) + tenv.vrengine.Open(tenv.ctx) + require.True(t, tenv.vrengine.IsOpen(), "vreplication engine was not open") + + tenv.tmc.tm = TabletManager{ + VREngine: tenv.vrengine, + DBConfigs: &dbconfigs.DBConfigs{ + DBName: tenv.dbName, + }, + } + + return tenv +} + +func (tenv *testEnv) close() { + tenv.mu.Lock() + defer tenv.mu.Unlock() + tenv.vrengine.Close() + tenv.ts.Close() +} + +//-------------------------------------- +// Tablets + +func (tenv *testEnv) addTablet(id int, keyspace, shard string) *fakeTabletConn { + tenv.mu.Lock() + defer tenv.mu.Unlock() + tablet := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: tenv.cells[0], + Uid: uint32(id), + }, + Keyspace: keyspace, + Shard: shard, + KeyRange: &topodatapb.KeyRange{}, + Type: topodatapb.TabletType_PRIMARY, + PortMap: map[string]int32{ + tenv.protoName: int32(id), + }, + } + if err := tenv.ts.InitTablet(tenv.ctx, tablet, false /* allowPrimaryOverride */, true /* createShardAndKeyspace */, false /* allowUpdate */); err != nil { + panic(err) + } + if _, err := tenv.ts.UpdateShardFields(tenv.ctx, keyspace, shard, func(si *topo.ShardInfo) error { + si.PrimaryAlias = tablet.Alias + si.IsPrimaryServing = true + return nil + }); err != nil { + panic(err) + } + if err := tenv.ts.EnsureVSchema(tenv.ctx, keyspace); err != nil { + panic(err) + } + + tenv.tablets[id] = &fakeTabletConn{tablet: tablet} + return tenv.tablets[id] +} + +func (tenv *testEnv) deleteTablet(tablet *topodatapb.Tablet) { + tenv.mu.Lock() + defer tenv.mu.Unlock() + tenv.ts.DeleteTablet(tenv.ctx, tablet.Alias) + // This is not automatically removed from shard replication, which results in log spam. + topo.DeleteTabletReplicationData(tenv.ctx, tenv.ts, tablet) +} + +// fakeTabletConn implements the TabletConn and QueryService interfaces. +type fakeTabletConn struct { + queryservice.QueryService + tablet *topodatapb.Tablet +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Begin(ctx context.Context, target *querypb.Target, options *querypb.ExecuteOptions) (queryservice.TransactionState, error) { + return queryservice.TransactionState{ + TransactionID: 1, + }, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Commit(ctx context.Context, target *querypb.Target, transactionID int64) (int64, error) { + return 0, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Rollback(ctx context.Context, target *querypb.Target, transactionID int64) (int64, error) { + return 0, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Prepare(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) CommitPrepared(ctx context.Context, target *querypb.Target, dtid string) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) RollbackPrepared(ctx context.Context, target *querypb.Target, dtid string, originalID int64) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) CreateTransaction(ctx context.Context, target *querypb.Target, dtid string, participants []*querypb.Target) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) SetRollback(ctx context.Context, target *querypb.Target, dtid string, transactionID int64) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ConcludeTransaction(ctx context.Context, target *querypb.Target, dtid string) (err error) { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ReadTransaction(ctx context.Context, target *querypb.Target, dtid string) (metadata *querypb.TransactionMetadata, err error) { + return nil, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Execute(ctx context.Context, target *querypb.Target, sql string, bindVariables map[string]*querypb.BindVariable, transactionID, reservedID int64, options *querypb.ExecuteOptions) (*sqltypes.Result, error) { + return nil, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) StreamExecute(ctx context.Context, target *querypb.Target, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, reservedID int64, options *querypb.ExecuteOptions, callback func(*sqltypes.Result) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) BeginExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, reservedID int64, options *querypb.ExecuteOptions) (queryservice.TransactionState, *sqltypes.Result, error) { + return queryservice.TransactionState{ + TransactionID: 1, + }, nil, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) BeginStreamExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, reservedID int64, options *querypb.ExecuteOptions, callback func(*sqltypes.Result) error) (queryservice.TransactionState, error) { + return queryservice.TransactionState{ + TransactionID: 1, + }, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) MessageStream(ctx context.Context, target *querypb.Target, name string, callback func(*sqltypes.Result) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) MessageAck(ctx context.Context, target *querypb.Target, name string, ids []*querypb.Value) (count int64, err error) { + return 0, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) VStream(ctx context.Context, request *binlogdatapb.VStreamRequest, send func([]*binlogdatapb.VEvent) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) VStreamRows(ctx context.Context, request *binlogdatapb.VStreamRowsRequest, send func(*binlogdatapb.VStreamRowsResponse) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) HandlePanic(err *error) { +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ReserveBeginExecute(ctx context.Context, target *querypb.Target, preQueries []string, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions) (queryservice.ReservedTransactionState, *sqltypes.Result, error) { + return queryservice.ReservedTransactionState{ + ReservedID: 1, + }, nil, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ReserveBeginStreamExecute(ctx context.Context, target *querypb.Target, preQueries []string, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions, callback func(*sqltypes.Result) error) (queryservice.ReservedTransactionState, error) { + return queryservice.ReservedTransactionState{ + ReservedID: 1, + }, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ReserveExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, options *querypb.ExecuteOptions) (queryservice.ReservedState, *sqltypes.Result, error) { + return queryservice.ReservedState{ + ReservedID: 1, + }, nil, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) ReserveStreamExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, options *querypb.ExecuteOptions, callback func(*sqltypes.Result) error) (queryservice.ReservedState, error) { + return queryservice.ReservedState{ + ReservedID: 1, + }, nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Release(ctx context.Context, target *querypb.Target, transactionID, reservedID int64) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) GetSchema(ctx context.Context, target *querypb.Target, tableType querypb.SchemaTableType, tableNames []string, callback func(schemaRes *querypb.GetSchemaResponse) error) error { + return nil +} + +// fakeTabletConn implements the QueryService interface. +func (ftc *fakeTabletConn) Close(ctx context.Context) error { + return nil +} + +//---------------------------------------------- +// fakeTMClient + +type fakeTMClient struct { + tmclient.TabletManagerClient + keyspace string + shards []string + tm TabletManager + schema *tabletmanagerdatapb.SchemaDefinition + vreQueries map[int]map[string]*querypb.QueryResult +} + +func newFakeTMClient() *fakeTMClient { + return &fakeTMClient{ + vreQueries: make(map[int]map[string]*querypb.QueryResult), + schema: &tabletmanagerdatapb.SchemaDefinition{}, + } +} + +func (tmc *fakeTMClient) GetSchema(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) { + return tmc.schema, nil +} + +func (tmc *fakeTMClient) SetSchema(schema *tabletmanagerdatapb.SchemaDefinition) { + tmc.schema = schema +} + +// ExecuteFetchAsApp is is needed for the materializer's checkTZConversion function. +func (tmc *fakeTMClient) ExecuteFetchAsApp(ctx context.Context, tablet *topodatapb.Tablet, usePool bool, req *tabletmanagerdatapb.ExecuteFetchAsAppRequest) (*querypb.QueryResult, error) { + return sqltypes.ResultToProto3( + sqltypes.MakeTestResult( + sqltypes.MakeTestFields("convert_tz", "varchar"), + "2023-07-14 09:05:01", + )), nil +} + +func (tmc *fakeTMClient) ExecuteFetchAsDba(ctx context.Context, tablet *topodatapb.Tablet, usePool bool, req *tabletmanagerdatapb.ExecuteFetchAsDbaRequest) (*querypb.QueryResult, error) { + // Reuse VReplicationExec + return tmc.VReplicationExec(ctx, tablet, string(req.Query)) +} + +// setVReplicationExecResults allows you to specify VReplicationExec queries +// and their results. You can specify exact strings or strings prefixed with +// a '/', in which case they will be treated as a valid regexp. +func (tmc *fakeTMClient) setVReplicationExecResults(tablet *topodatapb.Tablet, query string, result *sqltypes.Result) { + queries, ok := tmc.vreQueries[int(tablet.Alias.Uid)] + if !ok { + queries = make(map[string]*querypb.QueryResult) + tmc.vreQueries[int(tablet.Alias.Uid)] = queries + } + queries[query] = sqltypes.ResultToProto3(result) +} + +func (tmc *fakeTMClient) VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) { + if result, ok := tmc.vreQueries[int(tablet.Alias.Uid)][query]; ok { + return result, nil + } + for qry, res := range tmc.vreQueries[int(tablet.Alias.Uid)] { + if strings.HasPrefix(qry, "/") { + re := regexp.MustCompile(qry) + if re.MatchString(qry) { + return res, nil + } + } + } + return nil, fmt.Errorf("query %q not found for tablet %d", query, tablet.Alias.Uid) +} + +func (tmc *fakeTMClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + return tmc.tm.CreateVReplicationWorkflow(ctx, req) +} + +func (tmc *fakeTMClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + resp := &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ + Workflow: req.Workflow, + Streams: make([]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream, len(tmc.shards)), + } + for i, shard := range tmc.shards { + resp.Streams[i] = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + Id: int32(i + 1), + Bls: &binlogdatapb.BinlogSource{ + Keyspace: tmc.keyspace, + Shard: shard, + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: ".*", + }, + }, + }, + }, + } + } + + return resp, nil +} diff --git a/go/vt/vttablet/tabletmanager/rpc_agent.go b/go/vt/vttablet/tabletmanager/rpc_agent.go index 2482b84a256..5011f59db7c 100644 --- a/go/vt/vttablet/tabletmanager/rpc_agent.go +++ b/go/vt/vttablet/tabletmanager/rpc_agent.go @@ -99,9 +99,12 @@ type RPCTM interface { WaitForPosition(ctx context.Context, pos string) error // VReplication API + CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) + DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) + ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) VReplicationWaitForPos(ctx context.Context, id int32, pos string) error - UpdateVRWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) + UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) // VDiff API VDiff(ctx context.Context, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go index c6f2e8c5f3c..771cbf33f32 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go @@ -24,41 +24,211 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/discovery" + "vitess.io/vitess/go/vt/proto/vttime" + "vitess.io/vitess/go/vt/sidecardb" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vtctl/workflow" + "vitess.io/vitess/go/vt/vterrors" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" - "vitess.io/vitess/go/vt/sidecardb" - "vitess.io/vitess/go/vt/sqlparser" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + topoprotopb "vitess.io/vitess/go/vt/topo/topoproto" ) 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)" + // 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" + // 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. - sqlSelectVRWorkflowConfig = "select id, source, cell, tablet_types from %s.vreplication where workflow = %a" + sqlSelectVReplicationWorkflowConfig = "select id, source, cell, tablet_types, state, message from %s.vreplication where workflow = %a" // Update the configuration values for a workflow's vreplication stream. - sqlUpdateVRWorkflowConfig = "update %s.vreplication set source = %a, cell = %a, tablet_types = %a where id = %a" + sqlUpdateVReplicationWorkflowConfig = "update %s.vreplication set state = %a, source = %a, cell = %a, tablet_types = %a where id = %a" ) -// VReplicationExec executes a vreplication command. -func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) { - // Replace any provided sidecar databsae qualifiers with the correct one. - uq, err := sqlparser.ReplaceTableQualifiers(query, sidecardb.DefaultName, sidecardb.GetName()) +func (tm *TabletManager) CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + if req == nil || len(req.BinlogSource) == 0 { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid request, no binlog source specified") + } + res := &sqltypes.Result{} + for _, bls := range req.BinlogSource { + source, err := prototext.Marshal(bls) + if err != nil { + return nil, err + } + // Use the local cell if none are specified. + if len(req.Cells) == 0 || strings.TrimSpace(req.Cells[0]) == "" { + req.Cells = append(req.Cells, tm.Tablet().Alias.Cell) + } + wfState := binlogdatapb.VReplicationWorkflowState_Stopped.String() + tabletTypesStr := topoprotopb.MakeStringTypeCSV(req.TabletTypes) + if req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { + tabletTypesStr = discovery.InOrderHint + tabletTypesStr + } + bindVars := map[string]*querypb.BindVariable{ + "workflow": sqltypes.StringBindVariable(req.Workflow), + "source": sqltypes.StringBindVariable(string(source)), + "cells": sqltypes.StringBindVariable(strings.Join(req.Cells, ",")), + "tabletTypes": sqltypes.StringBindVariable(tabletTypesStr), + "state": sqltypes.StringBindVariable(wfState), + "dbname": sqltypes.StringBindVariable(tm.DBConfigs.DBName), + "workflowType": sqltypes.Int64BindVariable(int64(req.WorkflowType)), + "workflowSubType": sqltypes.Int64BindVariable(int64(req.WorkflowSubType)), + "deferSecondaryKeys": sqltypes.BoolBindVariable(req.DeferSecondaryKeys), + } + parsed := sqlparser.BuildParsedQuery(sqlCreateVReplicationWorkflow, sidecardb.GetIdentifier(), + ":workflow", ":source", ":cells", ":tabletTypes", ":state", ":dbname", ":workflowType", ":workflowSubType", ":deferSecondaryKeys", + ) + stmt, err := parsed.GenerateQuery(bindVars, nil) + if err != nil { + return nil, err + } + streamres, err := tm.VREngine.Exec(stmt) + + if err != nil { + return nil, err + } + res.RowsAffected += streamres.RowsAffected + } + return &tabletmanagerdatapb.CreateVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil +} + +func (tm *TabletManager) DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) { + if req == nil || req.Workflow == "" { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid request, no workflow provided") + } + res := &sqltypes.Result{} + bindVars := map[string]*querypb.BindVariable{ + "wf": sqltypes.StringBindVariable(req.Workflow), + "db": sqltypes.StringBindVariable(tm.DBConfigs.DBName), + } + parsed := sqlparser.BuildParsedQuery(sqlDeleteVReplicationWorkflow, sidecardb.GetIdentifier(), ":wf", ":db") + stmt, err := parsed.GenerateQuery(bindVars, nil) if err != nil { return nil, err } - qr, err := tm.VREngine.ExecWithDBA(uq) + streamres, err := tm.VREngine.Exec(stmt) + if err != nil { return nil, err } - return sqltypes.ResultToProto3(qr), nil + res.RowsAffected += streamres.RowsAffected + + return &tabletmanagerdatapb.DeleteVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil } -// VReplicationWaitForPos waits for the specified position. -func (tm *TabletManager) VReplicationWaitForPos(ctx context.Context, id int32, pos string) error { - return tm.VREngine.WaitForPos(ctx, id, pos) +func (tm *TabletManager) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + if req == nil || req.Workflow == "" { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid request, no workflow provided") + } + bindVars := map[string]*querypb.BindVariable{ + "wf": sqltypes.StringBindVariable(req.Workflow), + "db": sqltypes.StringBindVariable(tm.DBConfigs.DBName), + } + parsed := sqlparser.BuildParsedQuery(sqlReadVReplicationWorkflow, sidecardb.GetIdentifier(), ":wf", ":db") + stmt, err := parsed.GenerateQuery(bindVars, nil) + if err != nil { + return nil, err + } + res, err := tm.VREngine.Exec(stmt) + if err != nil { + return nil, err + } + if res == nil || len(res.Rows) == 0 { + return nil, nil + } + rows := res.Named().Rows + resp := &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{Workflow: req.Workflow} + streams := make([]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream, len(rows)) + + // First the things that are common to all streams. + resp.Cells = rows[0]["cell"].ToString() + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(rows[0]["tablet_types"].ToString()) + if err != nil { + return nil, vterrors.Wrap(err, "error parsing the tablet_types field from vreplication table record") + } + resp.TabletTypes = tabletTypes + resp.TabletSelectionPreference = tabletmanagerdatapb.TabletSelectionPreference_ANY + if inorder { + resp.TabletSelectionPreference = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + resp.DbName = rows[0]["db_name"].ToString() + resp.Tags = rows[0]["tags"].ToString() + wft, err := rows[0]["workflow_type"].ToInt32() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing workflow_type field from vreplication table record") + } + resp.WorkflowType = binlogdatapb.VReplicationWorkflowType(wft) + wfst, err := rows[0]["workflow_sub_type"].ToInt32() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing workflow_sub_type field from vreplication table record") + } + resp.WorkflowSubType = binlogdatapb.VReplicationWorkflowSubType(wfst) + resp.DeferSecondaryKeys = rows[0]["defer_secondary_keys"].ToString() == "1" + + // Now the individual streams (there can be more than 1 with shard merges). + for i, row := range rows { + streams[i] = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{} + if streams[i].Id, err = row["id"].ToInt32(); err != nil { + return nil, vterrors.Wrap(err, "error parsing id field from vreplication table record") + } + srcBytes, err := row["source"].ToBytes() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing binlog_source field from vreplication table record") + } + blspb := &binlogdatapb.BinlogSource{} + err = prototext.Unmarshal(srcBytes, blspb) + if err != nil { + return nil, vterrors.Wrap(err, "error unmarshaling binlog_source field from vreplication table record") + } + streams[i].Bls = blspb + streams[i].Pos = row["pos"].ToString() + streams[i].StopPos = row["stop_pos"].ToString() + if streams[i].MaxTps, err = row["max_tps"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing max_tps field from vreplication table record") + } + if streams[i].MaxReplicationLag, err = row["max_replication_lag"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing max_replication_lag field from vreplication table record") + } + timeUpdated, err := row["time_updated"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_updated field from vreplication table record") + } + streams[i].TimeUpdated = &vttime.Time{Seconds: timeUpdated} + txTimestamp, err := row["transaction_timestamp"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing transaction_timestamp field from vreplication table record") + } + streams[i].TransactionTimestamp = &vttime.Time{Seconds: txTimestamp} + streams[i].State = binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[row["state"].ToString()]) + streams[i].Message = row["message"].ToString() + if streams[i].RowsCopied, err = row["rows_copied"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing rows_copied field from vreplication table record") + } + timeHeartbeat, err := row["time_heartbeat"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_heartbeat field from vreplication table record") + } + streams[i].TimeHeartbeat = &vttime.Time{Seconds: timeHeartbeat} + timeThrottled, err := row["time_throttled"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_throttled field from vreplication table record") + } + streams[i].TimeThrottled = &vttime.Time{Seconds: timeThrottled} + streams[i].ComponentThrottled = row["component_throttled"].ToString() + } + resp.Streams = streams + + return resp, nil } -// UpdateVRWorkflow updates the sidecar databases's vreplication +// UpdateVReplicationWorkflow updates the sidecar databases's vreplication // record for this tablet's vreplication workflow stream(s). If there // is no stream for the given workflow on the tablet then a nil result // is returned as this is expected e.g. on source tablets of a @@ -67,11 +237,11 @@ func (tm *TabletManager) VReplicationWaitForPos(ctx context.Context, id int32, p // Note: the VReplication engine creates a new controller for the // workflow stream when the record is updated, so we also in effect // restart the workflow stream via the update. -func (tm *TabletManager) UpdateVRWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) { +func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { bindVars := map[string]*querypb.BindVariable{ "wf": sqltypes.StringBindVariable(req.Workflow), } - parsed := sqlparser.BuildParsedQuery(sqlSelectVRWorkflowConfig, sidecardb.GetIdentifier(), ":wf") + parsed := sqlparser.BuildParsedQuery(sqlSelectVReplicationWorkflowConfig, sidecardb.GetIdentifier(), ":wf") stmt, err := parsed.GenerateQuery(bindVars, nil) if err != nil { return nil, err @@ -85,15 +255,24 @@ func (tm *TabletManager) UpdateVRWorkflow(ctx context.Context, req *tabletmanage // expected e.g. on source tablets for Reshard // workflows. If callers want to treat this // scenario as an error they can. - return &tabletmanagerdatapb.UpdateVRWorkflowResponse{Result: nil}, nil + return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{Result: nil}, nil } row := res.Named().Row() id := row.AsInt64("id", 0) cells := strings.Split(row.AsString("cell", ""), ",") - tabletTypes := strings.Split(row.AsString("tablet_types", ""), ",") + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(row.AsString("tablet_types", "")) + if err != nil { + return nil, err + } bls := &binlogdatapb.BinlogSource{} source := row.AsBytes("source", []byte{}) + state := row.AsString("state", "") + message := row.AsString("message", "") + if req.State == binlogdatapb.VReplicationWorkflowState_Running && strings.ToUpper(message) == workflow.Frozen { + return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{Result: nil}, + vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "cannot start a workflow when it is frozen") + } // For the string based values, we use NULL to differentiate // from an empty string. The NULL value indicates that we // should keep the existing value. @@ -103,6 +282,11 @@ func (tm *TabletManager) UpdateVRWorkflow(ctx context.Context, req *tabletmanage if !textutil.ValueIsSimulatedNull(req.TabletTypes) { tabletTypes = req.TabletTypes } + tabletTypesStr := topoproto.MakeStringTypeCSV(tabletTypes) + if inorder && req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN || + req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { + tabletTypesStr = discovery.InOrderHint + tabletTypesStr + } if err = prototext.Unmarshal(source, bls); err != nil { return nil, err } @@ -115,13 +299,17 @@ func (tm *TabletManager) UpdateVRWorkflow(ctx context.Context, req *tabletmanage if err != nil { return nil, err } + if !textutil.ValueIsSimulatedNull(req.State) { + state = binlogdatapb.VReplicationWorkflowState_name[int32(req.State)] + } bindVars = map[string]*querypb.BindVariable{ + "st": sqltypes.StringBindVariable(state), "sc": sqltypes.StringBindVariable(string(source)), "cl": sqltypes.StringBindVariable(strings.Join(cells, ",")), - "tt": sqltypes.StringBindVariable(strings.Join(tabletTypes, ",")), + "tt": sqltypes.StringBindVariable(tabletTypesStr), "id": sqltypes.Int64BindVariable(id), } - parsed = sqlparser.BuildParsedQuery(sqlUpdateVRWorkflowConfig, sidecardb.GetIdentifier(), ":sc", ":cl", ":tt", ":id") + parsed = sqlparser.BuildParsedQuery(sqlUpdateVReplicationWorkflowConfig, sidecardb.GetIdentifier(), ":st", ":sc", ":cl", ":tt", ":id") stmt, err = parsed.GenerateQuery(bindVars, nil) if err != nil { return nil, err @@ -131,5 +319,24 @@ func (tm *TabletManager) UpdateVRWorkflow(ctx context.Context, req *tabletmanage if err != nil { return nil, err } - return &tabletmanagerdatapb.UpdateVRWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil + return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil +} + +// VReplicationExec executes a vreplication command. +func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) { + // Replace any provided sidecar databsae qualifiers with the correct one. + uq, err := sqlparser.ReplaceTableQualifiers(query, sidecardb.DefaultName, sidecardb.GetName()) + if err != nil { + return nil, err + } + qr, err := tm.VREngine.ExecWithDBA(uq) + if err != nil { + return nil, err + } + return sqltypes.ResultToProto3(qr), nil +} + +// VReplicationWaitForPos waits for the specified position. +func (tm *TabletManager) VReplicationWaitForPos(ctx context.Context, id int32, pos string) error { + return tm.VREngine.WaitForPos(ctx, id, pos) } diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go index 343f6de73b7..05708f924f0 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go @@ -19,61 +19,379 @@ package tabletmanager import ( "context" "fmt" + "runtime/debug" "testing" "github.com/stretchr/testify/require" - "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/mysql/fakesqldb" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" - "vitess.io/vitess/go/vt/binlog/binlogplayer" - "vitess.io/vitess/go/vt/dbconfigs" - "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/sidecardb" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtctl/workflow" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" - "vitess.io/vitess/go/vt/sidecardb" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/topo/memorytopo" - "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" - "vitess.io/vitess/go/vt/vttablet/tabletservermock" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + "vitess.io/vitess/go/vt/proto/vttime" +) + +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)" + getWorkflow = "select id from _vt.vreplication where db_name='vt_%s' and workflow='%s'" + checkForWorkflow = "select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'" + checkForFrozenWorkflow = "select 1 from _vt.vreplication where db_name='vt_%s' and message='FROZEN' and workflow_sub_type != 1" + checkForJournal = "/select val from _vt.resharding_journal where id=" + getWorkflowStatus = "select id, workflow, source, pos, stop_pos, max_replication_lag, state, db_name, time_updated, transaction_timestamp, message, tags, workflow_type, workflow_sub_type from _vt.vreplication where workflow = '%s' and db_name = 'vt_%s'" + 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=1" + getCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" + getNumCopyStateTable = "select count(distinct table_name) from _vt.copy_state where vrepl_id=1" + getLatestCopyState = "select table_name, lastpk from _vt.copy_state where vrepl_id = 1 and id in (select max(id) from _vt.copy_state where vrepl_id = 1 group by vrepl_id, table_name)" + getAutoIncrementStep = "select @@session.auto_increment_increment" + setSessionTZ = "set @@session.time_zone = '+00:00'" + setNames = "set names 'binary'" + setSQLMode = "set @@session.sql_mode = CONCAT(@@session.sql_mode, ',NO_AUTO_VALUE_ON_ZERO')" + setPermissiveSQLMode = "SET @@session.sql_mode='NO_AUTO_VALUE_ON_ZERO'" + setStrictSQLMode = "SET @@session.sql_mode='ONLY_FULL_GROUP_BY,NO_AUTO_VALUE_ON_ZERO,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'" + getSQLMode = "SELECT @@session.sql_mode AS sql_mode" + getFKChecks = "select @@foreign_key_checks;" + disableFKChecks = "set foreign_key_checks=1;" + sqlMode = "ONLY_FULL_GROUP_BY,NO_AUTO_VALUE_ON_ZERO,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION" + getBinlogRowImage = "select @@binlog_row_image" + insertStreamsCreatedLog = "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(1, 'Stream Created', '', '%s'" + getVReplicationRecord = "select * from _vt.vreplication where id = 1" + startWorkflow = "update _vt.vreplication set state='Running' where db_name='vt_%s' and workflow='%s'" + + position = "MySQL56/9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97" +) + +var ( + errShortCircuit = fmt.Errorf("short circuiting test") + defaultSchema = &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{ + { + Name: "t1", + Columns: []string{"c1", "c2"}, + PrimaryKeyColumns: []string{"c1"}, + Fields: sqltypes.MakeTestFields("c1|c2", "int64|int64"), + }, + }, + } ) -func TestUpdateVRWorkflow(t *testing.T) { +// TestCreateVReplicationWorkflow tests the query generated +// from a VtctldServer MoveTablesCreate request to ensure +// that the VReplication stream(s) are created correctly. +func TestCreateVReplicationWorkflow(t *testing.T) { + ctx := context.Background() + sourceKs := "sourceks" + sourceTabletUID := 200 + targetKs := "targetks" + targetTabletUID := 300 + shard := "0" + wf := "testwf" + defaultSchema := &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{ + { + Name: "t1", + Columns: []string{"c1", "c2"}, + PrimaryKeyColumns: []string{"c1"}, + Fields: sqltypes.MakeTestFields("c1|c2", "int64|int64"), + }, + }, + } + tenv := newTestEnv(t, targetKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(tenv.ts, tenv.tmc) + + tests := []struct { + name string + req *vtctldatapb.MoveTablesCreateRequest + schema *tabletmanagerdatapb.SchemaDefinition + query string + }{ + { + name: "defaults", + req: &vtctldatapb.MoveTablesCreateRequest{ + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + Workflow: wf, + 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)`, + insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), + }, + { + name: "all values", + schema: &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{ + { + Name: "t1", + Columns: []string{"c1", "c2"}, + PrimaryKeyColumns: []string{"c1"}, + Fields: sqltypes.MakeTestFields("c1|c2", "int64|int64"), + }, + { + Name: "wut", + Columns: []string{"c1"}, + PrimaryKeyColumns: []string{"c1"}, + Fields: sqltypes.MakeTestFields("c1", "int64"), + }, + }, + }, + req: &vtctldatapb.MoveTablesCreateRequest{ + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + Workflow: wf, + Cells: tenv.cells, + IncludeTables: []string{defaultSchema.TableDefinitions[0].Name}, + ExcludeTables: []string{"wut"}, + SourceTimeZone: "EDT", + OnDdl: binlogdatapb.OnDDLAction_EXEC.String(), + StopAfterCopy: true, + DropForeignKeys: true, + 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)`, + insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), + }, + } + + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'", + targetKs, wf), &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and message='FROZEN' and workflow_sub_type != 1", + targetKs), &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, "select val from _vt.resharding_journal where id=7224776740563431192", &sqltypes.Result{}) + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // This is needed because MockDBClient uses t.Fatal() + // which doesn't play well with subtests. + defer func() { + if err := recover(); err != nil { + t.Errorf("Recovered from panic: %v; Stack: %s", err, string(debug.Stack())) + } + }() + + require.NotNil(t, tt.req, "No MoveTablesCreate request provided") + require.NotEmpty(t, tt.query, "No expected query provided") + + if tt.schema == nil { + tt.schema = defaultSchema + } + tenv.tmc.SetSchema(tt.schema) + + tenv.vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + tenv.vrdbClient.ExpectRequest(tt.query, nil, errShortCircuit) + _, err := ws.MoveTablesCreate(ctx, tt.req) + tenv.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) + }) + } +} + +// TestMoveTables tests the query generated from a VtctldServer +// MoveTablesCreate request to ensure that the VReplication +// stream(s) are created correctly. Followed by ensuring that +// SwitchTraffic and ReverseTraffic work as expected. +func TestMoveTables(t *testing.T) { + ctx := context.Background() + sourceKs := "sourceks" + sourceTabletUID := 200 + targetKs := "targetks" + targetTabletUID := 300 + shard := "0" + wf := "testwf" + + tenv := newTestEnv(t, targetKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(tenv.ts, tenv.tmc) + + tenv.mysqld.Schema = defaultSchema + tenv.mysqld.Schema.DatabaseSchema = tenv.dbName + tenv.mysqld.FetchSuperQueryMap = make(map[string]*sqltypes.Result) + tenv.mysqld.FetchSuperQueryMap[`select character_set_name, collation_name, column_name, data_type, column_type, extra from information_schema.columns where .*`] = sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "character_set_name|collation_name|column_name|data_type|column_type|extra", + "varchar|varchar|varchar|varchar|varchar|varchar", + ), + "NULL|NULL|c1|bigint|bigint|", + "NULL|NULL|c2|bigint|bigint|", + ) + + req := &vtctldatapb.MoveTablesCreateRequest{ + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + Workflow: wf, + Cells: tenv.cells, + AllTables: true, + AutoStart: true, + } + insert := 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) + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}}", sourceKs, shard) + + tenv.tmc.SetSchema(defaultSchema) + + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(checkForWorkflow, targetKs, wf), &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(checkForFrozenWorkflow, targetKs), &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(getWorkflow, targetKs, wf), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id", + "int64", + ), + "1", + ), + ) + tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, checkForJournal, &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, getCopyState, &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(getWorkflowStatus, wf, targetKs), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|workflow|source|pos|stop_pos|max_replication_log|state|db_name|time_updated|transaction_timestamp|message|tags|workflow_type|workflow_sub_type", + "int64|varchar|blob|varchar|varchar|int64|varchar|varchar|int64|int64|varchar|varchar|int64|int64", + ), + fmt.Sprintf("1|%s|%s|%s|NULL|0|running|vt_%s|1686577659|0|||1|0", wf, bls, position, targetKs), + ), + ) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, getLatestCopyState, &sqltypes.Result{}) + + tenv.vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: 1}, nil) + tenv.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(getVReplicationRecord, + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("1|%s", bls), + ), nil) + tenv.vrdbClient.ExpectRequest(`update _vt.vreplication set message='Picked source tablet: cell:\"zone1\" uid:200' where id=1`, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(setSessionTZ, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(setNames, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(setSQLMode, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(getSQLMode, sqltypes.MakeTestResult( + sqltypes.MakeTestFields("sql_mode", "varchar"), + sqlMode, + ), nil) + tenv.vrdbClient.ExpectRequest(getWorkflowState, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", + "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", + ), + fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), + ), nil) + tenv.vrdbClient.ExpectRequest(getNumCopyStateTable, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "count(distinct table_name)", + "int64", + ), + "1", + ), nil) + tenv.vrdbClient.ExpectRequest(setPermissiveSQLMode, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(getFKChecks, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "@@foreign_key_checks", + "int64", + ), + "1", + ), nil) + tenv.vrdbClient.ExpectRequest(getWorkflowState, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", + "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", + ), + fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), + ), nil) + tenv.vrdbClient.ExpectRequest(getNumCopyStateTable, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "count(distinct table_name)", + "int64", + ), + "1", + ), nil) + tenv.vrdbClient.ExpectRequest(getBinlogRowImage, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "@@binlog_row_image", + "varchar", + ), + "FULL", + ), nil) + tenv.vrdbClient.ExpectRequest(disableFKChecks, &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(setStrictSQLMode, &sqltypes.Result{}, nil) + + tenv.vrdbClient.ExpectRequest(fmt.Sprintf(insertStreamsCreatedLog, bls), &sqltypes.Result{}, nil) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(getWorkflow, targetKs, wf), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id", + "int64", + ), + "1", + ), + ) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(startWorkflow, targetKs, wf), &sqltypes.Result{}) + _, err := ws.MoveTablesCreate(ctx, req) + require.NoError(t, err) + + tenv.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecardb.DefaultName), &sqltypes.Result{}, nil) + _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{ + Keyspace: req.TargetKeyspace, + Workflow: req.Workflow, + Cells: req.Cells, + TabletTypes: req.TabletTypes, + MaxReplicationLagAllowed: &vttime.Duration{Seconds: 922337203}, + Direction: int32(workflow.DirectionForward), + }) + require.NoError(t, err) + _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{ + Keyspace: req.TargetKeyspace, + Workflow: req.Workflow, + Cells: req.Cells, + TabletTypes: req.TabletTypes, + MaxReplicationLagAllowed: &vttime.Duration{Seconds: 922337203}, + Direction: int32(workflow.DirectionBackward), + }) + require.NoError(t, err) +} + +func TestUpdateVReplicationWorkflow(t *testing.T) { ctx := context.Background() cells := []string{"zone1"} tabletTypes := []string{"replica"} workflow := "testwf" - dbName := "test" + keyspace := "testks" vreplID := 1 - shortCircuitErr := fmt.Errorf("short circuiting test") - cp := mysql.ConnParams{} - db := fakesqldb.New(t) - ts := memorytopo.NewServer(cells[0]) - mysqld := mysqlctl.NewFakeMysqlDaemon(db) - dbClient := binlogplayer.NewMockDBClient(t) - dbClientFactory := func() binlogplayer.DBClient { return dbClient } - // Intentionally using Sprintf here as the query matching is exact - // and the engine uses `db_name=` w/o any spacing and the parser - // will add spacing. - dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vreplication where db_name='%s'", dbName), - &sqltypes.Result{}, nil) - vre := vreplication.NewSimpleTestEngine(ts, cells[0], mysqld, dbClientFactory, dbClientFactory, dbName, nil) - vre.Open(context.Background()) - tm := &TabletManager{ - MysqlDaemon: mysqld, - DBConfigs: dbconfigs.NewTestDBConfigs(cp, cp, dbName), - QueryServiceControl: tabletservermock.NewController(), - VREngine: vre, - } - defer func() { - vre.Close() - dbClient.Close() - mysqld.Close() - db.Close() - }() - parsed := sqlparser.BuildParsedQuery(sqlSelectVRWorkflowConfig, sidecardb.DefaultName, ":wf") + tabletUID := 100 + + tenv := newTestEnv(t, keyspace, []string{shard}) + defer tenv.close() + + tablet := tenv.addTablet(tabletUID, keyspace, shard) + defer tenv.deleteTablet(tablet.tablet) + + parsed := sqlparser.BuildParsedQuery(sqlSelectVReplicationWorkflowConfig, sidecardb.DefaultName, ":wf") bindVars := map[string]*querypb.BindVariable{ "wf": sqltypes.StringBindVariable(workflow), } @@ -101,66 +419,67 @@ func TestUpdateVRWorkflow(t *testing.T) { tests := []struct { name string - request *tabletmanagerdatapb.UpdateVRWorkflowRequest + request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest query string }{ { name: "update cells", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflow, Cells: []string{"zone2"}, // TabletTypes is an empty value, so the current value should be cleared }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '' where id in (%d)`, keyspace, shard, "zone2", vreplID), }, { name: "update cells, NULL tablet_types", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflow, Cells: []string{"zone3"}, - TabletTypes: textutil.SimulatedNullStringSlice, // So keep the current value of replica + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, // So keep the current value of replica }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '%s' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '%s' where id in (%d)`, keyspace, shard, "zone3", tabletTypes[0], vreplID), }, { name: "update tablet_types", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ - Workflow: workflow, - TabletTypes: []string{"in_order:rdonly", "replica"}, + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ + Workflow: workflow, + TabletSelectionPreference: tabletmanagerdatapb.TabletSelectionPreference_INORDER, + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_REPLICA}, }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '', tablet_types = '%s' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '', tablet_types = '%s' where id in (%d)`, keyspace, shard, "in_order:rdonly,replica", vreplID), }, { name: "update tablet_types, NULL cells", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflow, Cells: textutil.SimulatedNullStringSlice, // So keep the current value of zone1 - TabletTypes: []string{"rdonly"}, + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY}, }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '%s' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}}', cell = '%s', tablet_types = '%s' where id in (%d)`, keyspace, shard, cells[0], "rdonly", vreplID), }, { name: "update on_ddl", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflow, OnDdl: binlogdatapb.OnDDLAction_EXEC, }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}} on_ddl:%s', cell = '', tablet_types = '' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}} on_ddl:%s', cell = '', tablet_types = '' where id in (%d)`, keyspace, shard, binlogdatapb.OnDDLAction_EXEC.String(), vreplID), }, { name: "update cell,tablet_types,on_ddl", - request: &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Workflow: workflow, Cells: []string{"zone1", "zone2", "zone3"}, - TabletTypes: []string{"rdonly", "replica", "primary"}, + TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_REPLICA, topodatapb.TabletType_PRIMARY}, OnDdl: binlogdatapb.OnDDLAction_EXEC_IGNORE, }, - query: fmt.Sprintf(`update _vt.vreplication set source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}} on_ddl:%s', cell = '%s', tablet_types = '%s' where id in (%d)`, + query: fmt.Sprintf(`update _vt.vreplication set state = 'Stopped', source = 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"customer\" filter:\"select * from customer\"} rules:{match:\"corder\" filter:\"select * from corder\"}} on_ddl:%s', cell = '%s', tablet_types = '%s' where id in (%d)`, keyspace, shard, binlogdatapb.OnDDLAction_EXEC_IGNORE.String(), "zone1,zone2,zone3", "rdonly,replica,primary", vreplID), }, } @@ -178,19 +497,21 @@ func TestUpdateVRWorkflow(t *testing.T) { require.NotNil(t, tt.request, "No request provided") require.NotEqual(t, "", tt.query, "No expected query provided") + tt.request.State = binlogdatapb.VReplicationWorkflowState_Stopped + // These are the same for each RPC call. - dbClient.ExpectRequest(fmt.Sprintf("use %s", sidecardb.DefaultName), &sqltypes.Result{}, nil) - dbClient.ExpectRequest(selectQuery, selectRes, nil) - dbClient.ExpectRequest(fmt.Sprintf("use %s", sidecardb.DefaultName), &sqltypes.Result{}, nil) - dbClient.ExpectRequest(idQuery, idRes, nil) + tenv.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecardb.DefaultName), &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(selectQuery, selectRes, nil) + tenv.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecardb.DefaultName), &sqltypes.Result{}, nil) + tenv.vrdbClient.ExpectRequest(idQuery, idRes, nil) // This is our expected query, which will also short circuit // the test with an error as at this point we've tested what // we wanted to test. - dbClient.ExpectRequest(tt.query, &sqltypes.Result{RowsAffected: 1}, shortCircuitErr) - _, err = tm.UpdateVRWorkflow(ctx, tt.request) - dbClient.Wait() - require.ErrorIs(t, err, shortCircuitErr) + tenv.vrdbClient.ExpectRequest(tt.query, &sqltypes.Result{RowsAffected: 1}, errShortCircuit) + _, err = tenv.tmc.tm.UpdateVReplicationWorkflow(ctx, tt.request) + tenv.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) }) } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 34e28147ff1..e961afcb98d 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -100,7 +100,7 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor state := params["state"] blpStats.State.Store(state) // Nothing to do if replication is stopped or is known to have an unrecoverable error. - if state == binlogplayer.BlpStopped || state == binlogplayer.BlpError { + if state == binlogdatapb.VReplicationWorkflowState_Stopped.String() || state == binlogdatapb.VReplicationWorkflowState_Error.String() { ct.cancel = func() {} close(ct.done) return ct, nil @@ -270,7 +270,7 @@ func (ct *controller) runBlp(ctx context.Context) (err error) { // we cannot identify this as non-recoverable, but it has persisted beyond the retry limit (maxTimeToRetryError) if isUnrecoverableError(err) || !ct.lastWorkflowError.ShouldRetry() { log.Errorf("vreplication stream %d going into error state due to %+v", ct.id, err) - if errSetState := vr.setState(binlogplayer.BlpError, err.Error()); errSetState != nil { + if errSetState := vr.setState(binlogdatapb.VReplicationWorkflowState_Error, err.Error()); errSetState != nil { log.Errorf("INTERNAL: unable to setState() in controller. Attempting to set error text: [%v]; setState() error is: %v", err, errSetState) return err // yes, err and not errSetState. } diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go index 94762102f8f..efab9693fa2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go @@ -23,13 +23,13 @@ import ( "testing" "time" - "vitess.io/vitess/go/vt/mysqlctl" - querypb "vitess.io/vitess/go/vt/proto/query" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/mysqlctl/tmutils" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -54,11 +54,11 @@ var ( sqltypes.NULL, // stop_pos sqltypes.NewInt64(9223372036854775807), // max_tps sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(0), // workflow_sub_type - sqltypes.NewInt64(0), // defer_secondary_keys + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(0), // workflow_sub_type + sqltypes.NewInt64(0), // defer_secondary_keys }, }, } @@ -74,7 +74,7 @@ func TestControllerKeyRange(t *testing.T) { defer deleteTablet(wantTablet) params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName), } @@ -112,7 +112,7 @@ func TestControllerTables(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" tables:"table1" tables:"/funtables_/" `, env.KeyspaceName), } @@ -181,7 +181,7 @@ func TestControllerBadID(t *testing.T) { func TestControllerStopped(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpStopped, + "state": binlogdatapb.VReplicationWorkflowState_Stopped.String(), } ct, err := newController(context.Background(), params, nil, nil, nil, "", "", nil, nil) @@ -204,7 +204,7 @@ func TestControllerOverrides(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName), "cell": env.Cells[0], "tablet_types": "replica", @@ -243,7 +243,7 @@ func TestControllerCanceledContext(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName), } @@ -274,7 +274,7 @@ func TestControllerRetry(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName), "cell": env.Cells[0], "tablet_types": "replica", @@ -313,7 +313,7 @@ func TestControllerStopPosition(t *testing.T) { params := map[string]string{ "id": "1", - "state": binlogplayer.BlpRunning, + "state": binlogdatapb.VReplicationWorkflowState_Running.String(), "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName), } @@ -335,15 +335,15 @@ func TestControllerStopPosition(t *testing.T) { InsertID: 0, Rows: [][]sqltypes.Value{ { - sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos - sqltypes.NewVarBinary("MariaDB/0-1-1235"), // stop_pos - sqltypes.NewInt64(9223372036854775807), // max_tps - sqltypes.NewInt64(9223372036854775807), // max_replication_lag - sqltypes.NewVarBinary("Running"), // state - sqltypes.NewInt64(1), // workflow_type - sqltypes.NewVarChar("wf"), // workflow - sqltypes.NewInt64(1), // workflow_sub_type - sqltypes.NewInt64(1), // defer_secondary_keys + sqltypes.NewVarBinary("MariaDB/0-1-1083"), // pos + sqltypes.NewVarBinary("MariaDB/0-1-1235"), // stop_pos + sqltypes.NewInt64(9223372036854775807), // max_tps + sqltypes.NewInt64(9223372036854775807), // max_replication_lag + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), // state + sqltypes.NewInt64(1), // workflow_type + sqltypes.NewVarChar("wf"), // workflow + sqltypes.NewInt64(1), // workflow_sub_type + sqltypes.NewInt64(1), // defer_secondary_keys }, }, } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 52f9c072d49..451033cf0c4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -221,7 +221,6 @@ func (vre *Engine) Open(ctx context.Context) { } func (vre *Engine) openLocked(ctx context.Context) error { - rows, err := vre.readAllRows(ctx) if err != nil { return err @@ -685,7 +684,7 @@ func (vre *Engine) transitionJournal(je *journalEvent) { workflowType, _ := strconv.ParseInt(params["workflow_type"], 10, 32) workflowSubType, _ := strconv.ParseInt(params["workflow_sub_type"], 10, 32) deferSecondaryKeys, _ := strconv.ParseBool(params["defer_secondary_keys"]) - ig := NewInsertGenerator(binlogplayer.BlpRunning, vre.dbName) + ig := NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Running, vre.dbName) ig.AddRow(params["workflow"], bls, sgtid.Gtid, params["cell"], params["tablet_types"], binlogdatapb.VReplicationWorkflowType(workflowType), binlogdatapb.VReplicationWorkflowSubType(workflowSubType), deferSecondaryKeys) qr, err := dbClient.ExecuteFetch(ig.String(), maxRows) @@ -803,7 +802,7 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int32, pos string) error { return nil } - if qr.Rows[0][1].ToString() == binlogplayer.BlpStopped { + if qr.Rows[0][1].ToString() == binlogdatapb.VReplicationWorkflowState_Stopped.String() { return fmt.Errorf("replication has stopped at %v before reaching position %v, message: %s", current, mPos, qr.Rows[0][2].ToString()) } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index d490417784f..32add04c8e0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -31,6 +31,8 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/mysqlctl" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) func TestEngineOpen(t *testing.T) { @@ -383,12 +385,12 @@ func TestWaitForPos(t *testing.T) { dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), - sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), sqltypes.NewVarBinary(""), }}}, nil) dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1084"), - sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), sqltypes.NewVarBinary(""), }}}, nil) start := time.Now() @@ -451,7 +453,7 @@ func TestWaitForPosCancel(t *testing.T) { dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), - sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), sqltypes.NewVarBinary(""), }}}, nil) ctx, cancel := context.WithCancel(context.Background()) @@ -469,7 +471,7 @@ func TestWaitForPosCancel(t *testing.T) { }() dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), - sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(binlogdatapb.VReplicationWorkflowState_Running.String()), sqltypes.NewVarBinary(""), }}}, nil) err = vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go index 19e5933f428..da1753a8444 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go @@ -36,12 +36,12 @@ type InsertGenerator struct { } // NewInsertGenerator creates a new InsertGenerator. -func NewInsertGenerator(state, dbname string) *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 ") return &InsertGenerator{ buf: buf, - state: state, + state: state.String(), dbname: dbname, now: time.Now().Unix(), } diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go index 3f79a28a765..5ccdfe3da10 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go @@ -21,12 +21,11 @@ import ( "github.com/stretchr/testify/assert" - "vitess.io/vitess/go/vt/binlog/binlogplayer" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) func TestInsertGenerator(t *testing.T) { - ig := NewInsertGenerator(binlogplayer.BlpStopped, "a") + 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 ` + diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index b3728598768..4dfd4c129a3 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -237,7 +237,7 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { if _, err := vc.vr.dbClient.Execute(buf.String()); err != nil { return err } - if err := vc.vr.setState(binlogplayer.VReplicationCopying, ""); err != nil { + if err := vc.vr.setState(binlogdatapb.VReplicationWorkflowState_Copying, ""); err != nil { return err } if err := vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase started for %d table(s)", @@ -268,7 +268,7 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { } } } else { - if err := vc.vr.setState(binlogplayer.BlpStopped, "There is nothing to replicate"); err != nil { + if err := vc.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, "There is nothing to replicate"); err != nil { return err } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go index e1b1686b0d7..ddb2b26e5f7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go @@ -157,7 +157,7 @@ func testPlayerCopyCharPK(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -264,7 +264,7 @@ func testPlayerCopyVarcharPKCaseInsensitive(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -387,7 +387,7 @@ func testPlayerCopyVarcharCompositePKCaseSensitiveCollation(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -470,7 +470,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) require.NoError(t, err) @@ -593,7 +593,7 @@ func testPlayerCopyTables(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -732,7 +732,7 @@ func testPlayerCopyBigTable(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -862,7 +862,7 @@ func testPlayerCopyWildcardRule(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -999,7 +999,7 @@ func testPlayerCopyTableContinuation(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.BlpStopped, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Stopped, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1136,7 +1136,7 @@ func testPlayerCopyWildcardTableContinuation(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.BlpStopped, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Stopped, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1233,7 +1233,7 @@ func TestPlayerCopyWildcardTableContinuationWithOptimizeInserts(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.BlpStopped, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Stopped, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1302,7 +1302,7 @@ func testPlayerCopyTablesNone(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1356,7 +1356,7 @@ func testPlayerCopyTablesStopAfterCopy(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, StopAfterCopy: true, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1445,7 +1445,7 @@ func testPlayerCopyTablesGIPK(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, StopAfterCopy: true, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1543,7 +1543,7 @@ func testPlayerCopyTableCancel(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1626,7 +1626,7 @@ func testPlayerCopyTablesWithGeneratedColumn(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1707,7 +1707,7 @@ func testCopyTablesWithInvalidDates(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) require.NoError(t, err) @@ -1794,7 +1794,7 @@ func testCopyInvisibleColumns(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogdatapb.VReplicationWorkflowState_Init, playerEngine.dbName, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 640dd67a4c1..94287aff785 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -109,7 +109,7 @@ func (vp *vplayer) play(ctx context.Context) error { if !vp.stopPos.IsZero() && vp.startPos.AtLeast(vp.stopPos) { log.Infof("Stop position %v already reached: %v", vp.startPos, vp.stopPos) if vp.saveStop { - return vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.startPos, vp.stopPos)) + return vp.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, fmt.Sprintf("Stop position %v already reached: %v", vp.startPos, vp.stopPos)) } return nil } @@ -251,7 +251,7 @@ func (vp *vplayer) updatePos(ts int64) (posReached bool, err error) { if posReached { log.Infof("Stopped at position: %v", vp.stopPos) if vp.saveStop { - if err := vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { + if err := vp.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { return false, err } } @@ -544,7 +544,7 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m if _, err := vp.updatePos(event.Timestamp); err != nil { return err } - if err := vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Statement)); err != nil { + if err := vp.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, fmt.Sprintf("Stopped at DDL %s", event.Statement)); err != nil { return err } if err := vp.vr.dbClient.Commit(); err != nil { @@ -608,7 +608,7 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m switch { case found && notFound: // Some were found and some were not found. We can't handle this. - if err := vp.vr.setState(binlogplayer.BlpStopped, "unable to handle journal event: tables were partially matched"); err != nil { + if err := vp.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, "unable to handle journal event: tables were partially matched"); err != nil { return err } return io.EOF @@ -620,7 +620,7 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m } log.Infof("Binlog event registering journal event %+v", event.Journal) if err := vp.vr.vre.registerJournal(event.Journal, vp.vr.id); err != nil { - if err := vp.vr.setState(binlogplayer.BlpStopped, err.Error()); err != nil { + if err := vp.vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, err.Error()); err != nil { return err } return io.EOF diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go index a16ecb9e4e0..98e6954c0b6 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go @@ -1828,7 +1828,7 @@ func TestPlayerStopPos(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } startPos := primaryPosition(t) - query := binlogplayer.CreateVReplicationState("test", bls, startPos, binlogplayer.BlpStopped, vrepldb, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, startPos, binlogdatapb.VReplicationWorkflowState_Stopped, vrepldb, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -1932,7 +1932,7 @@ func TestPlayerStopAtOther(t *testing.T) { Filter: filter, OnDdl: binlogdatapb.OnDDLAction_IGNORE, } - query := binlogplayer.CreateVReplicationState("test", bls, startPos, binlogplayer.BlpStopped, vrepldb, 0, 0) + query := binlogplayer.CreateVReplicationState("test", bls, startPos, binlogdatapb.VReplicationWorkflowState_Stopped, vrepldb, 0, 0) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -2733,7 +2733,7 @@ func TestVReplicationLogs(t *testing.T) { for _, want := range expected { t.Run("", func(t *testing.T) { - err = insertLog(vdbc, LogMessage, 1, "Running", "message1") + err = insertLog(vdbc, LogMessage, 1, binlogdatapb.VReplicationWorkflowState_Running.String(), "message1") require.NoError(t, err) qr, err := env.Mysqld.FetchSuperQuery(context.Background(), query) require.NoError(t, err) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index f9c36d0b102..3a1e1cc4acd 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -100,7 +100,7 @@ type vreplicator struct { // source source *binlogdatapb.BinlogSource sourceVStreamer VStreamerClient - state string + state binlogdatapb.VReplicationWorkflowState stats *binlogplayer.Stats // mysqld is used to fetch the local schema. mysqld mysqlctl.MysqlDaemon @@ -263,7 +263,7 @@ func (vr *vreplicator) replicate(ctx context.Context) error { } // If any of the operations below changed state to Stopped or Error, we should return. - if settings.State == binlogplayer.BlpStopped || settings.State == binlogplayer.BlpError { + if settings.State == binlogdatapb.VReplicationWorkflowState_Stopped || settings.State == binlogdatapb.VReplicationWorkflowState_Error { return nil } switch { @@ -296,9 +296,9 @@ func (vr *vreplicator) replicate(ctx context.Context) error { return err } if vr.source.StopAfterCopy { - return vr.setState(binlogplayer.BlpStopped, "Stopped after copy.") + return vr.setState(binlogdatapb.VReplicationWorkflowState_Stopped, "Stopped after copy.") } - if err := vr.setState(binlogplayer.BlpRunning, ""); err != nil { + if err := vr.setState(binlogdatapb.VReplicationWorkflowState_Running, ""); err != nil { vr.stats.ErrorCounts.Add([]string{"Replicate"}, 1) return err } @@ -445,24 +445,24 @@ func (vr *vreplicator) setMessage(message string) error { if _, err := vr.dbClient.Execute(query); err != nil { return fmt.Errorf("could not set message: %v: %v", query, err) } - if err := insertLog(vr.dbClient, LogMessage, vr.id, vr.state, message); err != nil { + if err := insertLog(vr.dbClient, LogMessage, vr.id, vr.state.String(), message); err != nil { return err } return nil } func (vr *vreplicator) insertLog(typ, message string) error { - return insertLog(vr.dbClient, typ, vr.id, vr.state, message) + return insertLog(vr.dbClient, typ, vr.id, vr.state.String(), message) } -func (vr *vreplicator) setState(state, message string) error { +func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, message string) error { if message != "" { vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), Message: message, }) } - vr.stats.State.Store(state) + vr.stats.State.Store(state.String()) query := fmt.Sprintf("update _vt.vreplication set state='%v', message=%v where id=%v", state, encodeString(binlogplayer.MessageTruncate(message)), vr.id) if _, err := vr.dbClient.ExecuteFetch(query, 1); err != nil { return fmt.Errorf("could not set state: %v: %v", query, err) @@ -470,7 +470,7 @@ func (vr *vreplicator) setState(state, message string) error { if state == vr.state { return nil } - if err := insertLog(vr.dbClient, LogStateChange, vr.id, state, message); err != nil { + if err := insertLog(vr.dbClient, LogStateChange, vr.id, state.String(), message); err != nil { return err } vr.state = state diff --git a/go/vt/vttablet/tmclient/rpc_client_api.go b/go/vt/vttablet/tmclient/rpc_client_api.go index d77bcd74f24..9cab47febbe 100644 --- a/go/vt/vttablet/tmclient/rpc_client_api.go +++ b/go/vt/vttablet/tmclient/rpc_client_api.go @@ -171,11 +171,17 @@ type TabletManagerClient interface { // WaitForPosition waits for the position to be reached WaitForPosition(ctx context.Context, tablet *topodatapb.Tablet, pos string) error + // + // VReplication related methods + // + + CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) + DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) + ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) + UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) // VReplicationExec executes a VReplication command VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) VReplicationWaitForPos(ctx context.Context, tablet *topodatapb.Tablet, id int32, pos string) error - UpdateVRWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) - VDiff(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) // diff --git a/go/vt/vttablet/tmrpctest/test_tm_rpc.go b/go/vt/vttablet/tmrpctest/test_tm_rpc.go index c008b11e612..befb1753c43 100644 --- a/go/vt/vttablet/tmrpctest/test_tm_rpc.go +++ b/go/vt/vttablet/tmrpctest/test_tm_rpc.go @@ -54,12 +54,27 @@ type fakeRPCTM struct { mu sync.Mutex } -func (fra *fakeRPCTM) ResetSequences(ctx context.Context, tables []string) error { +func (fra *fakeRPCTM) CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + //TODO implement me + panic("implement me") +} + +func (fra *fakeRPCTM) DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) { + //TODO implement me + panic("implement me") +} + +func (fra *fakeRPCTM) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { //TODO implement me panic("implement me") } -func (fra *fakeRPCTM) UpdateVRWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVRWorkflowRequest) (*tabletmanagerdatapb.UpdateVRWorkflowResponse, error) { +func (fra *fakeRPCTM) UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { + //TODO implement me + panic("implement me") +} + +func (fra *fakeRPCTM) ResetSequences(ctx context.Context, tables []string) error { //TODO implement me panic("implement me") } diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go index c1f253fedf3..bf01ce9bf24 100644 --- a/go/vt/wrangler/keyspace.go +++ b/go/vt/wrangler/keyspace.go @@ -24,15 +24,15 @@ import ( "sync" "time" - "vitess.io/vitess/go/vt/proto/binlogdata" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/concurrency" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vterrors" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) const ( @@ -72,7 +72,7 @@ func (wr *Wrangler) validateNewWorkflow(ctx context.Context, keyspace, workflow fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and workflow=%s", encodeString(primary.DbName()), encodeString(workflow)), fmt.Sprintf("workflow %s already exists in keyspace %s on tablet %d", workflow, keyspace, primary.Alias.Uid), }, { - fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and message='FROZEN' and workflow_sub_type != %d", encodeString(primary.DbName()), binlogdata.VReplicationWorkflowSubType_Partial), + fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and message='FROZEN' and workflow_sub_type != %d", encodeString(primary.DbName()), binlogdatapb.VReplicationWorkflowSubType_Partial), fmt.Sprintf("found previous frozen workflow on tablet %d, please review and delete it first before creating a new workflow", primary.Alias.Uid), }} diff --git a/go/vt/wrangler/materializer.go b/go/vt/wrangler/materializer.go index 6dea5d3b9f1..de0b0225144 100644 --- a/go/vt/wrangler/materializer.go +++ b/go/vt/wrangler/materializer.go @@ -33,14 +33,15 @@ import ( "vitess.io/vitess/go/json2" "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl/tmutils" "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vtctl/schematools" "vitess.io/vitess/go/vt/vtctl/workflow" @@ -128,7 +129,7 @@ func shouldInclude(table string, excludes []string) bool { // MoveTables initiates moving table(s) over to another keyspace func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, targetKeyspace, tableSpecs, - cell, tabletTypes string, allTables bool, excludeTables string, autoStart, stopAfterCopy bool, + cell, tabletTypesStr string, allTables bool, excludeTables string, autoStart, stopAfterCopy bool, externalCluster string, dropForeignKeys, deferSecondaryKeys bool, sourceTimeZone, onDDL string, sourceShards []string) error { //FIXME validate tableSpecs, allTables, excludeTables var tables []string @@ -245,18 +246,27 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta if err := wr.ts.RebuildSrvVSchema(ctx, nil); err != nil { return err } + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(tabletTypesStr) + if err != nil { + return err + } + tsp := tabletmanagerdatapb.TabletSelectionPreference_ANY + if inorder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } ms := &vtctldatapb.MaterializeSettings{ - Workflow: workflow, - MaterializationIntent: vtctldatapb.MaterializationIntent_MOVETABLES, - SourceKeyspace: sourceKeyspace, - TargetKeyspace: targetKeyspace, - Cell: cell, - TabletTypes: tabletTypes, - StopAfterCopy: stopAfterCopy, - ExternalCluster: externalCluster, - SourceShards: sourceShards, - OnDdl: onDDL, - DeferSecondaryKeys: deferSecondaryKeys, + Workflow: workflow, + MaterializationIntent: vtctldatapb.MaterializationIntent_MOVETABLES, + SourceKeyspace: sourceKeyspace, + TargetKeyspace: targetKeyspace, + Cell: cell, + TabletTypes: topoproto.MakeStringTypeCSV(tabletTypes), + TabletSelectionPreference: tsp, + StopAfterCopy: stopAfterCopy, + ExternalCluster: externalCluster, + SourceShards: sourceShards, + OnDdl: onDDL, + DeferSecondaryKeys: deferSecondaryKeys, } if sourceTimeZone != "" { ms.SourceTimeZone = sourceTimeZone @@ -314,7 +324,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta if autoStart { return mz.startStreams(ctx) } - wr.Logger().Infof("Streams will not be started since -auto_start is set to false") + wr.Logger().Infof("Streams will not be started since --auto_start is set to false") return nil } @@ -424,7 +434,7 @@ func (wr *Wrangler) checkIfPreviousJournalExists(ctx context.Context, mz *materi } // CreateLookupVindex creates a lookup vindex and sets up the backfill. -func (wr *Wrangler) CreateLookupVindex(ctx context.Context, keyspace string, specs *vschemapb.Keyspace, cell, tabletTypes string, continueAfterCopyWithOwner bool) error { +func (wr *Wrangler) CreateLookupVindex(ctx context.Context, keyspace string, specs *vschemapb.Keyspace, cell, tabletTypesStr string, continueAfterCopyWithOwner bool) error { ms, sourceVSchema, targetVSchema, err := wr.prepareCreateLookup(ctx, keyspace, specs, continueAfterCopyWithOwner) if err != nil { return err @@ -433,7 +443,17 @@ func (wr *Wrangler) CreateLookupVindex(ctx context.Context, keyspace string, spe return err } ms.Cell = cell - ms.TabletTypes = tabletTypes + + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(tabletTypesStr) + if err != nil { + return err + } + tsp := tabletmanagerdatapb.TabletSelectionPreference_ANY + if inorder { + tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + ms.TabletTypes = topoproto.MakeStringTypeCSV(tabletTypes) + ms.TabletSelectionPreference = tsp if err := wr.Materialize(ctx, ms); err != nil { return err } @@ -800,7 +820,7 @@ func (wr *Wrangler) ExternalizeVindex(ctx context.Context, qualifiedVindexName s if err != nil { return err } - state := row[1].ToString() + state := binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[row[1].ToString()]) message := row[2].ToString() var bls binlogdatapb.BinlogSource sourceBytes, err := row[3].ToBytes() @@ -813,12 +833,12 @@ func (wr *Wrangler) ExternalizeVindex(ctx context.Context, qualifiedVindexName s if sourceVindex.Owner == "" || !bls.StopAfterCopy { // If there's no owner or we've requested that the workflow NOT be stopped // after the copy phase completes, then all streams need to be running. - if state != binlogplayer.BlpRunning { + if state != binlogdatapb.VReplicationWorkflowState_Running { return fmt.Errorf("stream %d for %v.%v is not in Running state: %v", id, targetShard.Keyspace(), targetShard.ShardName(), state) } } else { // If there is an owner, all streams need to be stopped after copy. - if state != binlogplayer.BlpStopped || !strings.Contains(message, "Stopped after copy") { + if state != binlogdatapb.VReplicationWorkflowState_Stopped || !strings.Contains(message, "Stopped after copy") { return fmt.Errorf("stream %d for %v.%v is not in Stopped after copy state: %v, %v", id, targetShard.Keyspace(), targetShard.ShardName(), state, message) } } @@ -1226,7 +1246,7 @@ func stripTableConstraints(ddl string) (string, error) { } func (mz *materializer) generateInserts(ctx context.Context, targetShard *topo.ShardInfo) (string, error) { - ig := vreplication.NewInsertGenerator(binlogplayer.BlpStopped, "{{.dbname}}") + ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, "{{.dbname}}") for _, sourceShard := range mz.sourceShards { // Don't create streams from sources which won't contain data for the target shard. @@ -1326,7 +1346,12 @@ func (mz *materializer) generateInserts(ctx context.Context, targetShard *topo.S workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex } - ig.AddRow(mz.ms.Workflow, bls, "", mz.ms.Cell, mz.ms.TabletTypes, + tabletTypeStr := mz.ms.TabletTypes + if mz.ms.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { + tabletTypeStr = discovery.InOrderHint + tabletTypeStr + } + + ig.AddRow(mz.ms.Workflow, bls, "", mz.ms.Cell, tabletTypeStr, workflowType, workflowSubType, mz.ms.DeferSecondaryKeys, diff --git a/go/vt/wrangler/materializer_test.go b/go/vt/wrangler/materializer_test.go index bd3e23fce7f..0001058a7b3 100644 --- a/go/vt/wrangler/materializer_test.go +++ b/go/vt/wrangler/materializer_test.go @@ -30,14 +30,16 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/test/utils" "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vtgate/vindexes" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" vschemapb "vitess.io/vitess/go/vt/proto/vschema" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" - "vitess.io/vitess/go/vt/topo/memorytopo" - "vitess.io/vitess/go/vt/vtgate/vindexes" ) const mzUpdateQuery = "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='workflow'" @@ -1771,8 +1773,11 @@ func TestMaterializerOneToOne(t *testing.T) { CreateDdl: "t4ddl", }, }, - Cell: "zone1", - TabletTypes: "primary,rdonly", + Cell: "zone1", + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), } env := newTestMaterializerEnv(t, ms, []string{"0"}, []string{"0"}) defer env.close() diff --git a/go/vt/wrangler/resharder.go b/go/vt/wrangler/resharder.go index 4a3ae9d8bb0..a81c3e8d598 100644 --- a/go/vt/wrangler/resharder.go +++ b/go/vt/wrangler/resharder.go @@ -30,16 +30,16 @@ import ( "vitess.io/vitess/go/vt/vtctl/workflow" "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/key" - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - vschemapb "vitess.io/vitess/go/vt/proto/vschema" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/vindexes" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" ) type resharder struct { @@ -312,7 +312,7 @@ func (rs *resharder) createStreams(ctx context.Context) error { err := rs.forAll(rs.targetShards, func(target *topo.ShardInfo) error { targetPrimary := rs.targetPrimaries[target.ShardName()] - ig := vreplication.NewInsertGenerator(binlogplayer.BlpStopped, targetPrimary.DbName()) + ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, targetPrimary.DbName()) // copy excludeRules to prevent data race. copyExcludeRules := append([]*binlogdatapb.Rule(nil), excludeRules...) diff --git a/go/vt/wrangler/traffic_switcher.go b/go/vt/wrangler/traffic_switcher.go index b2f77a2cf01..e3f45283ee2 100644 --- a/go/vt/wrangler/traffic_switcher.go +++ b/go/vt/wrangler/traffic_switcher.go @@ -26,12 +26,11 @@ import ( "sync" "time" - "vitess.io/vitess/go/sqlescape" - "vitess.io/vitess/go/vt/discovery" - "vitess.io/vitess/go/json2" + "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" @@ -53,7 +52,7 @@ import ( const ( errorNoStreams = "no streams found in keyspace %s for: %s" - // use pt-osc's naming convention, this format also ensures vstreamer ignores such tables + // Use pt-osc's naming convention, this format also ensures vstreamer ignores such tables. renameTableTemplate = "_%.59s_old" // limit table name to 64 characters sqlDeleteWorkflow = "delete from _vt.vreplication where db_name = %s and workflow = %s" @@ -66,9 +65,9 @@ const ( allowWrites = accessType(iota) disallowWrites - // number of LOCK TABLES cycles to perform on the sources during SwitchWrites + // Number of LOCK TABLES cycles to perform on the sources during SwitchWrites. lockTablesCycles = 2 - // time to wait between LOCK TABLES cycles on the sources during SwitchWrites + // Time to wait between LOCK TABLES cycles on the sources during SwitchWrites. lockTablesCycleDelay = time.Duration(100 * time.Millisecond) // How long to wait when refreshing the state of each tablet in a shard. Note that these @@ -90,7 +89,7 @@ type trafficSwitcher struct { wr *Wrangler workflow string - // if frozen is true, the rest of the fields are not set. + // If frozen is true, the rest of the fields are not set. frozen bool reverseWorkflow string id int64 @@ -101,8 +100,8 @@ type trafficSwitcher struct { tables []string keepRoutingRules bool sourceKSSchema *vindexes.KeyspaceSchema - optCells string //cells option passed to MoveTables/Reshard - optTabletTypes string //tabletTypes option passed to MoveTables/Reshard + optCells string // cells option passed to MoveTables/Reshard + optTabletTypes string // tabletTypes option passed to MoveTables/Reshard externalCluster string externalTopo *topo.Server sourceTimeZone string @@ -111,17 +110,6 @@ type trafficSwitcher struct { workflowSubType binlogdatapb.VReplicationWorkflowSubType } -/* -begin: implementation of workflow.ITrafficSwitcher - -(NOTE:@ajm188) Please see comments on that interface type for why this exists. -This is temporary to allow workflow.StreamMigrator to use this trafficSwitcher -code and should be removed in the very near-term when we move trafficSwitcher to -package workflow as well. -*/ - -var _ workflow.ITrafficSwitcher = (*trafficSwitcher)(nil) - func (ts *trafficSwitcher) TopoServer() *topo.Server { return ts.wr.ts } func (ts *trafficSwitcher) TabletManagerClient() tmclient.TabletManagerClient { return ts.wr.tmc } func (ts *trafficSwitcher) Logger() logutil.Logger { return ts.wr.logger } @@ -279,7 +267,8 @@ func (wr *Wrangler) getWorkflowState(ctx context.Context, targetKeyspace, workfl } for _, table := range ts.Tables() { rr := globalRules[table] - // if a rule exists for the table and points to the target keyspace, writes have been switched + // If a rule exists for the table and points to the target keyspace, writes + // have been switched. if len(rr) > 0 && rr[0] == fmt.Sprintf("%s.%s", targetKeyspace, table) { state.WritesSwitched = true break @@ -289,7 +278,7 @@ func (wr *Wrangler) getWorkflowState(ctx context.Context, targetKeyspace, workfl } else { state.WorkflowType = workflow.TypeReshard - // we assume a consistent state, so only choose one shard + // We assume a consistent state, so only choose one shard. var shard *topo.ShardInfo if reverse { shard = ts.TargetShards()[0] @@ -351,9 +340,10 @@ func (wr *Wrangler) SwitchReads(ctx context.Context, targetKeyspace, workflowNam } } - // if there are no rdonly tablets in the cells ask to switch rdonly tablets as well so that routing rules + // If there are no rdonly tablets in the cells ask to switch rdonly tablets as well so that routing rules // are updated for rdonly as well. Otherwise vitess will not know that the workflow has completed and will - // incorrectly report that not all reads have been switched. User currently is forced to switch non-existent rdonly tablets + // incorrectly report that not all reads have been switched. User currently is forced to switch + // non-existent rdonly tablets. if switchReplicas && !switchRdonly { var err error rdonlyTabletsExist, err := topotools.DoCellsHaveRdonlyTablets(ctx, wr.ts, cells) @@ -365,7 +355,7 @@ func (wr *Wrangler) SwitchReads(ctx context.Context, targetKeyspace, workflowNam } } - // If journals exist notify user and fail + // If journals exist notify user and fail. journalsExist, _, err := ts.checkJournals(ctx) if err != nil { wr.Logger().Errorf("checkJournals failed: %v", err) @@ -648,7 +638,8 @@ func (wr *Wrangler) SwitchWrites(ctx context.Context, targetKeyspace, workflowNa return ts.id, sw.logs(), nil } -// DropTargets cleans up target tables, shards and denied tables if a MoveTables/Reshard is cancelled +// DropTargets cleans up target tables, shards and denied tables if a +// MoveTables/Reshard is cancelled. func (wr *Wrangler) DropTargets(ctx context.Context, targetKeyspace, workflow string, keepData, keepRoutingRules, dryRun bool) (*[]string, error) { ts, err := wr.buildTrafficSwitcher(ctx, targetKeyspace, workflow) if err != nil { @@ -725,7 +716,7 @@ func (wr *Wrangler) dropArtifacts(ctx context.Context, keepRoutingRules bool, sw } // finalizeMigrateWorkflow deletes the streams for the Migrate workflow. -// We only cleanup the target for external sources +// We only cleanup the target for external sources. func (wr *Wrangler) finalizeMigrateWorkflow(ctx context.Context, targetKeyspace, workflow, tableSpecs string, cancel, keepData, keepRoutingRules, dryRun bool) (*[]string, error) { ts, err := wr.buildTrafficSwitcher(ctx, targetKeyspace, workflow) @@ -765,7 +756,8 @@ func (wr *Wrangler) finalizeMigrateWorkflow(ctx context.Context, targetKeyspace, return sw.logs(), nil } -// DropSources cleans up source tables, shards and denied tables after a MoveTables/Reshard is completed +// DropSources cleans up source tables, shards and denied tables after a +// MoveTables/Reshard is completed. func (wr *Wrangler) DropSources(ctx context.Context, targetKeyspace, workflowName string, removalType workflow.TableRemovalType, keepData, keepRoutingRules, force, dryRun bool) (*[]string, error) { ts, err := wr.buildTrafficSwitcher(ctx, targetKeyspace, workflowName) if err != nil { @@ -830,7 +822,7 @@ func (wr *Wrangler) DropSources(ctx context.Context, targetKeyspace, workflowNam } func (wr *Wrangler) buildTrafficSwitcher(ctx context.Context, targetKeyspace, workflowName string) (*trafficSwitcher, error) { - tgtInfo, err := workflow.BuildTargets(ctx, wr.ts, wr.tmc, targetKeyspace, workflowName) + tgtInfo, err := workflow.LegacyBuildTargets(ctx, wr.ts, wr.tmc, targetKeyspace, workflowName) if err != nil { log.Infof("Error building targets: %s", err) return nil, err @@ -854,7 +846,7 @@ func (wr *Wrangler) buildTrafficSwitcher(ctx context.Context, targetKeyspace, wo log.Infof("Migration ID for workflow %s: %d", workflowName, ts.id) sourceTopo := wr.ts - // Build the sources + // Build the sources. for _, target := range targets { for _, bls := range target.Sources { if ts.sourceKeyspace == "" { @@ -897,6 +889,10 @@ func (wr *Wrangler) buildTrafficSwitcher(ctx context.Context, targetKeyspace, wo if err != nil { return nil, err } + if sourcesi.PrimaryAlias == nil { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "source shard %s/%s currently has no primary tablet", + bls.Keyspace, bls.Shard) + } sourcePrimary, err := sourceTopo.GetTablet(ctx, sourcesi.PrimaryAlias) if err != nil { return nil, err @@ -950,8 +946,9 @@ func (ts *trafficSwitcher) getSourceAndTargetShardsNames() ([]string, []string) return sourceShards, targetShards } -// isPartialMoveTables returns true if whe workflow is MoveTables, -// has the same number of shards, is not covering the entire shard range, and has one-to-one shards in source and target +// isPartialMoveTables returns true if whe workflow is MoveTables, has the +// same number of shards, is not covering the entire shard range, and has +// one-to-one shards in source and target. func (ts *trafficSwitcher) isPartialMoveTables(sourceShards, targetShards []string) (bool, error) { if ts.MigrationType() != binlogdatapb.MigrationType_TABLES { @@ -983,11 +980,12 @@ func getSourceAndTargetKeyRanges(sourceShards, targetShards []string) (*topodata return krs[0], nil } - // happily string sorting of shards also sorts them in the ascending order of key ranges in vitess + // Happily string sorting of shards also sorts them in the ascending order of + // key ranges in vitess. sort.Strings(sourceShards) sort.Strings(targetShards) getFullKeyRange := func(shards []string) (*topodatapb.KeyRange, error) { - // expect sorted shards + // Expect sorted shards. kr1, err := getKeyRange(sourceShards[0]) if err != nil { return nil, err @@ -1051,7 +1049,7 @@ func (ts *trafficSwitcher) switchTableReads(ctx context.Context, cells []string, // table -> sourceKeyspace.table // targetKeyspace.table -> sourceKeyspace.table // For forward migration, we add tablet type specific rules to redirect traffic to the target. - // For backward, we redirect to source + // For backward, we redirect to source. for _, servedType := range servedTypes { tt := strings.ToLower(servedType.String()) for _, table := range ts.Tables() { @@ -1192,7 +1190,7 @@ func (ts *trafficSwitcher) executeLockTablesOnSource(ctx context.Context) error for _, tableName := range ts.Tables() { sb.WriteString(fmt.Sprintf("%s READ,", sqlescape.EscapeID(tableName))) } - // trim extra trailing comma + // Trim extra trailing comma. lockStmt := sb.String()[:sb.Len()-1] return ts.ForAllSources(func(source *workflow.MigrationSource) error { @@ -1213,7 +1211,7 @@ func (ts *trafficSwitcher) executeLockTablesOnSource(ctx context.Context) error func (ts *trafficSwitcher) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error { ctx, cancel := context.WithTimeout(ctx, filteredReplicationWaitTime) defer cancel() - // source writes have been stopped, wait for all streams on targets to catch up + // Source writes have been stopped, wait for all streams on targets to catch up. if err := ts.ForAllUIDs(func(target *workflow.MigrationTarget, uid int32) error { ts.Logger().Infof("Before Catchup: uid: %d, target primary %s, target position %s, shard %s", uid, target.GetPrimary().AliasString(), target.Position, target.GetShard().String()) @@ -1354,7 +1352,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error source.GetPrimary().Alias, ts.ReverseWorkflowName(), target.Position) _, err := ts.VReplicationExec(ctx, source.GetPrimary().Alias, binlogplayer.CreateVReplicationState(ts.ReverseWorkflowName(), reverseBls, target.Position, - binlogplayer.BlpStopped, source.GetPrimary().DbName(), ts.workflowType, ts.workflowSubType)) + binlogdatapb.VReplicationWorkflowState_Stopped, source.GetPrimary().DbName(), ts.workflowType, ts.workflowSubType)) if err != nil { return err } diff --git a/go/vt/wrangler/traffic_switcher_test.go b/go/vt/wrangler/traffic_switcher_test.go index 1c59ae3140d..f7ae7998410 100644 --- a/go/vt/wrangler/traffic_switcher_test.go +++ b/go/vt/wrangler/traffic_switcher_test.go @@ -2257,11 +2257,11 @@ func getResult(id int, state string, keyspace string, shard string) *sqltypes.Re } func stoppedResult(id int) *sqltypes.Result { - return getResult(id, "Stopped", tpChoice.keyspace, tpChoice.shard) + return getResult(id, binlogdatapb.VReplicationWorkflowState_Stopped.String(), tpChoice.keyspace, tpChoice.shard) } func runningResult(id int) *sqltypes.Result { - return getResult(id, "Running", tpChoice.keyspace, tpChoice.shard) + return getResult(id, binlogdatapb.VReplicationWorkflowState_Running.String(), tpChoice.keyspace, tpChoice.shard) } func switchWrites(tmeT any) { diff --git a/go/vt/wrangler/vexec.go b/go/vt/wrangler/vexec.go index 75529a11044..ba840d50ca0 100644 --- a/go/vt/wrangler/vexec.go +++ b/go/vt/wrangler/vexec.go @@ -28,6 +28,7 @@ import ( "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/topo/topoproto" workflow2 "vitess.io/vitess/go/vt/vtctl/workflow" "google.golang.org/protobuf/encoding/prototext" @@ -37,14 +38,15 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/vterrors" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/topo" vtctldvexec "vitess.io/vitess/go/vt/vtctl/workflow/vexec" // renamed to avoid a collision with the vexec struct in this package - "vitess.io/vitess/go/vt/vterrors" ) const ( @@ -369,9 +371,9 @@ func (wr *Wrangler) getWorkflowActionQuery(action string) (string, error) { updateSQL := "update _vt.vreplication set state = %s" switch action { case "stop": - query = fmt.Sprintf(updateSQL, encodeString("Stopped")) + query = fmt.Sprintf(updateSQL, encodeString(binlogdatapb.VReplicationWorkflowState_Stopped.String())) case "start": - query = fmt.Sprintf(updateSQL, encodeString("Running")) + query = fmt.Sprintf(updateSQL, encodeString(binlogdatapb.VReplicationWorkflowState_Running.String())) case "update": // We don't use the SQL interface, so there's no query // and no error. @@ -390,7 +392,7 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, return nil, err } if action == "update" { - rpcReq, ok := rpcReq.(*tabletmanagerdatapb.UpdateVRWorkflowRequest) + rpcReq, ok := rpcReq.(*tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) if !ok { return nil, fmt.Errorf("invalid RPC request: %+v", rpcReq) } @@ -406,7 +408,7 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, } if !textutil.ValueIsSimulatedNull(rpcReq.TabletTypes) { changes = true - dryRunChanges.WriteString(fmt.Sprintf(" tablet_types=%q\n", strings.Join(rpcReq.TabletTypes, ","))) + dryRunChanges.WriteString(fmt.Sprintf(" tablet_types=%q\n", topoproto.MakeStringTypeCSV(rpcReq.TabletTypes))) } if !textutil.ValueIsSimulatedNull(rpcReq.OnDdl) { changes = true @@ -433,7 +435,7 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, return nil, nil } else { callback = func(ctx context.Context, tablet *topo.TabletInfo) (*querypb.QueryResult, error) { - res, err := wr.tmc.UpdateVRWorkflow(ctx, tablet.Tablet, rpcReq) + res, err := wr.tmc.UpdateVReplicationWorkflow(ctx, tablet.Tablet, rpcReq) if err != nil { return nil, err } @@ -552,7 +554,8 @@ func (wr *Wrangler) getReplicationStatusFromRow(ctx context.Context, row sqltype var err error var id int32 var timeUpdated, transactionTimestamp, timeHeartbeat, timeThrottled int64 - var state, dbName, pos, stopPos, message, tags, componentThrottled string + var dbName, pos, stopPos, message, tags, componentThrottled string + var state string var workflowType, workflowSubType int32 var deferSecondaryKeys bool var bls binlogdatapb.BinlogSource @@ -658,7 +661,7 @@ func (wr *Wrangler) getReplicationStatusFromRow(ctx context.Context, row sqltype return nil, "", err } - status.State = updateState(message, status.State, status.CopyState, timeUpdated) + status.State = updateState(message, binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[state]), status.CopyState, timeUpdated) return status, bls.Keyspace, nil } @@ -754,7 +757,7 @@ func (wr *Wrangler) getStreams(ctx context.Context, workflow, keyspace string) ( // All timestamps are in seconds since epoch lastTransactionTimestamp := status.TransactionTimestamp lastHeartbeatTime := status.TimeHeartbeat - if status.State == "Copying" { + if status.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { rsr.MaxVReplicationTransactionLag = math.MaxInt64 } else { if lastTransactionTimestamp == 0 /* no new events after copy */ || @@ -838,15 +841,15 @@ func (wr *Wrangler) ShowWorkflow(ctx context.Context, workflow, keyspace string) return replStatus, nil } -func updateState(message, state string, cs []copyState, timeUpdated int64) string { +func updateState(message string, state binlogdatapb.VReplicationWorkflowState, cs []copyState, timeUpdated int64) string { if strings.Contains(strings.ToLower(message), "error") { - state = "Error" - } else if state == "Running" && len(cs) > 0 { - state = "Copying" - } else if state == "Running" && int64(time.Now().Second())-timeUpdated > 10 /* seconds */ { - state = "Lagging" + state = binlogdatapb.VReplicationWorkflowState_Error + } else if state == binlogdatapb.VReplicationWorkflowState_Running && len(cs) > 0 { + state = binlogdatapb.VReplicationWorkflowState_Copying + } else if state == binlogdatapb.VReplicationWorkflowState_Running && int64(time.Now().Second())-timeUpdated > 10 /* seconds */ { + state = binlogdatapb.VReplicationWorkflowState_Lagging } - return state + return state.String() } func dumpStreamListAsJSON(replStatus *ReplicationStatusResult, wr *Wrangler) error { diff --git a/go/vt/wrangler/vexec_test.go b/go/vt/wrangler/vexec_test.go index e0af1e5fb22..031db6c7a08 100644 --- a/go/vt/wrangler/vexec_test.go +++ b/go/vt/wrangler/vexec_test.go @@ -32,6 +32,7 @@ import ( "vitess.io/vitess/go/vt/logutil" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) func TestVExec(t *testing.T) { @@ -172,11 +173,11 @@ func TestVExec(t *testing.T) { } func TestWorkflowStatusUpdate(t *testing.T) { - require.Equal(t, "Running", updateState("for vdiff", "Running", nil, int64(time.Now().Second()))) - require.Equal(t, "Running", updateState("", "Running", nil, int64(time.Now().Second()))) - require.Equal(t, "Lagging", updateState("", "Running", nil, int64(time.Now().Second())-100)) - require.Equal(t, "Copying", updateState("", "Running", []copyState{{Table: "t1", LastPK: "[[INT64(10)]]"}}, int64(time.Now().Second()))) - require.Equal(t, "Error", updateState("error: primary tablet not contactable", "Running", nil, 0)) + require.Equal(t, binlogdatapb.VReplicationWorkflowState_Running.String(), updateState("for vdiff", binlogdatapb.VReplicationWorkflowState_Running, nil, int64(time.Now().Second()))) + require.Equal(t, binlogdatapb.VReplicationWorkflowState_Running.String(), updateState("", binlogdatapb.VReplicationWorkflowState_Running, nil, int64(time.Now().Second()))) + require.Equal(t, binlogdatapb.VReplicationWorkflowState_Lagging.String(), updateState("", binlogdatapb.VReplicationWorkflowState_Running, nil, int64(time.Now().Second())-100)) + require.Equal(t, binlogdatapb.VReplicationWorkflowState_Copying.String(), updateState("", binlogdatapb.VReplicationWorkflowState_Running, []copyState{{Table: "t1", LastPK: "[[INT64(10)]]"}}, int64(time.Now().Second()))) + require.Equal(t, binlogdatapb.VReplicationWorkflowState_Error.String(), updateState("error: primary tablet not contactable", binlogdatapb.VReplicationWorkflowState_Running, nil, 0)) } func TestWorkflowListStreams(t *testing.T) { @@ -425,12 +426,12 @@ func TestVExecValidations(t *testing.T) { actions := []action{ { name: "start", - want: fmt.Sprintf(updateSQL, encodeString("Running")), + want: fmt.Sprintf(updateSQL, encodeString(binlogdatapb.VReplicationWorkflowState_Running.String())), expectedError: nil, }, { name: "stop", - want: fmt.Sprintf(updateSQL, encodeString("Stopped")), + want: fmt.Sprintf(updateSQL, encodeString(binlogdatapb.VReplicationWorkflowState_Stopped.String())), expectedError: nil, }, { @@ -472,7 +473,7 @@ func TestWorkflowUpdate(t *testing.T) { tests := []struct { name string cells []string - tabletTypes []string + tabletTypes []topodatapb.TabletType onDDL binlogdatapb.OnDDLAction output string wantErr string @@ -480,35 +481,35 @@ func TestWorkflowUpdate(t *testing.T) { { name: "no flags", cells: nullSlice, - tabletTypes: nullSlice, + tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, onDDL: nullOnDDL, wantErr: "no updates were provided; use --cells, --tablet-types, or --on-ddl to specify new values", }, { name: "only cells", cells: []string{"zone1"}, - tabletTypes: nullSlice, + tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, onDDL: nullOnDDL, output: "The following workflow fields will be updated:\n cells=\"zone1\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n", }, { name: "only tablet types", cells: nullSlice, - tabletTypes: []string{"primary", "replica"}, + tabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY, topodatapb.TabletType_REPLICA}, onDDL: nullOnDDL, output: "The following workflow fields will be updated:\n tablet_types=\"primary,replica\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n", }, { name: "only on-ddl", cells: nullSlice, - tabletTypes: nullSlice, + tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, onDDL: binlogdatapb.OnDDLAction_EXEC_IGNORE, output: "The following workflow fields will be updated:\n on_ddl=\"EXEC_IGNORE\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n", }, { name: "all flags", cells: []string{"zone1", "zone2"}, - tabletTypes: []string{"rdonly", "spare"}, + tabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_SPARE}, onDDL: binlogdatapb.OnDDLAction_EXEC, output: "The following workflow fields will be updated:\n cells=\"zone1,zone2\"\n tablet_types=\"rdonly,spare\"\n on_ddl=\"EXEC\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n", }, @@ -516,7 +517,7 @@ func TestWorkflowUpdate(t *testing.T) { for _, tcase := range tests { t.Run(tcase.name, func(t *testing.T) { - rpcReq := &tabletmanagerdatapb.UpdateVRWorkflowRequest{ + rpcReq := &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ Cells: tcase.cells, TabletTypes: tcase.tabletTypes, OnDdl: tcase.onDDL, diff --git a/go/vt/wrangler/workflow.go b/go/vt/wrangler/workflow.go index 0cbe1dc2062..40acaf1f55d 100644 --- a/go/vt/wrangler/workflow.go +++ b/go/vt/wrangler/workflow.go @@ -17,6 +17,7 @@ import ( "vitess.io/vitess/go/vt/vtctl/workflow" "vitess.io/vitess/go/vt/vtgate/evalengine" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -267,7 +268,7 @@ func (vrw *VReplicationWorkflow) GetStreamCount() (int64, int64, []*WorkflowErro if st.Pos == "" { continue } - if st.State == "Running" || st.State == "Copying" { + if st.State == binlogdatapb.VReplicationWorkflowState_Running.String() || st.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { started++ } } @@ -525,9 +526,9 @@ func (vrw *VReplicationWorkflow) canSwitch(keyspace, workflowName string) (reaso statuses := result.ShardStatuses[ksShard].PrimaryReplicationStatuses for _, st := range statuses { switch st.State { - case "Copying": + case binlogdatapb.VReplicationWorkflowState_Copying.String(): return cannotSwitchCopyIncomplete, nil - case "Error": + case binlogdatapb.VReplicationWorkflowState_Error.String(): return cannotSwitchError, nil } } diff --git a/go/vt/wrangler/workflow_test.go b/go/vt/wrangler/workflow_test.go index 73a720296a4..be3589a3f58 100644 --- a/go/vt/wrangler/workflow_test.go +++ b/go/vt/wrangler/workflow_test.go @@ -153,9 +153,9 @@ func TestCanSwitch(t *testing.T) { } testCases := []testCase{ - {"In Copy Phase", "Copying", 0, 0, regexp.MustCompile(cannotSwitchCopyIncomplete)}, - {"High Lag", "Running", 6, 5, regexp.MustCompile(strings.ReplaceAll(cannotSwitchHighLag, "%d", "(\\d+)"))}, - {"Acceptable Lag", "Running", 4, 5, nil}, + {"In Copy Phase", binlogdatapb.VReplicationWorkflowState_Copying.String(), 0, 0, regexp.MustCompile(cannotSwitchCopyIncomplete)}, + {"High Lag", binlogdatapb.VReplicationWorkflowState_Running.String(), 6, 5, regexp.MustCompile(strings.ReplaceAll(cannotSwitchHighLag, "%d", "(\\d+)"))}, + {"Acceptable Lag", binlogdatapb.VReplicationWorkflowState_Running.String(), 4, 5, nil}, } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 4ebcf272952..2141aa99c3a 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -224,6 +224,17 @@ enum VReplicationWorkflowSubType { Partial = 1; } +// VReplicationWorklfowState defines the valid states that a workflow can be in. +enum VReplicationWorkflowState { + Unknown = 0; + Init = 1; + Stopped = 2; + Copying = 3; + Running = 4; + Error = 5; + Lagging = 6; +} + // BinlogSource specifies the source and filter parameters for // Filtered Replication. KeyRange and Tables are legacy. Filter // is the new way to specify the filtering rules. diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index 74f42f549b5..ca28584ac27 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -34,6 +34,14 @@ import "vtrpc.proto"; // Data structures // +// This structure allows us to manage tablet selection preferences +// which are eventually passed to a TabletPicker. +enum TabletSelectionPreference { + ANY = 0; + INORDER = 1; + UNKNOWN = 3; // Don't change any existing value +} + message TableDefinition { // the table name string name = 1; @@ -504,6 +512,74 @@ message RestoreFromBackupResponse { logutil.Event event = 1; } +// +// VReplication related messages +// + +message CreateVReplicationWorkflowRequest { + string workflow = 1; + repeated binlogdata.BinlogSource binlog_source = 2; + // Optional parameters. + repeated string cells = 3; + // TabletTypes is the list of tablet types to use when selecting source tablets. + repeated topodata.TabletType tablet_types = 4; + TabletSelectionPreference tablet_selection_preference = 5; + binlogdata.VReplicationWorkflowType workflow_type = 6; + binlogdata.VReplicationWorkflowSubType workflow_sub_type = 7; + // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table + // copy finishes. + bool defer_secondary_keys = 8; + // AutoStart specifies if the workflow should be started when created. + bool auto_start = 9; + // Should the workflow stop after the copy phase. + bool stop_after_copy = 10; +} + +message CreateVReplicationWorkflowResponse { + query.QueryResult result = 1; +} + +message DeleteVReplicationWorkflowRequest { + string workflow = 1; +} + +message DeleteVReplicationWorkflowResponse { + query.QueryResult result = 1; +} + +message ReadVReplicationWorkflowRequest { + string workflow = 1; +} + +message ReadVReplicationWorkflowResponse { + string workflow = 2; + string cells = 3; + repeated topodata.TabletType tablet_types = 4; + TabletSelectionPreference tablet_selection_preference = 5; + string db_name = 6; + string tags = 7; + binlogdata.VReplicationWorkflowType workflow_type = 8; + binlogdata.VReplicationWorkflowSubType workflow_sub_type = 9; + bool defer_secondary_keys = 10; + message Stream { + int32 id = 1; + binlogdata.BinlogSource bls = 2; + string pos = 3; + string stop_pos = 4; + int64 max_tps = 5; + int64 max_replication_lag = 6; + vttime.Time time_updated = 7; + vttime.Time transaction_timestamp = 8; + binlogdata.VReplicationWorkflowState state = 9; + string message = 10; + int64 rows_copied = 11; + vttime.Time time_heartbeat = 12; + vttime.Time time_throttled = 13; + string component_throttled = 14; + } + repeated Stream streams = 11; +} + message VDiffRequest { string keyspace = 1; string workflow = 2; @@ -550,14 +626,16 @@ message VDiffOptions { VDiffReportOptions report_options = 3; } -message UpdateVRWorkflowRequest { +message UpdateVReplicationWorkflowRequest { string workflow = 1; repeated string cells = 2; - repeated string tablet_types = 3; - binlogdata.OnDDLAction on_ddl = 4; + repeated topodata.TabletType tablet_types = 3; + TabletSelectionPreference tablet_selection_preference = 4; + binlogdata.OnDDLAction on_ddl = 5; + binlogdata.VReplicationWorkflowState state = 6; } -message UpdateVRWorkflowResponse { +message UpdateVReplicationWorkflowResponse { query.QueryResult result = 1; } diff --git a/proto/tabletmanagerservice.proto b/proto/tabletmanagerservice.proto index 15d717fe933..20e6cbbc390 100644 --- a/proto/tabletmanagerservice.proto +++ b/proto/tabletmanagerservice.proto @@ -114,9 +114,12 @@ service TabletManager { rpc GetReplicas(tabletmanagerdata.GetReplicasRequest) returns (tabletmanagerdata.GetReplicasResponse) {}; // VReplication API + rpc CreateVReplicationWorkflow(tabletmanagerdata.CreateVReplicationWorkflowRequest) returns (tabletmanagerdata.CreateVReplicationWorkflowResponse) {}; + rpc DeleteVReplicationWorkflow(tabletmanagerdata.DeleteVReplicationWorkflowRequest) returns(tabletmanagerdata.DeleteVReplicationWorkflowResponse) {}; + rpc ReadVReplicationWorkflow(tabletmanagerdata.ReadVReplicationWorkflowRequest) returns(tabletmanagerdata.ReadVReplicationWorkflowResponse) {}; rpc VReplicationExec(tabletmanagerdata.VReplicationExecRequest) returns(tabletmanagerdata.VReplicationExecResponse) {}; rpc VReplicationWaitForPos(tabletmanagerdata.VReplicationWaitForPosRequest) returns(tabletmanagerdata.VReplicationWaitForPosResponse) {}; - rpc UpdateVRWorkflow(tabletmanagerdata.UpdateVRWorkflowRequest) returns(tabletmanagerdata.UpdateVRWorkflowResponse) {}; + rpc UpdateVReplicationWorkflow(tabletmanagerdata.UpdateVReplicationWorkflowRequest) returns(tabletmanagerdata.UpdateVReplicationWorkflowResponse) {}; // VDiff API rpc VDiff(tabletmanagerdata.VDiffRequest) returns(tabletmanagerdata.VDiffResponse) {}; diff --git a/proto/vtctldata.proto b/proto/vtctldata.proto index cf8c7af93ae..56a31bbb86a 100644 --- a/proto/vtctldata.proto +++ b/proto/vtctldata.proto @@ -95,6 +95,7 @@ message MaterializeSettings { string on_ddl = 13; // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. bool defer_secondary_keys = 14; + tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 15; } /* Data types for VtctldServer */ @@ -800,6 +801,8 @@ message GetWorkflowsRequest { string keyspace = 1; bool active_only = 2; bool name_only = 3; + // If you only want a specific workflow then set this field. + string workflow = 4; } message GetWorkflowsResponse { @@ -818,6 +821,59 @@ message InitShardPrimaryResponse { repeated logutil.Event events = 1; } +message MoveTablesCreateRequest { + // The necessary info gets passed on to each primary tablet involved + // in the workflow via the CreateVReplicationWorkflow tabletmanager RPC. + string workflow = 1; + string source_keyspace = 2; + string target_keyspace = 3; + repeated string cells = 4; + repeated topodata.TabletType tablet_types = 5; + tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 6; + repeated string source_shards = 7; + bool all_tables = 8; + repeated string include_tables = 9; + repeated string exclude_tables = 10; + // The name of the external cluster mounted in topo server. + string external_cluster_name = 11; + // SourceTimeZone is the time zone in which datetimes on the source were stored, provided as an option in MoveTables + string source_time_zone = 12; + // OnDdl specifies the action to be taken when a DDL is encountered. + string on_ddl = 13; + // StopAfterCopy specifies if vreplication should be stopped after copying. + bool stop_after_copy = 14; + // DropForeignKeys specifies if foreign key constraints should be elided on the target. + bool drop_foreign_keys = 15; + // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. + bool defer_secondary_keys = 16; + // Start the workflow after creating it. + bool auto_start = 17; +} + +message MoveTablesCreateResponse { + message TabletInfo { + topodata.TabletAlias tablet = 1; + // Created is set if the workflow was created on this tablet or not. + bool created = 2; + } + string summary = 1; + repeated TabletInfo details = 2; +} + +message MoveTablesCompleteRequest { + string workflow = 1; + string target_keyspace = 3; + bool keep_data = 4; + bool keep_routing_rules = 5; + bool rename_tables = 6; + bool dry_run = 7; +} + +message MoveTablesCompleteResponse { + string summary = 1; + repeated string dry_run_results = 2; +} + message PingTabletRequest { topodata.TabletAlias tablet_alias = 1; } @@ -1313,19 +1369,84 @@ message ValidateVSchemaResponse { map results_by_shard = 2; } +message WorkflowDeleteRequest { + string keyspace = 1; + string workflow = 2; + bool keep_data = 3; + bool keep_routing_rules = 4; +} + +message WorkflowDeleteResponse { + message TabletInfo { + topodata.TabletAlias tablet = 1; + // Delete is set if the workflow was deleted on this tablet. + bool deleted = 2; + } + string summary = 1; + repeated TabletInfo details = 2; +} + +message WorkflowStatusRequest { + string keyspace = 1; + string workflow = 2; +} + +message WorkflowStatusResponse { + message TableCopyState { + int64 rows_copied = 1; + int64 rows_total = 2; + float rows_percentage = 3; + int64 bytes_copied = 4; + int64 bytes_total = 5; + float bytes_percentage = 6; + } + message ShardStreamState { + int32 id = 1; + topodata.TabletAlias tablet = 2; + string source_shard = 3; + string position = 4; + string status = 5; + string info = 6; + } + message ShardStreams { + repeated ShardStreamState streams = 2; + } + // The key is keyspace/shard. + map table_copy_state = 1; + map shard_streams = 2; +} + +message WorkflowSwitchTrafficRequest { + string keyspace = 1; + string workflow = 2; + repeated string cells = 3; + repeated topodata.TabletType tablet_types = 4; + vttime.Duration max_replication_lag_allowed = 5; + bool enable_reverse_replication = 6; + int32 direction = 7; + vttime.Duration timeout = 8; + bool dry_run = 9; +} + +message WorkflowSwitchTrafficResponse { + string summary = 1; + string start_state = 2; + string current_state = 3; + repeated string dry_run_results = 4; +} + message WorkflowUpdateRequest { string keyspace = 1; // TabletRequest gets passed on to each primary tablet involved - // in the workflow via the UpdateVRWorkflow tabletmanager RPC. - tabletmanagerdata.UpdateVRWorkflowRequest tablet_request = 2; + // in the workflow via the UpdateVReplicationWorkflow tabletmanager RPC. + tabletmanagerdata.UpdateVReplicationWorkflowRequest tablet_request = 2; } message WorkflowUpdateResponse { message TabletInfo { - string tablet = 1; + topodata.TabletAlias tablet = 1; // Changed is true if any of the provided values were different - // than what was already stored. The value is based on the query - // result's RowsAffected being 0 or not. + // than what was already stored on this tablet. bool changed = 2; } string summary = 1; diff --git a/proto/vtctlservice.proto b/proto/vtctlservice.proto index 82a60433055..47444b08fc1 100644 --- a/proto/vtctlservice.proto +++ b/proto/vtctlservice.proto @@ -156,6 +156,12 @@ service Vtctld { // PlannedReparentShard or EmergencyReparentShard should be used in those // cases instead. rpc InitShardPrimary(vtctldata.InitShardPrimaryRequest) returns (vtctldata.InitShardPrimaryResponse) {}; + // MoveTablesCreate creates a workflow which moves one or more tables from a + // source keyspace to a target keyspace. + rpc MoveTablesCreate(vtctldata.MoveTablesCreateRequest) returns (vtctldata.WorkflowStatusResponse) {}; + // MoveTablesComplete completes the move and cleans up the workflow and + // its related artifacts. + rpc MoveTablesComplete(vtctldata.MoveTablesCompleteRequest) returns (vtctldata.MoveTablesCompleteResponse) {}; // PingTablet checks that the specified tablet is awake and responding to RPCs. // This command can be blocked by other in-flight operations. rpc PingTablet(vtctldata.PingTabletRequest) returns (vtctldata.PingTabletResponse) {}; @@ -293,6 +299,10 @@ service Vtctld { rpc ValidateVersionShard(vtctldata.ValidateVersionShardRequest) returns (vtctldata.ValidateVersionShardResponse) {}; // ValidateVSchema compares the schema of each primary tablet in "keyspace/shards..." to the vschema and errs if there are differences. rpc ValidateVSchema(vtctldata.ValidateVSchemaRequest) returns (vtctldata.ValidateVSchemaResponse) {}; + // WorkflowDelete deletes a vreplication workflow. + rpc WorkflowDelete(vtctldata.WorkflowDeleteRequest) returns (vtctldata.WorkflowDeleteResponse) {}; + rpc WorkflowStatus(vtctldata.WorkflowStatusRequest) returns (vtctldata.WorkflowStatusResponse) {}; + rpc WorkflowSwitchTraffic(vtctldata.WorkflowSwitchTrafficRequest) returns (vtctldata.WorkflowSwitchTrafficResponse) {}; // WorkflowUpdate updates the configuration of a vreplication workflow // using the provided updated parameters. rpc WorkflowUpdate(vtctldata.WorkflowUpdateRequest) returns (vtctldata.WorkflowUpdateResponse) {}; diff --git a/test/local_example.sh b/test/local_example.sh index 0274a44403c..2e1a003d5ba 100755 --- a/test/local_example.sh +++ b/test/local_example.sh @@ -56,7 +56,6 @@ for shard in "customer/0"; do done ./202_move_tables.sh -sleep 3 # required for now ./203_switch_reads.sh diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 0363cd3efca..b39cb64655a 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -16915,6 +16915,13 @@ export namespace topodata { /** Namespace tabletmanagerdata. */ export namespace tabletmanagerdata { + /** TabletSelectionPreference enum. */ + enum TabletSelectionPreference { + ANY = 0, + INORDER = 1, + UNKNOWN = 3 + } + /** Properties of a TableDefinition. */ interface ITableDefinition { @@ -26353,15534 +26360,16431 @@ export namespace tabletmanagerdata { public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VDiffRequest. */ - interface IVDiffRequest { - - /** VDiffRequest keyspace */ - keyspace?: (string|null); + /** Properties of a CreateVReplicationWorkflowRequest. */ + interface ICreateVReplicationWorkflowRequest { - /** VDiffRequest workflow */ + /** CreateVReplicationWorkflowRequest workflow */ workflow?: (string|null); - /** VDiffRequest action */ - action?: (string|null); + /** CreateVReplicationWorkflowRequest binlog_source */ + binlog_source?: (binlogdata.IBinlogSource[]|null); - /** VDiffRequest action_arg */ - action_arg?: (string|null); + /** CreateVReplicationWorkflowRequest cells */ + cells?: (string[]|null); - /** VDiffRequest vdiff_uuid */ - vdiff_uuid?: (string|null); + /** CreateVReplicationWorkflowRequest tablet_types */ + tablet_types?: (topodata.TabletType[]|null); - /** VDiffRequest options */ - options?: (tabletmanagerdata.IVDiffOptions|null); - } + /** CreateVReplicationWorkflowRequest tablet_selection_preference */ + tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); - /** Represents a VDiffRequest. */ - class VDiffRequest implements IVDiffRequest { + /** CreateVReplicationWorkflowRequest workflow_type */ + workflow_type?: (binlogdata.VReplicationWorkflowType|null); - /** - * Constructs a new VDiffRequest. - * @param [properties] Properties to set - */ - constructor(properties?: tabletmanagerdata.IVDiffRequest); + /** CreateVReplicationWorkflowRequest workflow_sub_type */ + workflow_sub_type?: (binlogdata.VReplicationWorkflowSubType|null); - /** VDiffRequest keyspace. */ - public keyspace: string; - - /** VDiffRequest workflow. */ - public workflow: string; - - /** VDiffRequest action. */ - public action: string; + /** CreateVReplicationWorkflowRequest defer_secondary_keys */ + defer_secondary_keys?: (boolean|null); - /** VDiffRequest action_arg. */ - public action_arg: string; + /** CreateVReplicationWorkflowRequest auto_start */ + auto_start?: (boolean|null); - /** VDiffRequest vdiff_uuid. */ - public vdiff_uuid: string; + /** CreateVReplicationWorkflowRequest stop_after_copy */ + stop_after_copy?: (boolean|null); + } - /** VDiffRequest options. */ - public options?: (tabletmanagerdata.IVDiffOptions|null); + /** Represents a CreateVReplicationWorkflowRequest. */ + class CreateVReplicationWorkflowRequest implements ICreateVReplicationWorkflowRequest { /** - * Creates a new VDiffRequest instance using the specified properties. + * Constructs a new CreateVReplicationWorkflowRequest. * @param [properties] Properties to set - * @returns VDiffRequest instance - */ - public static create(properties?: tabletmanagerdata.IVDiffRequest): tabletmanagerdata.VDiffRequest; - - /** - * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. - * @param message VDiffRequest message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: tabletmanagerdata.IVDiffRequest, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. - * @param message VDiffRequest message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffRequest, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Decodes a VDiffRequest message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns VDiffRequest - * @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): tabletmanagerdata.VDiffRequest; - - /** - * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns VDiffRequest - * @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)): tabletmanagerdata.VDiffRequest; - - /** - * Verifies a VDiffRequest 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 VDiffRequest message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns VDiffRequest - */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffRequest; - - /** - * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. - * @param message VDiffRequest - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: tabletmanagerdata.VDiffRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; - - /** - * Converts this VDiffRequest to JSON. - * @returns JSON object */ - public toJSON(): { [k: string]: any }; + constructor(properties?: tabletmanagerdata.ICreateVReplicationWorkflowRequest); - /** - * Gets the default type url for VDiffRequest - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } + /** CreateVReplicationWorkflowRequest workflow. */ + public workflow: string; - /** Properties of a VDiffResponse. */ - interface IVDiffResponse { + /** CreateVReplicationWorkflowRequest binlog_source. */ + public binlog_source: binlogdata.IBinlogSource[]; - /** VDiffResponse id */ - id?: (number|Long|null); + /** CreateVReplicationWorkflowRequest cells. */ + public cells: string[]; - /** VDiffResponse output */ - output?: (query.IQueryResult|null); + /** CreateVReplicationWorkflowRequest tablet_types. */ + public tablet_types: topodata.TabletType[]; - /** VDiffResponse vdiff_uuid */ - vdiff_uuid?: (string|null); - } + /** CreateVReplicationWorkflowRequest tablet_selection_preference. */ + public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; - /** Represents a VDiffResponse. */ - class VDiffResponse implements IVDiffResponse { + /** CreateVReplicationWorkflowRequest workflow_type. */ + public workflow_type: binlogdata.VReplicationWorkflowType; - /** - * Constructs a new VDiffResponse. - * @param [properties] Properties to set - */ - constructor(properties?: tabletmanagerdata.IVDiffResponse); + /** CreateVReplicationWorkflowRequest workflow_sub_type. */ + public workflow_sub_type: binlogdata.VReplicationWorkflowSubType; - /** VDiffResponse id. */ - public id: (number|Long); + /** CreateVReplicationWorkflowRequest defer_secondary_keys. */ + public defer_secondary_keys: boolean; - /** VDiffResponse output. */ - public output?: (query.IQueryResult|null); + /** CreateVReplicationWorkflowRequest auto_start. */ + public auto_start: boolean; - /** VDiffResponse vdiff_uuid. */ - public vdiff_uuid: string; + /** CreateVReplicationWorkflowRequest stop_after_copy. */ + public stop_after_copy: boolean; /** - * Creates a new VDiffResponse instance using the specified properties. + * Creates a new CreateVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set - * @returns VDiffResponse instance + * @returns CreateVReplicationWorkflowRequest instance */ - public static create(properties?: tabletmanagerdata.IVDiffResponse): tabletmanagerdata.VDiffResponse; + public static create(properties?: tabletmanagerdata.ICreateVReplicationWorkflowRequest): tabletmanagerdata.CreateVReplicationWorkflowRequest; /** - * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. - * @param message VDiffResponse message or plain object to encode + * Encodes the specified CreateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowRequest.verify|verify} messages. + * @param message CreateVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IVDiffResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.ICreateVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. - * @param message VDiffResponse message or plain object to encode + * Encodes the specified CreateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowRequest.verify|verify} messages. + * @param message CreateVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.ICreateVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VDiffResponse message from the specified reader or buffer. + * Decodes a CreateVReplicationWorkflowRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VDiffResponse + * @returns CreateVReplicationWorkflowRequest * @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): tabletmanagerdata.VDiffResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.CreateVReplicationWorkflowRequest; /** - * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VDiffResponse + * @returns CreateVReplicationWorkflowRequest * @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)): tabletmanagerdata.VDiffResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.CreateVReplicationWorkflowRequest; /** - * Verifies a VDiffResponse message. + * Verifies a CreateVReplicationWorkflowRequest 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 VDiffResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VDiffResponse + * @returns CreateVReplicationWorkflowRequest */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffResponse; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.CreateVReplicationWorkflowRequest; /** - * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. - * @param message VDiffResponse + * Creates a plain object from a CreateVReplicationWorkflowRequest message. Also converts values to other types if specified. + * @param message CreateVReplicationWorkflowRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.VDiffResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.CreateVReplicationWorkflowRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VDiffResponse to JSON. + * Converts this CreateVReplicationWorkflowRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VDiffResponse + * Gets the default type url for CreateVReplicationWorkflowRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VDiffPickerOptions. */ - interface IVDiffPickerOptions { - - /** VDiffPickerOptions tablet_types */ - tablet_types?: (string|null); - - /** VDiffPickerOptions source_cell */ - source_cell?: (string|null); + /** Properties of a CreateVReplicationWorkflowResponse. */ + interface ICreateVReplicationWorkflowResponse { - /** VDiffPickerOptions target_cell */ - target_cell?: (string|null); + /** CreateVReplicationWorkflowResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a VDiffPickerOptions. */ - class VDiffPickerOptions implements IVDiffPickerOptions { + /** Represents a CreateVReplicationWorkflowResponse. */ + class CreateVReplicationWorkflowResponse implements ICreateVReplicationWorkflowResponse { /** - * Constructs a new VDiffPickerOptions. + * Constructs a new CreateVReplicationWorkflowResponse. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IVDiffPickerOptions); - - /** VDiffPickerOptions tablet_types. */ - public tablet_types: string; - - /** VDiffPickerOptions source_cell. */ - public source_cell: string; + constructor(properties?: tabletmanagerdata.ICreateVReplicationWorkflowResponse); - /** VDiffPickerOptions target_cell. */ - public target_cell: string; + /** CreateVReplicationWorkflowResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new VDiffPickerOptions instance using the specified properties. + * Creates a new CreateVReplicationWorkflowResponse instance using the specified properties. * @param [properties] Properties to set - * @returns VDiffPickerOptions instance + * @returns CreateVReplicationWorkflowResponse instance */ - public static create(properties?: tabletmanagerdata.IVDiffPickerOptions): tabletmanagerdata.VDiffPickerOptions; + public static create(properties?: tabletmanagerdata.ICreateVReplicationWorkflowResponse): tabletmanagerdata.CreateVReplicationWorkflowResponse; /** - * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. - * @param message VDiffPickerOptions message or plain object to encode + * Encodes the specified CreateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowResponse.verify|verify} messages. + * @param message CreateVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IVDiffPickerOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.ICreateVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. - * @param message VDiffPickerOptions message or plain object to encode + * Encodes the specified CreateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowResponse.verify|verify} messages. + * @param message CreateVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffPickerOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.ICreateVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer. + * Decodes a CreateVReplicationWorkflowResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VDiffPickerOptions + * @returns CreateVReplicationWorkflowResponse * @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): tabletmanagerdata.VDiffPickerOptions; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.CreateVReplicationWorkflowResponse; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. + * Decodes a CreateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VDiffPickerOptions + * @returns CreateVReplicationWorkflowResponse * @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)): tabletmanagerdata.VDiffPickerOptions; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.CreateVReplicationWorkflowResponse; /** - * Verifies a VDiffPickerOptions message. + * Verifies a CreateVReplicationWorkflowResponse 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 VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. + * Creates a CreateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VDiffPickerOptions + * @returns CreateVReplicationWorkflowResponse */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffPickerOptions; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.CreateVReplicationWorkflowResponse; /** - * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. - * @param message VDiffPickerOptions + * Creates a plain object from a CreateVReplicationWorkflowResponse message. Also converts values to other types if specified. + * @param message CreateVReplicationWorkflowResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.VDiffPickerOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.CreateVReplicationWorkflowResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VDiffPickerOptions to JSON. + * Converts this CreateVReplicationWorkflowResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VDiffPickerOptions + * Gets the default type url for CreateVReplicationWorkflowResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VDiffReportOptions. */ - interface IVDiffReportOptions { - - /** VDiffReportOptions only_pks */ - only_pks?: (boolean|null); - - /** VDiffReportOptions debug_query */ - debug_query?: (boolean|null); + /** Properties of a DeleteVReplicationWorkflowRequest. */ + interface IDeleteVReplicationWorkflowRequest { - /** VDiffReportOptions format */ - format?: (string|null); + /** DeleteVReplicationWorkflowRequest workflow */ + workflow?: (string|null); } - /** Represents a VDiffReportOptions. */ - class VDiffReportOptions implements IVDiffReportOptions { + /** Represents a DeleteVReplicationWorkflowRequest. */ + class DeleteVReplicationWorkflowRequest implements IDeleteVReplicationWorkflowRequest { /** - * Constructs a new VDiffReportOptions. + * Constructs a new DeleteVReplicationWorkflowRequest. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IVDiffReportOptions); + constructor(properties?: tabletmanagerdata.IDeleteVReplicationWorkflowRequest); - /** VDiffReportOptions only_pks. */ - public only_pks: boolean; - - /** VDiffReportOptions debug_query. */ - public debug_query: boolean; - - /** VDiffReportOptions format. */ - public format: string; + /** DeleteVReplicationWorkflowRequest workflow. */ + public workflow: string; /** - * Creates a new VDiffReportOptions instance using the specified properties. + * Creates a new DeleteVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set - * @returns VDiffReportOptions instance + * @returns DeleteVReplicationWorkflowRequest instance */ - public static create(properties?: tabletmanagerdata.IVDiffReportOptions): tabletmanagerdata.VDiffReportOptions; + public static create(properties?: tabletmanagerdata.IDeleteVReplicationWorkflowRequest): tabletmanagerdata.DeleteVReplicationWorkflowRequest; /** - * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. - * @param message VDiffReportOptions message or plain object to encode + * Encodes the specified DeleteVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowRequest.verify|verify} messages. + * @param message DeleteVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IVDiffReportOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IDeleteVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. - * @param message VDiffReportOptions message or plain object to encode + * Encodes the specified DeleteVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowRequest.verify|verify} messages. + * @param message DeleteVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffReportOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IDeleteVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer. + * Decodes a DeleteVReplicationWorkflowRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VDiffReportOptions + * @returns DeleteVReplicationWorkflowRequest * @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): tabletmanagerdata.VDiffReportOptions; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.DeleteVReplicationWorkflowRequest; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. + * Decodes a DeleteVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VDiffReportOptions + * @returns DeleteVReplicationWorkflowRequest * @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)): tabletmanagerdata.VDiffReportOptions; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.DeleteVReplicationWorkflowRequest; /** - * Verifies a VDiffReportOptions message. + * Verifies a DeleteVReplicationWorkflowRequest 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 VDiffReportOptions message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VDiffReportOptions + * @returns DeleteVReplicationWorkflowRequest */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffReportOptions; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.DeleteVReplicationWorkflowRequest; /** - * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. - * @param message VDiffReportOptions + * Creates a plain object from a DeleteVReplicationWorkflowRequest message. Also converts values to other types if specified. + * @param message DeleteVReplicationWorkflowRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.VDiffReportOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.DeleteVReplicationWorkflowRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VDiffReportOptions to JSON. + * Converts this DeleteVReplicationWorkflowRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VDiffReportOptions + * Gets the default type url for DeleteVReplicationWorkflowRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VDiffCoreOptions. */ - interface IVDiffCoreOptions { - - /** VDiffCoreOptions tables */ - tables?: (string|null); - - /** VDiffCoreOptions auto_retry */ - auto_retry?: (boolean|null); - - /** VDiffCoreOptions max_rows */ - max_rows?: (number|Long|null); - - /** VDiffCoreOptions checksum */ - checksum?: (boolean|null); - - /** VDiffCoreOptions sample_pct */ - sample_pct?: (number|Long|null); - - /** VDiffCoreOptions timeout_seconds */ - timeout_seconds?: (number|Long|null); + /** Properties of a DeleteVReplicationWorkflowResponse. */ + interface IDeleteVReplicationWorkflowResponse { - /** VDiffCoreOptions max_extra_rows_to_compare */ - max_extra_rows_to_compare?: (number|Long|null); - - /** VDiffCoreOptions update_table_stats */ - update_table_stats?: (boolean|null); + /** DeleteVReplicationWorkflowResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a VDiffCoreOptions. */ - class VDiffCoreOptions implements IVDiffCoreOptions { + /** Represents a DeleteVReplicationWorkflowResponse. */ + class DeleteVReplicationWorkflowResponse implements IDeleteVReplicationWorkflowResponse { /** - * Constructs a new VDiffCoreOptions. + * Constructs a new DeleteVReplicationWorkflowResponse. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IVDiffCoreOptions); - - /** VDiffCoreOptions tables. */ - public tables: string; - - /** VDiffCoreOptions auto_retry. */ - public auto_retry: boolean; - - /** VDiffCoreOptions max_rows. */ - public max_rows: (number|Long); - - /** VDiffCoreOptions checksum. */ - public checksum: boolean; - - /** VDiffCoreOptions sample_pct. */ - public sample_pct: (number|Long); + constructor(properties?: tabletmanagerdata.IDeleteVReplicationWorkflowResponse); - /** VDiffCoreOptions timeout_seconds. */ - public timeout_seconds: (number|Long); - - /** VDiffCoreOptions max_extra_rows_to_compare. */ - public max_extra_rows_to_compare: (number|Long); - - /** VDiffCoreOptions update_table_stats. */ - public update_table_stats: boolean; + /** DeleteVReplicationWorkflowResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new VDiffCoreOptions instance using the specified properties. + * Creates a new DeleteVReplicationWorkflowResponse instance using the specified properties. * @param [properties] Properties to set - * @returns VDiffCoreOptions instance + * @returns DeleteVReplicationWorkflowResponse instance */ - public static create(properties?: tabletmanagerdata.IVDiffCoreOptions): tabletmanagerdata.VDiffCoreOptions; + public static create(properties?: tabletmanagerdata.IDeleteVReplicationWorkflowResponse): tabletmanagerdata.DeleteVReplicationWorkflowResponse; /** - * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. - * @param message VDiffCoreOptions message or plain object to encode + * Encodes the specified DeleteVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowResponse.verify|verify} messages. + * @param message DeleteVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IVDiffCoreOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IDeleteVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. - * @param message VDiffCoreOptions message or plain object to encode + * Encodes the specified DeleteVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowResponse.verify|verify} messages. + * @param message DeleteVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffCoreOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IDeleteVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer. + * Decodes a DeleteVReplicationWorkflowResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VDiffCoreOptions + * @returns DeleteVReplicationWorkflowResponse * @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): tabletmanagerdata.VDiffCoreOptions; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.DeleteVReplicationWorkflowResponse; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. + * Decodes a DeleteVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VDiffCoreOptions + * @returns DeleteVReplicationWorkflowResponse * @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)): tabletmanagerdata.VDiffCoreOptions; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.DeleteVReplicationWorkflowResponse; /** - * Verifies a VDiffCoreOptions message. + * Verifies a DeleteVReplicationWorkflowResponse 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 VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VDiffCoreOptions + * @returns DeleteVReplicationWorkflowResponse */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffCoreOptions; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.DeleteVReplicationWorkflowResponse; /** - * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. - * @param message VDiffCoreOptions + * Creates a plain object from a DeleteVReplicationWorkflowResponse message. Also converts values to other types if specified. + * @param message DeleteVReplicationWorkflowResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.VDiffCoreOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.DeleteVReplicationWorkflowResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VDiffCoreOptions to JSON. + * Converts this DeleteVReplicationWorkflowResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VDiffCoreOptions + * Gets the default type url for DeleteVReplicationWorkflowResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VDiffOptions. */ - interface IVDiffOptions { - - /** VDiffOptions picker_options */ - picker_options?: (tabletmanagerdata.IVDiffPickerOptions|null); + /** Properties of a ReadVReplicationWorkflowRequest. */ + interface IReadVReplicationWorkflowRequest { - /** VDiffOptions core_options */ - core_options?: (tabletmanagerdata.IVDiffCoreOptions|null); - - /** VDiffOptions report_options */ - report_options?: (tabletmanagerdata.IVDiffReportOptions|null); + /** ReadVReplicationWorkflowRequest workflow */ + workflow?: (string|null); } - /** Represents a VDiffOptions. */ - class VDiffOptions implements IVDiffOptions { + /** Represents a ReadVReplicationWorkflowRequest. */ + class ReadVReplicationWorkflowRequest implements IReadVReplicationWorkflowRequest { /** - * Constructs a new VDiffOptions. + * Constructs a new ReadVReplicationWorkflowRequest. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IVDiffOptions); - - /** VDiffOptions picker_options. */ - public picker_options?: (tabletmanagerdata.IVDiffPickerOptions|null); + constructor(properties?: tabletmanagerdata.IReadVReplicationWorkflowRequest); - /** VDiffOptions core_options. */ - public core_options?: (tabletmanagerdata.IVDiffCoreOptions|null); - - /** VDiffOptions report_options. */ - public report_options?: (tabletmanagerdata.IVDiffReportOptions|null); + /** ReadVReplicationWorkflowRequest workflow. */ + public workflow: string; /** - * Creates a new VDiffOptions instance using the specified properties. + * Creates a new ReadVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set - * @returns VDiffOptions instance + * @returns ReadVReplicationWorkflowRequest instance */ - public static create(properties?: tabletmanagerdata.IVDiffOptions): tabletmanagerdata.VDiffOptions; + public static create(properties?: tabletmanagerdata.IReadVReplicationWorkflowRequest): tabletmanagerdata.ReadVReplicationWorkflowRequest; /** - * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. - * @param message VDiffOptions message or plain object to encode + * Encodes the specified ReadVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * @param message ReadVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IVDiffOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IReadVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. - * @param message VDiffOptions message or plain object to encode + * Encodes the specified ReadVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * @param message ReadVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IVDiffOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IReadVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VDiffOptions message from the specified reader or buffer. + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VDiffOptions + * @returns ReadVReplicationWorkflowRequest * @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): tabletmanagerdata.VDiffOptions; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ReadVReplicationWorkflowRequest; /** - * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VDiffOptions + * @returns ReadVReplicationWorkflowRequest * @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)): tabletmanagerdata.VDiffOptions; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ReadVReplicationWorkflowRequest; /** - * Verifies a VDiffOptions message. + * Verifies a ReadVReplicationWorkflowRequest 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 VDiffOptions message from a plain object. Also converts values to their respective internal types. + * Creates a ReadVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VDiffOptions + * @returns ReadVReplicationWorkflowRequest */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffOptions; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ReadVReplicationWorkflowRequest; /** - * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. - * @param message VDiffOptions + * Creates a plain object from a ReadVReplicationWorkflowRequest message. Also converts values to other types if specified. + * @param message ReadVReplicationWorkflowRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.VDiffOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.ReadVReplicationWorkflowRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VDiffOptions to JSON. + * Converts this ReadVReplicationWorkflowRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VDiffOptions + * Gets the default type url for ReadVReplicationWorkflowRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateVRWorkflowRequest. */ - interface IUpdateVRWorkflowRequest { + /** Properties of a ReadVReplicationWorkflowResponse. */ + interface IReadVReplicationWorkflowResponse { - /** UpdateVRWorkflowRequest workflow */ + /** ReadVReplicationWorkflowResponse workflow */ workflow?: (string|null); - /** UpdateVRWorkflowRequest cells */ - cells?: (string[]|null); + /** ReadVReplicationWorkflowResponse cells */ + cells?: (string|null); - /** UpdateVRWorkflowRequest tablet_types */ - tablet_types?: (string[]|null); + /** ReadVReplicationWorkflowResponse tablet_types */ + tablet_types?: (topodata.TabletType[]|null); - /** UpdateVRWorkflowRequest on_ddl */ - on_ddl?: (binlogdata.OnDDLAction|null); + /** ReadVReplicationWorkflowResponse tablet_selection_preference */ + tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); + + /** ReadVReplicationWorkflowResponse db_name */ + db_name?: (string|null); + + /** ReadVReplicationWorkflowResponse tags */ + tags?: (string|null); + + /** ReadVReplicationWorkflowResponse workflow_type */ + workflow_type?: (binlogdata.VReplicationWorkflowType|null); + + /** ReadVReplicationWorkflowResponse workflow_sub_type */ + workflow_sub_type?: (binlogdata.VReplicationWorkflowSubType|null); + + /** ReadVReplicationWorkflowResponse defer_secondary_keys */ + defer_secondary_keys?: (boolean|null); + + /** ReadVReplicationWorkflowResponse streams */ + streams?: (tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream[]|null); } - /** Represents an UpdateVRWorkflowRequest. */ - class UpdateVRWorkflowRequest implements IUpdateVRWorkflowRequest { + /** Represents a ReadVReplicationWorkflowResponse. */ + class ReadVReplicationWorkflowResponse implements IReadVReplicationWorkflowResponse { /** - * Constructs a new UpdateVRWorkflowRequest. + * Constructs a new ReadVReplicationWorkflowResponse. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IUpdateVRWorkflowRequest); + constructor(properties?: tabletmanagerdata.IReadVReplicationWorkflowResponse); - /** UpdateVRWorkflowRequest workflow. */ + /** ReadVReplicationWorkflowResponse workflow. */ public workflow: string; - /** UpdateVRWorkflowRequest cells. */ - public cells: string[]; + /** ReadVReplicationWorkflowResponse cells. */ + public cells: string; - /** UpdateVRWorkflowRequest tablet_types. */ - public tablet_types: string[]; + /** ReadVReplicationWorkflowResponse tablet_types. */ + public tablet_types: topodata.TabletType[]; - /** UpdateVRWorkflowRequest on_ddl. */ - public on_ddl: binlogdata.OnDDLAction; + /** ReadVReplicationWorkflowResponse tablet_selection_preference. */ + public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; + + /** ReadVReplicationWorkflowResponse db_name. */ + public db_name: string; + + /** ReadVReplicationWorkflowResponse tags. */ + public tags: string; + + /** ReadVReplicationWorkflowResponse workflow_type. */ + public workflow_type: binlogdata.VReplicationWorkflowType; + + /** ReadVReplicationWorkflowResponse workflow_sub_type. */ + public workflow_sub_type: binlogdata.VReplicationWorkflowSubType; + + /** ReadVReplicationWorkflowResponse defer_secondary_keys. */ + public defer_secondary_keys: boolean; + + /** ReadVReplicationWorkflowResponse streams. */ + public streams: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream[]; /** - * Creates a new UpdateVRWorkflowRequest instance using the specified properties. + * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateVRWorkflowRequest instance + * @returns ReadVReplicationWorkflowResponse instance */ - public static create(properties?: tabletmanagerdata.IUpdateVRWorkflowRequest): tabletmanagerdata.UpdateVRWorkflowRequest; + public static create(properties?: tabletmanagerdata.IReadVReplicationWorkflowResponse): tabletmanagerdata.ReadVReplicationWorkflowResponse; /** - * Encodes the specified UpdateVRWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowRequest.verify|verify} messages. - * @param message UpdateVRWorkflowRequest message or plain object to encode + * Encodes the specified ReadVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * @param message ReadVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IUpdateVRWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IReadVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateVRWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowRequest.verify|verify} messages. - * @param message UpdateVRWorkflowRequest message or plain object to encode + * Encodes the specified ReadVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * @param message ReadVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IUpdateVRWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IReadVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateVRWorkflowRequest message from the specified reader or buffer. + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateVRWorkflowRequest + * @returns ReadVReplicationWorkflowResponse * @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): tabletmanagerdata.UpdateVRWorkflowRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ReadVReplicationWorkflowResponse; /** - * Decodes an UpdateVRWorkflowRequest message from the specified reader or buffer, length delimited. + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateVRWorkflowRequest + * @returns ReadVReplicationWorkflowResponse * @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)): tabletmanagerdata.UpdateVRWorkflowRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ReadVReplicationWorkflowResponse; /** - * Verifies an UpdateVRWorkflowRequest message. + * Verifies a ReadVReplicationWorkflowResponse 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 UpdateVRWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReadVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateVRWorkflowRequest + * @returns ReadVReplicationWorkflowResponse */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVRWorkflowRequest; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ReadVReplicationWorkflowResponse; /** - * Creates a plain object from an UpdateVRWorkflowRequest message. Also converts values to other types if specified. - * @param message UpdateVRWorkflowRequest + * Creates a plain object from a ReadVReplicationWorkflowResponse message. Also converts values to other types if specified. + * @param message ReadVReplicationWorkflowResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.UpdateVRWorkflowRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.ReadVReplicationWorkflowResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateVRWorkflowRequest to JSON. + * Converts this ReadVReplicationWorkflowResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateVRWorkflowRequest + * Gets the default type url for ReadVReplicationWorkflowResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateVRWorkflowResponse. */ - interface IUpdateVRWorkflowResponse { + namespace ReadVReplicationWorkflowResponse { - /** UpdateVRWorkflowResponse result */ - result?: (query.IQueryResult|null); - } + /** Properties of a Stream. */ + interface IStream { - /** Represents an UpdateVRWorkflowResponse. */ - class UpdateVRWorkflowResponse implements IUpdateVRWorkflowResponse { + /** Stream id */ + id?: (number|null); - /** - * Constructs a new UpdateVRWorkflowResponse. - * @param [properties] Properties to set - */ - constructor(properties?: tabletmanagerdata.IUpdateVRWorkflowResponse); + /** Stream bls */ + bls?: (binlogdata.IBinlogSource|null); - /** UpdateVRWorkflowResponse result. */ - public result?: (query.IQueryResult|null); + /** Stream pos */ + pos?: (string|null); - /** - * Creates a new UpdateVRWorkflowResponse instance using the specified properties. - * @param [properties] Properties to set - * @returns UpdateVRWorkflowResponse instance - */ - public static create(properties?: tabletmanagerdata.IUpdateVRWorkflowResponse): tabletmanagerdata.UpdateVRWorkflowResponse; + /** Stream stop_pos */ + stop_pos?: (string|null); - /** - * Encodes the specified UpdateVRWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowResponse.verify|verify} messages. - * @param message UpdateVRWorkflowResponse message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: tabletmanagerdata.IUpdateVRWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; + /** Stream max_tps */ + max_tps?: (number|Long|null); - /** - * Encodes the specified UpdateVRWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowResponse.verify|verify} messages. - * @param message UpdateVRWorkflowResponse message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: tabletmanagerdata.IUpdateVRWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; + /** Stream max_replication_lag */ + max_replication_lag?: (number|Long|null); - /** - * Decodes an UpdateVRWorkflowResponse message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns UpdateVRWorkflowResponse - * @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): tabletmanagerdata.UpdateVRWorkflowResponse; + /** Stream time_updated */ + time_updated?: (vttime.ITime|null); - /** - * Decodes an UpdateVRWorkflowResponse message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns UpdateVRWorkflowResponse - * @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)): tabletmanagerdata.UpdateVRWorkflowResponse; + /** Stream transaction_timestamp */ + transaction_timestamp?: (vttime.ITime|null); - /** - * Verifies an UpdateVRWorkflowResponse 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); + /** Stream state */ + state?: (binlogdata.VReplicationWorkflowState|null); - /** - * Creates an UpdateVRWorkflowResponse message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns UpdateVRWorkflowResponse - */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVRWorkflowResponse; + /** Stream message */ + message?: (string|null); - /** - * Creates a plain object from an UpdateVRWorkflowResponse message. Also converts values to other types if specified. - * @param message UpdateVRWorkflowResponse - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: tabletmanagerdata.UpdateVRWorkflowResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** Stream rows_copied */ + rows_copied?: (number|Long|null); - /** - * Converts this UpdateVRWorkflowResponse to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** Stream time_heartbeat */ + time_heartbeat?: (vttime.ITime|null); - /** - * Gets the default type url for UpdateVRWorkflowResponse - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; + /** Stream time_throttled */ + time_throttled?: (vttime.ITime|null); + + /** Stream component_throttled */ + component_throttled?: (string|null); + } + + /** Represents a Stream. */ + class Stream implements IStream { + + /** + * Constructs a new Stream. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream); + + /** Stream id. */ + public id: number; + + /** Stream bls. */ + public bls?: (binlogdata.IBinlogSource|null); + + /** Stream pos. */ + public pos: string; + + /** Stream stop_pos. */ + public stop_pos: string; + + /** Stream max_tps. */ + public max_tps: (number|Long); + + /** Stream max_replication_lag. */ + public max_replication_lag: (number|Long); + + /** Stream time_updated. */ + public time_updated?: (vttime.ITime|null); + + /** Stream transaction_timestamp. */ + public transaction_timestamp?: (vttime.ITime|null); + + /** Stream state. */ + public state: binlogdata.VReplicationWorkflowState; + + /** Stream message. */ + public message: string; + + /** Stream rows_copied. */ + public rows_copied: (number|Long); + + /** Stream time_heartbeat. */ + public time_heartbeat?: (vttime.ITime|null); + + /** Stream time_throttled. */ + public time_throttled?: (vttime.ITime|null); + + /** Stream component_throttled. */ + public component_throttled: string; + + /** + * Creates a new Stream instance using the specified properties. + * @param [properties] Properties to set + * @returns Stream instance + */ + public static create(properties?: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream): tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream; + + /** + * Encodes the specified Stream message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @param message Stream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Stream message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @param message Stream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Stream message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Stream + * @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): tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream; + + /** + * Decodes a Stream message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Stream + * @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)): tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream; + + /** + * Verifies a Stream 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 Stream message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Stream + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream; + + /** + * Creates a plain object from a Stream message. Also converts values to other types if specified. + * @param message Stream + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Stream to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Stream + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } } - /** Properties of a ResetSequencesRequest. */ - interface IResetSequencesRequest { + /** Properties of a VDiffRequest. */ + interface IVDiffRequest { - /** ResetSequencesRequest tables */ - tables?: (string[]|null); + /** VDiffRequest keyspace */ + keyspace?: (string|null); + + /** VDiffRequest workflow */ + workflow?: (string|null); + + /** VDiffRequest action */ + action?: (string|null); + + /** VDiffRequest action_arg */ + action_arg?: (string|null); + + /** VDiffRequest vdiff_uuid */ + vdiff_uuid?: (string|null); + + /** VDiffRequest options */ + options?: (tabletmanagerdata.IVDiffOptions|null); } - /** Represents a ResetSequencesRequest. */ - class ResetSequencesRequest implements IResetSequencesRequest { + /** Represents a VDiffRequest. */ + class VDiffRequest implements IVDiffRequest { /** - * Constructs a new ResetSequencesRequest. + * Constructs a new VDiffRequest. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IResetSequencesRequest); + constructor(properties?: tabletmanagerdata.IVDiffRequest); - /** ResetSequencesRequest tables. */ - public tables: string[]; + /** VDiffRequest keyspace. */ + public keyspace: string; + + /** VDiffRequest workflow. */ + public workflow: string; + + /** VDiffRequest action. */ + public action: string; + + /** VDiffRequest action_arg. */ + public action_arg: string; + + /** VDiffRequest vdiff_uuid. */ + public vdiff_uuid: string; + + /** VDiffRequest options. */ + public options?: (tabletmanagerdata.IVDiffOptions|null); /** - * Creates a new ResetSequencesRequest instance using the specified properties. + * Creates a new VDiffRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ResetSequencesRequest instance + * @returns VDiffRequest instance */ - public static create(properties?: tabletmanagerdata.IResetSequencesRequest): tabletmanagerdata.ResetSequencesRequest; + public static create(properties?: tabletmanagerdata.IVDiffRequest): tabletmanagerdata.VDiffRequest; /** - * Encodes the specified ResetSequencesRequest message. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. - * @param message ResetSequencesRequest message or plain object to encode + * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * @param message VDiffRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IResetSequencesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ResetSequencesRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. - * @param message ResetSequencesRequest message or plain object to encode + * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * @param message VDiffRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IResetSequencesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ResetSequencesRequest message from the specified reader or buffer. + * Decodes a VDiffRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ResetSequencesRequest + * @returns VDiffRequest * @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): tabletmanagerdata.ResetSequencesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffRequest; /** - * Decodes a ResetSequencesRequest message from the specified reader or buffer, length delimited. + * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ResetSequencesRequest + * @returns VDiffRequest * @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)): tabletmanagerdata.ResetSequencesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffRequest; /** - * Verifies a ResetSequencesRequest message. + * Verifies a VDiffRequest 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 ResetSequencesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ResetSequencesRequest + * @returns VDiffRequest */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ResetSequencesRequest; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffRequest; /** - * Creates a plain object from a ResetSequencesRequest message. Also converts values to other types if specified. - * @param message ResetSequencesRequest + * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. + * @param message VDiffRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.ResetSequencesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ResetSequencesRequest to JSON. + * Converts this VDiffRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ResetSequencesRequest + * Gets the default type url for VDiffRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ResetSequencesResponse. */ - interface IResetSequencesResponse { + /** Properties of a VDiffResponse. */ + interface IVDiffResponse { + + /** VDiffResponse id */ + id?: (number|Long|null); + + /** VDiffResponse output */ + output?: (query.IQueryResult|null); + + /** VDiffResponse vdiff_uuid */ + vdiff_uuid?: (string|null); } - /** Represents a ResetSequencesResponse. */ - class ResetSequencesResponse implements IResetSequencesResponse { + /** Represents a VDiffResponse. */ + class VDiffResponse implements IVDiffResponse { /** - * Constructs a new ResetSequencesResponse. + * Constructs a new VDiffResponse. * @param [properties] Properties to set */ - constructor(properties?: tabletmanagerdata.IResetSequencesResponse); + constructor(properties?: tabletmanagerdata.IVDiffResponse); + + /** VDiffResponse id. */ + public id: (number|Long); + + /** VDiffResponse output. */ + public output?: (query.IQueryResult|null); + + /** VDiffResponse vdiff_uuid. */ + public vdiff_uuid: string; /** - * Creates a new ResetSequencesResponse instance using the specified properties. + * Creates a new VDiffResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ResetSequencesResponse instance + * @returns VDiffResponse instance */ - public static create(properties?: tabletmanagerdata.IResetSequencesResponse): tabletmanagerdata.ResetSequencesResponse; + public static create(properties?: tabletmanagerdata.IVDiffResponse): tabletmanagerdata.VDiffResponse; /** - * Encodes the specified ResetSequencesResponse message. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. - * @param message ResetSequencesResponse message or plain object to encode + * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * @param message VDiffResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: tabletmanagerdata.IResetSequencesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ResetSequencesResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. - * @param message ResetSequencesResponse message or plain object to encode + * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * @param message VDiffResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: tabletmanagerdata.IResetSequencesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ResetSequencesResponse message from the specified reader or buffer. + * Decodes a VDiffResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ResetSequencesResponse + * @returns VDiffResponse * @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): tabletmanagerdata.ResetSequencesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffResponse; /** - * Decodes a ResetSequencesResponse message from the specified reader or buffer, length delimited. + * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ResetSequencesResponse + * @returns VDiffResponse * @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)): tabletmanagerdata.ResetSequencesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffResponse; /** - * Verifies a ResetSequencesResponse message. + * Verifies a VDiffResponse 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 ResetSequencesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ResetSequencesResponse + * @returns VDiffResponse */ - public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ResetSequencesResponse; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffResponse; /** - * Creates a plain object from a ResetSequencesResponse message. Also converts values to other types if specified. - * @param message ResetSequencesResponse + * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. + * @param message VDiffResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: tabletmanagerdata.ResetSequencesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ResetSequencesResponse to JSON. + * Converts this VDiffResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ResetSequencesResponse + * Gets the default type url for VDiffResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} - -/** Namespace binlogdata. */ -export namespace binlogdata { - /** Properties of a Charset. */ - interface ICharset { + /** Properties of a VDiffPickerOptions. */ + interface IVDiffPickerOptions { - /** Charset client */ - client?: (number|null); + /** VDiffPickerOptions tablet_types */ + tablet_types?: (string|null); - /** Charset conn */ - conn?: (number|null); + /** VDiffPickerOptions source_cell */ + source_cell?: (string|null); - /** Charset server */ - server?: (number|null); + /** VDiffPickerOptions target_cell */ + target_cell?: (string|null); } - /** Represents a Charset. */ - class Charset implements ICharset { + /** Represents a VDiffPickerOptions. */ + class VDiffPickerOptions implements IVDiffPickerOptions { /** - * Constructs a new Charset. + * Constructs a new VDiffPickerOptions. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.ICharset); + constructor(properties?: tabletmanagerdata.IVDiffPickerOptions); - /** Charset client. */ - public client: number; + /** VDiffPickerOptions tablet_types. */ + public tablet_types: string; - /** Charset conn. */ - public conn: number; + /** VDiffPickerOptions source_cell. */ + public source_cell: string; - /** Charset server. */ - public server: number; + /** VDiffPickerOptions target_cell. */ + public target_cell: string; /** - * Creates a new Charset instance using the specified properties. + * Creates a new VDiffPickerOptions instance using the specified properties. * @param [properties] Properties to set - * @returns Charset instance + * @returns VDiffPickerOptions instance */ - public static create(properties?: binlogdata.ICharset): binlogdata.Charset; + public static create(properties?: tabletmanagerdata.IVDiffPickerOptions): tabletmanagerdata.VDiffPickerOptions; /** - * Encodes the specified Charset message. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. - * @param message Charset message or plain object to encode + * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * @param message VDiffPickerOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.ICharset, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffPickerOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Charset message, length delimited. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. - * @param message Charset message or plain object to encode + * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * @param message VDiffPickerOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.ICharset, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffPickerOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Charset message from the specified reader or buffer. + * Decodes a VDiffPickerOptions message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Charset + * @returns VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Charset; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffPickerOptions; /** - * Decodes a Charset message from the specified reader or buffer, length delimited. + * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Charset + * @returns VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Charset; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffPickerOptions; /** - * Verifies a Charset message. + * Verifies a VDiffPickerOptions 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 Charset message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Charset + * @returns VDiffPickerOptions */ - public static fromObject(object: { [k: string]: any }): binlogdata.Charset; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffPickerOptions; /** - * Creates a plain object from a Charset message. Also converts values to other types if specified. - * @param message Charset + * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. + * @param message VDiffPickerOptions * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.Charset, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffPickerOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Charset to JSON. + * Converts this VDiffPickerOptions to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Charset + * Gets the default type url for VDiffPickerOptions * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BinlogTransaction. */ - interface IBinlogTransaction { + /** Properties of a VDiffReportOptions. */ + interface IVDiffReportOptions { - /** BinlogTransaction statements */ - statements?: (binlogdata.BinlogTransaction.IStatement[]|null); + /** VDiffReportOptions only_pks */ + only_pks?: (boolean|null); - /** BinlogTransaction event_token */ - event_token?: (query.IEventToken|null); + /** VDiffReportOptions debug_query */ + debug_query?: (boolean|null); + + /** VDiffReportOptions format */ + format?: (string|null); } - /** Represents a BinlogTransaction. */ - class BinlogTransaction implements IBinlogTransaction { + /** Represents a VDiffReportOptions. */ + class VDiffReportOptions implements IVDiffReportOptions { /** - * Constructs a new BinlogTransaction. + * Constructs a new VDiffReportOptions. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IBinlogTransaction); + constructor(properties?: tabletmanagerdata.IVDiffReportOptions); - /** BinlogTransaction statements. */ - public statements: binlogdata.BinlogTransaction.IStatement[]; + /** VDiffReportOptions only_pks. */ + public only_pks: boolean; - /** BinlogTransaction event_token. */ - public event_token?: (query.IEventToken|null); + /** VDiffReportOptions debug_query. */ + public debug_query: boolean; + + /** VDiffReportOptions format. */ + public format: string; /** - * Creates a new BinlogTransaction instance using the specified properties. + * Creates a new VDiffReportOptions instance using the specified properties. * @param [properties] Properties to set - * @returns BinlogTransaction instance + * @returns VDiffReportOptions instance */ - public static create(properties?: binlogdata.IBinlogTransaction): binlogdata.BinlogTransaction; + public static create(properties?: tabletmanagerdata.IVDiffReportOptions): tabletmanagerdata.VDiffReportOptions; /** - * Encodes the specified BinlogTransaction message. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. - * @param message BinlogTransaction message or plain object to encode + * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * @param message VDiffReportOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IBinlogTransaction, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffReportOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BinlogTransaction message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. - * @param message BinlogTransaction message or plain object to encode + * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * @param message VDiffReportOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IBinlogTransaction, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffReportOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BinlogTransaction message from the specified reader or buffer. + * Decodes a VDiffReportOptions message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BinlogTransaction + * @returns VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogTransaction; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffReportOptions; /** - * Decodes a BinlogTransaction message from the specified reader or buffer, length delimited. + * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BinlogTransaction + * @returns VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogTransaction; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffReportOptions; /** - * Verifies a BinlogTransaction message. + * Verifies a VDiffReportOptions 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 BinlogTransaction message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffReportOptions message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BinlogTransaction + * @returns VDiffReportOptions */ - public static fromObject(object: { [k: string]: any }): binlogdata.BinlogTransaction; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffReportOptions; /** - * Creates a plain object from a BinlogTransaction message. Also converts values to other types if specified. - * @param message BinlogTransaction + * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. + * @param message VDiffReportOptions * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.BinlogTransaction, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffReportOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BinlogTransaction to JSON. + * Converts this VDiffReportOptions to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BinlogTransaction + * Gets the default type url for VDiffReportOptions * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace BinlogTransaction { - - /** Properties of a Statement. */ - interface IStatement { + /** Properties of a VDiffCoreOptions. */ + interface IVDiffCoreOptions { - /** Statement category */ - category?: (binlogdata.BinlogTransaction.Statement.Category|null); - - /** Statement charset */ - charset?: (binlogdata.ICharset|null); - - /** Statement sql */ - sql?: (Uint8Array|null); - } - - /** Represents a Statement. */ - class Statement implements IStatement { - - /** - * Constructs a new Statement. - * @param [properties] Properties to set - */ - constructor(properties?: binlogdata.BinlogTransaction.IStatement); - - /** Statement category. */ - public category: binlogdata.BinlogTransaction.Statement.Category; - - /** Statement charset. */ - public charset?: (binlogdata.ICharset|null); - - /** Statement sql. */ - public sql: Uint8Array; - - /** - * Creates a new Statement instance using the specified properties. - * @param [properties] Properties to set - * @returns Statement instance - */ - public static create(properties?: binlogdata.BinlogTransaction.IStatement): binlogdata.BinlogTransaction.Statement; - - /** - * Encodes the specified Statement message. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. - * @param message Statement message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: binlogdata.BinlogTransaction.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified Statement message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. - * @param message Statement message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: binlogdata.BinlogTransaction.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Decodes a Statement message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogTransaction.Statement; - - /** - * Decodes a Statement message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogTransaction.Statement; + /** VDiffCoreOptions tables */ + tables?: (string|null); - /** - * Verifies a Statement 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); + /** VDiffCoreOptions auto_retry */ + auto_retry?: (boolean|null); - /** - * Creates a Statement message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns Statement - */ - public static fromObject(object: { [k: string]: any }): binlogdata.BinlogTransaction.Statement; + /** VDiffCoreOptions max_rows */ + max_rows?: (number|Long|null); - /** - * Creates a plain object from a Statement message. Also converts values to other types if specified. - * @param message Statement - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: binlogdata.BinlogTransaction.Statement, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** VDiffCoreOptions checksum */ + checksum?: (boolean|null); - /** - * Converts this Statement to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** VDiffCoreOptions sample_pct */ + sample_pct?: (number|Long|null); - /** - * Gets the default type url for Statement - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } + /** VDiffCoreOptions timeout_seconds */ + timeout_seconds?: (number|Long|null); - namespace Statement { + /** VDiffCoreOptions max_extra_rows_to_compare */ + max_extra_rows_to_compare?: (number|Long|null); - /** Category enum. */ - enum Category { - BL_UNRECOGNIZED = 0, - BL_BEGIN = 1, - BL_COMMIT = 2, - BL_ROLLBACK = 3, - BL_DML_DEPRECATED = 4, - BL_DDL = 5, - BL_SET = 6, - BL_INSERT = 7, - BL_UPDATE = 8, - BL_DELETE = 9 - } - } + /** VDiffCoreOptions update_table_stats */ + update_table_stats?: (boolean|null); } - /** Properties of a StreamKeyRangeRequest. */ - interface IStreamKeyRangeRequest { + /** Represents a VDiffCoreOptions. */ + class VDiffCoreOptions implements IVDiffCoreOptions { - /** StreamKeyRangeRequest position */ - position?: (string|null); + /** + * Constructs a new VDiffCoreOptions. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IVDiffCoreOptions); - /** StreamKeyRangeRequest key_range */ - key_range?: (topodata.IKeyRange|null); + /** VDiffCoreOptions tables. */ + public tables: string; - /** StreamKeyRangeRequest charset */ - charset?: (binlogdata.ICharset|null); - } + /** VDiffCoreOptions auto_retry. */ + public auto_retry: boolean; - /** Represents a StreamKeyRangeRequest. */ - class StreamKeyRangeRequest implements IStreamKeyRangeRequest { + /** VDiffCoreOptions max_rows. */ + public max_rows: (number|Long); - /** - * Constructs a new StreamKeyRangeRequest. - * @param [properties] Properties to set - */ - constructor(properties?: binlogdata.IStreamKeyRangeRequest); + /** VDiffCoreOptions checksum. */ + public checksum: boolean; - /** StreamKeyRangeRequest position. */ - public position: string; + /** VDiffCoreOptions sample_pct. */ + public sample_pct: (number|Long); - /** StreamKeyRangeRequest key_range. */ - public key_range?: (topodata.IKeyRange|null); + /** VDiffCoreOptions timeout_seconds. */ + public timeout_seconds: (number|Long); - /** StreamKeyRangeRequest charset. */ - public charset?: (binlogdata.ICharset|null); + /** VDiffCoreOptions max_extra_rows_to_compare. */ + public max_extra_rows_to_compare: (number|Long); + + /** VDiffCoreOptions update_table_stats. */ + public update_table_stats: boolean; /** - * Creates a new StreamKeyRangeRequest instance using the specified properties. + * Creates a new VDiffCoreOptions instance using the specified properties. * @param [properties] Properties to set - * @returns StreamKeyRangeRequest instance + * @returns VDiffCoreOptions instance */ - public static create(properties?: binlogdata.IStreamKeyRangeRequest): binlogdata.StreamKeyRangeRequest; + public static create(properties?: tabletmanagerdata.IVDiffCoreOptions): tabletmanagerdata.VDiffCoreOptions; /** - * Encodes the specified StreamKeyRangeRequest message. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. - * @param message StreamKeyRangeRequest message or plain object to encode + * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * @param message VDiffCoreOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IStreamKeyRangeRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffCoreOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamKeyRangeRequest message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. - * @param message StreamKeyRangeRequest message or plain object to encode + * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * @param message VDiffCoreOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IStreamKeyRangeRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffCoreOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamKeyRangeRequest message from the specified reader or buffer. + * Decodes a VDiffCoreOptions message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamKeyRangeRequest + * @returns VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamKeyRangeRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffCoreOptions; /** - * Decodes a StreamKeyRangeRequest message from the specified reader or buffer, length delimited. + * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamKeyRangeRequest + * @returns VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamKeyRangeRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffCoreOptions; /** - * Verifies a StreamKeyRangeRequest message. + * Verifies a VDiffCoreOptions 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 StreamKeyRangeRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamKeyRangeRequest + * @returns VDiffCoreOptions */ - public static fromObject(object: { [k: string]: any }): binlogdata.StreamKeyRangeRequest; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffCoreOptions; /** - * Creates a plain object from a StreamKeyRangeRequest message. Also converts values to other types if specified. - * @param message StreamKeyRangeRequest + * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. + * @param message VDiffCoreOptions * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.StreamKeyRangeRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffCoreOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamKeyRangeRequest to JSON. + * Converts this VDiffCoreOptions to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamKeyRangeRequest + * Gets the default type url for VDiffCoreOptions * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamKeyRangeResponse. */ - interface IStreamKeyRangeResponse { + /** Properties of a VDiffOptions. */ + interface IVDiffOptions { - /** StreamKeyRangeResponse binlog_transaction */ - binlog_transaction?: (binlogdata.IBinlogTransaction|null); + /** VDiffOptions picker_options */ + picker_options?: (tabletmanagerdata.IVDiffPickerOptions|null); + + /** VDiffOptions core_options */ + core_options?: (tabletmanagerdata.IVDiffCoreOptions|null); + + /** VDiffOptions report_options */ + report_options?: (tabletmanagerdata.IVDiffReportOptions|null); } - /** Represents a StreamKeyRangeResponse. */ - class StreamKeyRangeResponse implements IStreamKeyRangeResponse { + /** Represents a VDiffOptions. */ + class VDiffOptions implements IVDiffOptions { /** - * Constructs a new StreamKeyRangeResponse. + * Constructs a new VDiffOptions. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IStreamKeyRangeResponse); + constructor(properties?: tabletmanagerdata.IVDiffOptions); - /** StreamKeyRangeResponse binlog_transaction. */ - public binlog_transaction?: (binlogdata.IBinlogTransaction|null); + /** VDiffOptions picker_options. */ + public picker_options?: (tabletmanagerdata.IVDiffPickerOptions|null); + + /** VDiffOptions core_options. */ + public core_options?: (tabletmanagerdata.IVDiffCoreOptions|null); + + /** VDiffOptions report_options. */ + public report_options?: (tabletmanagerdata.IVDiffReportOptions|null); /** - * Creates a new StreamKeyRangeResponse instance using the specified properties. + * Creates a new VDiffOptions instance using the specified properties. * @param [properties] Properties to set - * @returns StreamKeyRangeResponse instance + * @returns VDiffOptions instance */ - public static create(properties?: binlogdata.IStreamKeyRangeResponse): binlogdata.StreamKeyRangeResponse; + public static create(properties?: tabletmanagerdata.IVDiffOptions): tabletmanagerdata.VDiffOptions; /** - * Encodes the specified StreamKeyRangeResponse message. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. - * @param message StreamKeyRangeResponse message or plain object to encode + * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. + * @param message VDiffOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IStreamKeyRangeResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IVDiffOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamKeyRangeResponse message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. - * @param message StreamKeyRangeResponse message or plain object to encode + * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. + * @param message VDiffOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IStreamKeyRangeResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IVDiffOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamKeyRangeResponse message from the specified reader or buffer. + * Decodes a VDiffOptions message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamKeyRangeResponse + * @returns VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamKeyRangeResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.VDiffOptions; /** - * Decodes a StreamKeyRangeResponse message from the specified reader or buffer, length delimited. + * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamKeyRangeResponse + * @returns VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamKeyRangeResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.VDiffOptions; /** - * Verifies a StreamKeyRangeResponse message. + * Verifies a VDiffOptions 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 StreamKeyRangeResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffOptions message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamKeyRangeResponse + * @returns VDiffOptions */ - public static fromObject(object: { [k: string]: any }): binlogdata.StreamKeyRangeResponse; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.VDiffOptions; /** - * Creates a plain object from a StreamKeyRangeResponse message. Also converts values to other types if specified. - * @param message StreamKeyRangeResponse + * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. + * @param message VDiffOptions * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.StreamKeyRangeResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.VDiffOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamKeyRangeResponse to JSON. + * Converts this VDiffOptions to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamKeyRangeResponse + * Gets the default type url for VDiffOptions * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamTablesRequest. */ - interface IStreamTablesRequest { + /** Properties of an UpdateVReplicationWorkflowRequest. */ + interface IUpdateVReplicationWorkflowRequest { - /** StreamTablesRequest position */ - position?: (string|null); + /** UpdateVReplicationWorkflowRequest workflow */ + workflow?: (string|null); - /** StreamTablesRequest tables */ - tables?: (string[]|null); + /** UpdateVReplicationWorkflowRequest cells */ + cells?: (string[]|null); - /** StreamTablesRequest charset */ - charset?: (binlogdata.ICharset|null); + /** UpdateVReplicationWorkflowRequest tablet_types */ + tablet_types?: (topodata.TabletType[]|null); + + /** UpdateVReplicationWorkflowRequest tablet_selection_preference */ + tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); + + /** UpdateVReplicationWorkflowRequest on_ddl */ + on_ddl?: (binlogdata.OnDDLAction|null); + + /** UpdateVReplicationWorkflowRequest state */ + state?: (binlogdata.VReplicationWorkflowState|null); } - /** Represents a StreamTablesRequest. */ - class StreamTablesRequest implements IStreamTablesRequest { + /** Represents an UpdateVReplicationWorkflowRequest. */ + class UpdateVReplicationWorkflowRequest implements IUpdateVReplicationWorkflowRequest { /** - * Constructs a new StreamTablesRequest. + * Constructs a new UpdateVReplicationWorkflowRequest. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IStreamTablesRequest); + constructor(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowRequest); - /** StreamTablesRequest position. */ - public position: string; + /** UpdateVReplicationWorkflowRequest workflow. */ + public workflow: string; - /** StreamTablesRequest tables. */ - public tables: string[]; + /** UpdateVReplicationWorkflowRequest cells. */ + public cells: string[]; - /** StreamTablesRequest charset. */ - public charset?: (binlogdata.ICharset|null); + /** UpdateVReplicationWorkflowRequest tablet_types. */ + public tablet_types: topodata.TabletType[]; + + /** UpdateVReplicationWorkflowRequest tablet_selection_preference. */ + public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; + + /** UpdateVReplicationWorkflowRequest on_ddl. */ + public on_ddl: binlogdata.OnDDLAction; + + /** UpdateVReplicationWorkflowRequest state. */ + public state: binlogdata.VReplicationWorkflowState; /** - * Creates a new StreamTablesRequest instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set - * @returns StreamTablesRequest instance + * @returns UpdateVReplicationWorkflowRequest instance */ - public static create(properties?: binlogdata.IStreamTablesRequest): binlogdata.StreamTablesRequest; + public static create(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowRequest): tabletmanagerdata.UpdateVReplicationWorkflowRequest; /** - * Encodes the specified StreamTablesRequest message. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. - * @param message StreamTablesRequest message or plain object to encode + * Encodes the specified UpdateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. + * @param message UpdateVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IUpdateVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. - * @param message StreamTablesRequest message or plain object to encode + * Encodes the specified UpdateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. + * @param message UpdateVReplicationWorkflowRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IUpdateVReplicationWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamTablesRequest message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamTablesRequest + * @returns UpdateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamTablesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.UpdateVReplicationWorkflowRequest; /** - * Decodes a StreamTablesRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamTablesRequest + * @returns UpdateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamTablesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.UpdateVReplicationWorkflowRequest; /** - * Verifies a StreamTablesRequest message. + * Verifies an UpdateVReplicationWorkflowRequest 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 StreamTablesRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamTablesRequest + * @returns UpdateVReplicationWorkflowRequest */ - public static fromObject(object: { [k: string]: any }): binlogdata.StreamTablesRequest; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVReplicationWorkflowRequest; /** - * Creates a plain object from a StreamTablesRequest message. Also converts values to other types if specified. - * @param message StreamTablesRequest + * Creates a plain object from an UpdateVReplicationWorkflowRequest message. Also converts values to other types if specified. + * @param message UpdateVReplicationWorkflowRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.StreamTablesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.UpdateVReplicationWorkflowRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamTablesRequest to JSON. + * Converts this UpdateVReplicationWorkflowRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamTablesRequest + * Gets the default type url for UpdateVReplicationWorkflowRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamTablesResponse. */ - interface IStreamTablesResponse { + /** Properties of an UpdateVReplicationWorkflowResponse. */ + interface IUpdateVReplicationWorkflowResponse { - /** StreamTablesResponse binlog_transaction */ - binlog_transaction?: (binlogdata.IBinlogTransaction|null); + /** UpdateVReplicationWorkflowResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a StreamTablesResponse. */ - class StreamTablesResponse implements IStreamTablesResponse { + /** Represents an UpdateVReplicationWorkflowResponse. */ + class UpdateVReplicationWorkflowResponse implements IUpdateVReplicationWorkflowResponse { /** - * Constructs a new StreamTablesResponse. + * Constructs a new UpdateVReplicationWorkflowResponse. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IStreamTablesResponse); + constructor(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowResponse); - /** StreamTablesResponse binlog_transaction. */ - public binlog_transaction?: (binlogdata.IBinlogTransaction|null); + /** UpdateVReplicationWorkflowResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new StreamTablesResponse instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowResponse instance using the specified properties. * @param [properties] Properties to set - * @returns StreamTablesResponse instance + * @returns UpdateVReplicationWorkflowResponse instance */ - public static create(properties?: binlogdata.IStreamTablesResponse): binlogdata.StreamTablesResponse; + public static create(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowResponse): tabletmanagerdata.UpdateVReplicationWorkflowResponse; /** - * Encodes the specified StreamTablesResponse message. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. - * @param message StreamTablesResponse message or plain object to encode + * Encodes the specified UpdateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. + * @param message UpdateVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IUpdateVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. - * @param message StreamTablesResponse message or plain object to encode + * Encodes the specified UpdateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. + * @param message UpdateVReplicationWorkflowResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IUpdateVReplicationWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamTablesResponse message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamTablesResponse + * @returns UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamTablesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.UpdateVReplicationWorkflowResponse; /** - * Decodes a StreamTablesResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamTablesResponse + * @returns UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamTablesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.UpdateVReplicationWorkflowResponse; /** - * Verifies a StreamTablesResponse message. + * Verifies an UpdateVReplicationWorkflowResponse 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 StreamTablesResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamTablesResponse + * @returns UpdateVReplicationWorkflowResponse */ - public static fromObject(object: { [k: string]: any }): binlogdata.StreamTablesResponse; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVReplicationWorkflowResponse; /** - * Creates a plain object from a StreamTablesResponse message. Also converts values to other types if specified. - * @param message StreamTablesResponse + * Creates a plain object from an UpdateVReplicationWorkflowResponse message. Also converts values to other types if specified. + * @param message UpdateVReplicationWorkflowResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.StreamTablesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.UpdateVReplicationWorkflowResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamTablesResponse to JSON. + * Converts this UpdateVReplicationWorkflowResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamTablesResponse + * Gets the default type url for UpdateVReplicationWorkflowResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CharsetConversion. */ - interface ICharsetConversion { - - /** CharsetConversion from_charset */ - from_charset?: (string|null); + /** Properties of a ResetSequencesRequest. */ + interface IResetSequencesRequest { - /** CharsetConversion to_charset */ - to_charset?: (string|null); + /** ResetSequencesRequest tables */ + tables?: (string[]|null); } - /** Represents a CharsetConversion. */ - class CharsetConversion implements ICharsetConversion { + /** Represents a ResetSequencesRequest. */ + class ResetSequencesRequest implements IResetSequencesRequest { /** - * Constructs a new CharsetConversion. + * Constructs a new ResetSequencesRequest. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.ICharsetConversion); - - /** CharsetConversion from_charset. */ - public from_charset: string; + constructor(properties?: tabletmanagerdata.IResetSequencesRequest); - /** CharsetConversion to_charset. */ - public to_charset: string; + /** ResetSequencesRequest tables. */ + public tables: string[]; /** - * Creates a new CharsetConversion instance using the specified properties. + * Creates a new ResetSequencesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns CharsetConversion instance + * @returns ResetSequencesRequest instance */ - public static create(properties?: binlogdata.ICharsetConversion): binlogdata.CharsetConversion; + public static create(properties?: tabletmanagerdata.IResetSequencesRequest): tabletmanagerdata.ResetSequencesRequest; /** - * Encodes the specified CharsetConversion message. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. - * @param message CharsetConversion message or plain object to encode + * Encodes the specified ResetSequencesRequest message. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. + * @param message ResetSequencesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.ICharsetConversion, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IResetSequencesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CharsetConversion message, length delimited. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. - * @param message CharsetConversion message or plain object to encode + * Encodes the specified ResetSequencesRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. + * @param message ResetSequencesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.ICharsetConversion, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IResetSequencesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CharsetConversion message from the specified reader or buffer. + * Decodes a ResetSequencesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CharsetConversion + * @returns ResetSequencesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.CharsetConversion; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ResetSequencesRequest; /** - * Decodes a CharsetConversion message from the specified reader or buffer, length delimited. + * Decodes a ResetSequencesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CharsetConversion + * @returns ResetSequencesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.CharsetConversion; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ResetSequencesRequest; /** - * Verifies a CharsetConversion message. + * Verifies a ResetSequencesRequest 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 CharsetConversion message from a plain object. Also converts values to their respective internal types. + * Creates a ResetSequencesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CharsetConversion + * @returns ResetSequencesRequest */ - public static fromObject(object: { [k: string]: any }): binlogdata.CharsetConversion; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ResetSequencesRequest; /** - * Creates a plain object from a CharsetConversion message. Also converts values to other types if specified. - * @param message CharsetConversion + * Creates a plain object from a ResetSequencesRequest message. Also converts values to other types if specified. + * @param message ResetSequencesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.CharsetConversion, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.ResetSequencesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CharsetConversion to JSON. + * Converts this ResetSequencesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CharsetConversion + * Gets the default type url for ResetSequencesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Rule. */ - interface IRule { - - /** Rule match */ - match?: (string|null); - - /** Rule filter */ - filter?: (string|null); - - /** Rule convert_enum_to_text */ - convert_enum_to_text?: ({ [k: string]: string }|null); - - /** Rule convert_charset */ - convert_charset?: ({ [k: string]: binlogdata.ICharsetConversion }|null); - - /** Rule source_unique_key_columns */ - source_unique_key_columns?: (string|null); - - /** Rule target_unique_key_columns */ - target_unique_key_columns?: (string|null); - - /** Rule source_unique_key_target_columns */ - source_unique_key_target_columns?: (string|null); - - /** Rule convert_int_to_enum */ - convert_int_to_enum?: ({ [k: string]: boolean }|null); + /** Properties of a ResetSequencesResponse. */ + interface IResetSequencesResponse { } - /** Represents a Rule. */ - class Rule implements IRule { + /** Represents a ResetSequencesResponse. */ + class ResetSequencesResponse implements IResetSequencesResponse { /** - * Constructs a new Rule. + * Constructs a new ResetSequencesResponse. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IRule); - - /** Rule match. */ - public match: string; - - /** Rule filter. */ - public filter: string; - - /** Rule convert_enum_to_text. */ - public convert_enum_to_text: { [k: string]: string }; - - /** Rule convert_charset. */ - public convert_charset: { [k: string]: binlogdata.ICharsetConversion }; - - /** Rule source_unique_key_columns. */ - public source_unique_key_columns: string; - - /** Rule target_unique_key_columns. */ - public target_unique_key_columns: string; - - /** Rule source_unique_key_target_columns. */ - public source_unique_key_target_columns: string; - - /** Rule convert_int_to_enum. */ - public convert_int_to_enum: { [k: string]: boolean }; + constructor(properties?: tabletmanagerdata.IResetSequencesResponse); /** - * Creates a new Rule instance using the specified properties. + * Creates a new ResetSequencesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns Rule instance + * @returns ResetSequencesResponse instance */ - public static create(properties?: binlogdata.IRule): binlogdata.Rule; + public static create(properties?: tabletmanagerdata.IResetSequencesResponse): tabletmanagerdata.ResetSequencesResponse; /** - * Encodes the specified Rule message. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. - * @param message Rule message or plain object to encode + * Encodes the specified ResetSequencesResponse message. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. + * @param message ResetSequencesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: tabletmanagerdata.IResetSequencesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Rule message, length delimited. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. - * @param message Rule message or plain object to encode + * Encodes the specified ResetSequencesResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. + * @param message ResetSequencesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: tabletmanagerdata.IResetSequencesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Rule message from the specified reader or buffer. + * Decodes a ResetSequencesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Rule + * @returns ResetSequencesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Rule; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ResetSequencesResponse; /** - * Decodes a Rule message from the specified reader or buffer, length delimited. + * Decodes a ResetSequencesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Rule + * @returns ResetSequencesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Rule; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ResetSequencesResponse; /** - * Verifies a Rule message. + * Verifies a ResetSequencesResponse 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 Rule message from a plain object. Also converts values to their respective internal types. + * Creates a ResetSequencesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Rule + * @returns ResetSequencesResponse */ - public static fromObject(object: { [k: string]: any }): binlogdata.Rule; + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ResetSequencesResponse; /** - * Creates a plain object from a Rule message. Also converts values to other types if specified. - * @param message Rule + * Creates a plain object from a ResetSequencesResponse message. Also converts values to other types if specified. + * @param message ResetSequencesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.Rule, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: tabletmanagerdata.ResetSequencesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Rule to JSON. + * Converts this ResetSequencesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Rule + * Gets the default type url for ResetSequencesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } +} - /** Properties of a Filter. */ - interface IFilter { +/** Namespace binlogdata. */ +export namespace binlogdata { - /** Filter rules */ - rules?: (binlogdata.IRule[]|null); + /** Properties of a Charset. */ + interface ICharset { - /** Filter field_event_mode */ - field_event_mode?: (binlogdata.Filter.FieldEventMode|null); + /** Charset client */ + client?: (number|null); - /** Filter workflow_type */ - workflow_type?: (number|Long|null); + /** Charset conn */ + conn?: (number|null); - /** Filter workflow_name */ - workflow_name?: (string|null); + /** Charset server */ + server?: (number|null); } - /** Represents a Filter. */ - class Filter implements IFilter { + /** Represents a Charset. */ + class Charset implements ICharset { /** - * Constructs a new Filter. + * Constructs a new Charset. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IFilter); - - /** Filter rules. */ - public rules: binlogdata.IRule[]; + constructor(properties?: binlogdata.ICharset); - /** Filter field_event_mode. */ - public field_event_mode: binlogdata.Filter.FieldEventMode; + /** Charset client. */ + public client: number; - /** Filter workflow_type. */ - public workflow_type: (number|Long); + /** Charset conn. */ + public conn: number; - /** Filter workflow_name. */ - public workflow_name: string; + /** Charset server. */ + public server: number; /** - * Creates a new Filter instance using the specified properties. + * Creates a new Charset instance using the specified properties. * @param [properties] Properties to set - * @returns Filter instance + * @returns Charset instance */ - public static create(properties?: binlogdata.IFilter): binlogdata.Filter; + public static create(properties?: binlogdata.ICharset): binlogdata.Charset; /** - * Encodes the specified Filter message. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. - * @param message Filter message or plain object to encode + * Encodes the specified Charset message. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. + * @param message Charset message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IFilter, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.ICharset, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Filter message, length delimited. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. - * @param message Filter message or plain object to encode + * Encodes the specified Charset message, length delimited. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. + * @param message Charset message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IFilter, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.ICharset, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Filter message from the specified reader or buffer. + * Decodes a Charset message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Filter + * @returns Charset * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Filter; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Charset; /** - * Decodes a Filter message from the specified reader or buffer, length delimited. + * Decodes a Charset message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Filter + * @returns Charset * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Filter; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Charset; /** - * Verifies a Filter message. + * Verifies a Charset 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 Filter message from a plain object. Also converts values to their respective internal types. + * Creates a Charset message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Filter + * @returns Charset */ - public static fromObject(object: { [k: string]: any }): binlogdata.Filter; + public static fromObject(object: { [k: string]: any }): binlogdata.Charset; /** - * Creates a plain object from a Filter message. Also converts values to other types if specified. - * @param message Filter + * Creates a plain object from a Charset message. Also converts values to other types if specified. + * @param message Charset * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.Filter, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.Charset, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Filter to JSON. + * Converts this Charset to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Filter + * Gets the default type url for Charset * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace Filter { - - /** FieldEventMode enum. */ - enum FieldEventMode { - ERR_ON_MISMATCH = 0, - BEST_EFFORT = 1 - } - } - - /** OnDDLAction enum. */ - enum OnDDLAction { - IGNORE = 0, - STOP = 1, - EXEC = 2, - EXEC_IGNORE = 3 - } - - /** VReplicationWorkflowType enum. */ - enum VReplicationWorkflowType { - Materialize = 0, - MoveTables = 1, - CreateLookupIndex = 2, - Migrate = 3, - Reshard = 4, - OnlineDDL = 5 - } - - /** VReplicationWorkflowSubType enum. */ - enum VReplicationWorkflowSubType { - None = 0, - Partial = 1 - } - - /** Properties of a BinlogSource. */ - interface IBinlogSource { - - /** BinlogSource keyspace */ - keyspace?: (string|null); - - /** BinlogSource shard */ - shard?: (string|null); - - /** BinlogSource tablet_type */ - tablet_type?: (topodata.TabletType|null); - - /** BinlogSource key_range */ - key_range?: (topodata.IKeyRange|null); - - /** BinlogSource tables */ - tables?: (string[]|null); - - /** BinlogSource filter */ - filter?: (binlogdata.IFilter|null); - - /** BinlogSource on_ddl */ - on_ddl?: (binlogdata.OnDDLAction|null); - - /** BinlogSource external_mysql */ - external_mysql?: (string|null); - - /** BinlogSource stop_after_copy */ - stop_after_copy?: (boolean|null); - - /** BinlogSource external_cluster */ - external_cluster?: (string|null); + /** Properties of a BinlogTransaction. */ + interface IBinlogTransaction { - /** BinlogSource source_time_zone */ - source_time_zone?: (string|null); + /** BinlogTransaction statements */ + statements?: (binlogdata.BinlogTransaction.IStatement[]|null); - /** BinlogSource target_time_zone */ - target_time_zone?: (string|null); + /** BinlogTransaction event_token */ + event_token?: (query.IEventToken|null); } - /** Represents a BinlogSource. */ - class BinlogSource implements IBinlogSource { + /** Represents a BinlogTransaction. */ + class BinlogTransaction implements IBinlogTransaction { /** - * Constructs a new BinlogSource. + * Constructs a new BinlogTransaction. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IBinlogSource); - - /** BinlogSource keyspace. */ - public keyspace: string; - - /** BinlogSource shard. */ - public shard: string; - - /** BinlogSource tablet_type. */ - public tablet_type: topodata.TabletType; - - /** BinlogSource key_range. */ - public key_range?: (topodata.IKeyRange|null); - - /** BinlogSource tables. */ - public tables: string[]; - - /** BinlogSource filter. */ - public filter?: (binlogdata.IFilter|null); - - /** BinlogSource on_ddl. */ - public on_ddl: binlogdata.OnDDLAction; - - /** BinlogSource external_mysql. */ - public external_mysql: string; - - /** BinlogSource stop_after_copy. */ - public stop_after_copy: boolean; - - /** BinlogSource external_cluster. */ - public external_cluster: string; + constructor(properties?: binlogdata.IBinlogTransaction); - /** BinlogSource source_time_zone. */ - public source_time_zone: string; + /** BinlogTransaction statements. */ + public statements: binlogdata.BinlogTransaction.IStatement[]; - /** BinlogSource target_time_zone. */ - public target_time_zone: string; + /** BinlogTransaction event_token. */ + public event_token?: (query.IEventToken|null); /** - * Creates a new BinlogSource instance using the specified properties. + * Creates a new BinlogTransaction instance using the specified properties. * @param [properties] Properties to set - * @returns BinlogSource instance + * @returns BinlogTransaction instance */ - public static create(properties?: binlogdata.IBinlogSource): binlogdata.BinlogSource; + public static create(properties?: binlogdata.IBinlogTransaction): binlogdata.BinlogTransaction; /** - * Encodes the specified BinlogSource message. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. - * @param message BinlogSource message or plain object to encode + * Encodes the specified BinlogTransaction message. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. + * @param message BinlogTransaction message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IBinlogSource, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IBinlogTransaction, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BinlogSource message, length delimited. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. - * @param message BinlogSource message or plain object to encode + * Encodes the specified BinlogTransaction message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. + * @param message BinlogTransaction message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IBinlogSource, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IBinlogTransaction, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BinlogSource message from the specified reader or buffer. + * Decodes a BinlogTransaction message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BinlogSource + * @returns BinlogTransaction * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogSource; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogTransaction; /** - * Decodes a BinlogSource message from the specified reader or buffer, length delimited. + * Decodes a BinlogTransaction message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BinlogSource + * @returns BinlogTransaction * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogSource; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogTransaction; /** - * Verifies a BinlogSource message. + * Verifies a BinlogTransaction 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 BinlogSource message from a plain object. Also converts values to their respective internal types. + * Creates a BinlogTransaction message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BinlogSource + * @returns BinlogTransaction */ - public static fromObject(object: { [k: string]: any }): binlogdata.BinlogSource; + public static fromObject(object: { [k: string]: any }): binlogdata.BinlogTransaction; /** - * Creates a plain object from a BinlogSource message. Also converts values to other types if specified. - * @param message BinlogSource + * Creates a plain object from a BinlogTransaction message. Also converts values to other types if specified. + * @param message BinlogTransaction * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.BinlogSource, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.BinlogTransaction, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BinlogSource to JSON. + * Converts this BinlogTransaction to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BinlogSource + * Gets the default type url for BinlogTransaction * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** VEventType enum. */ - enum VEventType { - UNKNOWN = 0, - GTID = 1, - BEGIN = 2, - COMMIT = 3, - ROLLBACK = 4, - DDL = 5, - INSERT = 6, - REPLACE = 7, - UPDATE = 8, - DELETE = 9, - SET = 10, - OTHER = 11, - ROW = 12, - FIELD = 13, - HEARTBEAT = 14, - VGTID = 15, - JOURNAL = 16, - VERSION = 17, - LASTPK = 18, - SAVEPOINT = 19, - COPY_COMPLETED = 20 - } - - /** Properties of a RowChange. */ - interface IRowChange { - - /** RowChange before */ - before?: (query.IRow|null); - - /** RowChange after */ - after?: (query.IRow|null); - - /** RowChange data_columns */ - data_columns?: (binlogdata.RowChange.IBitmap|null); - } - - /** Represents a RowChange. */ - class RowChange implements IRowChange { - - /** - * Constructs a new RowChange. - * @param [properties] Properties to set - */ - constructor(properties?: binlogdata.IRowChange); - - /** RowChange before. */ - public before?: (query.IRow|null); - - /** RowChange after. */ - public after?: (query.IRow|null); - - /** RowChange data_columns. */ - public data_columns?: (binlogdata.RowChange.IBitmap|null); - - /** - * Creates a new RowChange instance using the specified properties. - * @param [properties] Properties to set - * @returns RowChange instance - */ - public static create(properties?: binlogdata.IRowChange): binlogdata.RowChange; - - /** - * Encodes the specified RowChange message. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. - * @param message RowChange message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: binlogdata.IRowChange, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified RowChange message, length delimited. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. - * @param message RowChange message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: binlogdata.IRowChange, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Decodes a RowChange message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns RowChange - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowChange; - - /** - * Decodes a RowChange message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns RowChange - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowChange; - - /** - * Verifies a RowChange 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 RowChange message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns RowChange - */ - public static fromObject(object: { [k: string]: any }): binlogdata.RowChange; - - /** - * Creates a plain object from a RowChange message. Also converts values to other types if specified. - * @param message RowChange - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: binlogdata.RowChange, options?: $protobuf.IConversionOptions): { [k: string]: any }; - - /** - * Converts this RowChange to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; - - /** - * Gets the default type url for RowChange - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } + namespace BinlogTransaction { - namespace RowChange { + /** Properties of a Statement. */ + interface IStatement { - /** Properties of a Bitmap. */ - interface IBitmap { + /** Statement category */ + category?: (binlogdata.BinlogTransaction.Statement.Category|null); - /** Bitmap count */ - count?: (number|Long|null); + /** Statement charset */ + charset?: (binlogdata.ICharset|null); - /** Bitmap cols */ - cols?: (Uint8Array|null); + /** Statement sql */ + sql?: (Uint8Array|null); } - /** Represents a Bitmap. */ - class Bitmap implements IBitmap { + /** Represents a Statement. */ + class Statement implements IStatement { /** - * Constructs a new Bitmap. + * Constructs a new Statement. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.RowChange.IBitmap); + constructor(properties?: binlogdata.BinlogTransaction.IStatement); - /** Bitmap count. */ - public count: (number|Long); + /** Statement category. */ + public category: binlogdata.BinlogTransaction.Statement.Category; - /** Bitmap cols. */ - public cols: Uint8Array; + /** Statement charset. */ + public charset?: (binlogdata.ICharset|null); + + /** Statement sql. */ + public sql: Uint8Array; /** - * Creates a new Bitmap instance using the specified properties. + * Creates a new Statement instance using the specified properties. * @param [properties] Properties to set - * @returns Bitmap instance + * @returns Statement instance */ - public static create(properties?: binlogdata.RowChange.IBitmap): binlogdata.RowChange.Bitmap; + public static create(properties?: binlogdata.BinlogTransaction.IStatement): binlogdata.BinlogTransaction.Statement; /** - * Encodes the specified Bitmap message. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. - * @param message Bitmap message or plain object to encode + * Encodes the specified Statement message. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. + * @param message Statement message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.RowChange.IBitmap, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.BinlogTransaction.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Bitmap message, length delimited. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. - * @param message Bitmap message or plain object to encode + * Encodes the specified Statement message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. + * @param message Statement message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.RowChange.IBitmap, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.BinlogTransaction.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Bitmap message from the specified reader or buffer. + * Decodes a Statement message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Bitmap + * @returns Statement * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowChange.Bitmap; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogTransaction.Statement; /** - * Decodes a Bitmap message from the specified reader or buffer, length delimited. + * Decodes a Statement message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Bitmap + * @returns Statement * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowChange.Bitmap; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogTransaction.Statement; /** - * Verifies a Bitmap message. + * Verifies a Statement 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 Bitmap message from a plain object. Also converts values to their respective internal types. + * Creates a Statement message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Bitmap + * @returns Statement */ - public static fromObject(object: { [k: string]: any }): binlogdata.RowChange.Bitmap; + public static fromObject(object: { [k: string]: any }): binlogdata.BinlogTransaction.Statement; /** - * Creates a plain object from a Bitmap message. Also converts values to other types if specified. - * @param message Bitmap + * Creates a plain object from a Statement message. Also converts values to other types if specified. + * @param message Statement * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.RowChange.Bitmap, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.BinlogTransaction.Statement, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Bitmap to JSON. + * Converts this Statement to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Bitmap + * Gets the default type url for Statement * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - } - /** Properties of a RowEvent. */ - interface IRowEvent { + namespace Statement { - /** RowEvent table_name */ - table_name?: (string|null); + /** Category enum. */ + enum Category { + BL_UNRECOGNIZED = 0, + BL_BEGIN = 1, + BL_COMMIT = 2, + BL_ROLLBACK = 3, + BL_DML_DEPRECATED = 4, + BL_DDL = 5, + BL_SET = 6, + BL_INSERT = 7, + BL_UPDATE = 8, + BL_DELETE = 9 + } + } + } - /** RowEvent row_changes */ - row_changes?: (binlogdata.IRowChange[]|null); + /** Properties of a StreamKeyRangeRequest. */ + interface IStreamKeyRangeRequest { - /** RowEvent keyspace */ - keyspace?: (string|null); + /** StreamKeyRangeRequest position */ + position?: (string|null); - /** RowEvent shard */ - shard?: (string|null); + /** StreamKeyRangeRequest key_range */ + key_range?: (topodata.IKeyRange|null); + + /** StreamKeyRangeRequest charset */ + charset?: (binlogdata.ICharset|null); } - /** Represents a RowEvent. */ - class RowEvent implements IRowEvent { + /** Represents a StreamKeyRangeRequest. */ + class StreamKeyRangeRequest implements IStreamKeyRangeRequest { /** - * Constructs a new RowEvent. + * Constructs a new StreamKeyRangeRequest. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IRowEvent); - - /** RowEvent table_name. */ - public table_name: string; + constructor(properties?: binlogdata.IStreamKeyRangeRequest); - /** RowEvent row_changes. */ - public row_changes: binlogdata.IRowChange[]; + /** StreamKeyRangeRequest position. */ + public position: string; - /** RowEvent keyspace. */ - public keyspace: string; + /** StreamKeyRangeRequest key_range. */ + public key_range?: (topodata.IKeyRange|null); - /** RowEvent shard. */ - public shard: string; + /** StreamKeyRangeRequest charset. */ + public charset?: (binlogdata.ICharset|null); /** - * Creates a new RowEvent instance using the specified properties. + * Creates a new StreamKeyRangeRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RowEvent instance + * @returns StreamKeyRangeRequest instance */ - public static create(properties?: binlogdata.IRowEvent): binlogdata.RowEvent; + public static create(properties?: binlogdata.IStreamKeyRangeRequest): binlogdata.StreamKeyRangeRequest; /** - * Encodes the specified RowEvent message. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. - * @param message RowEvent message or plain object to encode + * Encodes the specified StreamKeyRangeRequest message. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. + * @param message StreamKeyRangeRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IRowEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IStreamKeyRangeRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RowEvent message, length delimited. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. - * @param message RowEvent message or plain object to encode + * Encodes the specified StreamKeyRangeRequest message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. + * @param message StreamKeyRangeRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IRowEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IStreamKeyRangeRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RowEvent message from the specified reader or buffer. + * Decodes a StreamKeyRangeRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RowEvent + * @returns StreamKeyRangeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowEvent; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamKeyRangeRequest; /** - * Decodes a RowEvent message from the specified reader or buffer, length delimited. + * Decodes a StreamKeyRangeRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RowEvent + * @returns StreamKeyRangeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowEvent; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamKeyRangeRequest; /** - * Verifies a RowEvent message. + * Verifies a StreamKeyRangeRequest 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 RowEvent message from a plain object. Also converts values to their respective internal types. + * Creates a StreamKeyRangeRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RowEvent + * @returns StreamKeyRangeRequest */ - public static fromObject(object: { [k: string]: any }): binlogdata.RowEvent; + public static fromObject(object: { [k: string]: any }): binlogdata.StreamKeyRangeRequest; /** - * Creates a plain object from a RowEvent message. Also converts values to other types if specified. - * @param message RowEvent + * Creates a plain object from a StreamKeyRangeRequest message. Also converts values to other types if specified. + * @param message StreamKeyRangeRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.RowEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.StreamKeyRangeRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RowEvent to JSON. + * Converts this StreamKeyRangeRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RowEvent + * Gets the default type url for StreamKeyRangeRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a FieldEvent. */ - interface IFieldEvent { - - /** FieldEvent table_name */ - table_name?: (string|null); - - /** FieldEvent fields */ - fields?: (query.IField[]|null); - - /** FieldEvent keyspace */ - keyspace?: (string|null); + /** Properties of a StreamKeyRangeResponse. */ + interface IStreamKeyRangeResponse { - /** FieldEvent shard */ - shard?: (string|null); + /** StreamKeyRangeResponse binlog_transaction */ + binlog_transaction?: (binlogdata.IBinlogTransaction|null); } - /** Represents a FieldEvent. */ - class FieldEvent implements IFieldEvent { + /** Represents a StreamKeyRangeResponse. */ + class StreamKeyRangeResponse implements IStreamKeyRangeResponse { /** - * Constructs a new FieldEvent. + * Constructs a new StreamKeyRangeResponse. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IFieldEvent); - - /** FieldEvent table_name. */ - public table_name: string; - - /** FieldEvent fields. */ - public fields: query.IField[]; - - /** FieldEvent keyspace. */ - public keyspace: string; + constructor(properties?: binlogdata.IStreamKeyRangeResponse); - /** FieldEvent shard. */ - public shard: string; + /** StreamKeyRangeResponse binlog_transaction. */ + public binlog_transaction?: (binlogdata.IBinlogTransaction|null); /** - * Creates a new FieldEvent instance using the specified properties. + * Creates a new StreamKeyRangeResponse instance using the specified properties. * @param [properties] Properties to set - * @returns FieldEvent instance + * @returns StreamKeyRangeResponse instance */ - public static create(properties?: binlogdata.IFieldEvent): binlogdata.FieldEvent; + public static create(properties?: binlogdata.IStreamKeyRangeResponse): binlogdata.StreamKeyRangeResponse; /** - * Encodes the specified FieldEvent message. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. - * @param message FieldEvent message or plain object to encode + * Encodes the specified StreamKeyRangeResponse message. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * @param message StreamKeyRangeResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IFieldEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IStreamKeyRangeResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified FieldEvent message, length delimited. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. - * @param message FieldEvent message or plain object to encode + * Encodes the specified StreamKeyRangeResponse message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * @param message StreamKeyRangeResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IFieldEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IStreamKeyRangeResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a FieldEvent message from the specified reader or buffer. + * Decodes a StreamKeyRangeResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns FieldEvent + * @returns StreamKeyRangeResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.FieldEvent; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamKeyRangeResponse; /** - * Decodes a FieldEvent message from the specified reader or buffer, length delimited. + * Decodes a StreamKeyRangeResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns FieldEvent + * @returns StreamKeyRangeResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.FieldEvent; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamKeyRangeResponse; /** - * Verifies a FieldEvent message. + * Verifies a StreamKeyRangeResponse 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 FieldEvent message from a plain object. Also converts values to their respective internal types. + * Creates a StreamKeyRangeResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns FieldEvent + * @returns StreamKeyRangeResponse */ - public static fromObject(object: { [k: string]: any }): binlogdata.FieldEvent; + public static fromObject(object: { [k: string]: any }): binlogdata.StreamKeyRangeResponse; /** - * Creates a plain object from a FieldEvent message. Also converts values to other types if specified. - * @param message FieldEvent + * Creates a plain object from a StreamKeyRangeResponse message. Also converts values to other types if specified. + * @param message StreamKeyRangeResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.FieldEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.StreamKeyRangeResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this FieldEvent to JSON. + * Converts this StreamKeyRangeResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for FieldEvent + * Gets the default type url for StreamKeyRangeResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardGtid. */ - interface IShardGtid { - - /** ShardGtid keyspace */ - keyspace?: (string|null); + /** Properties of a StreamTablesRequest. */ + interface IStreamTablesRequest { - /** ShardGtid shard */ - shard?: (string|null); + /** StreamTablesRequest position */ + position?: (string|null); - /** ShardGtid gtid */ - gtid?: (string|null); + /** StreamTablesRequest tables */ + tables?: (string[]|null); - /** ShardGtid table_p_ks */ - table_p_ks?: (binlogdata.ITableLastPK[]|null); + /** StreamTablesRequest charset */ + charset?: (binlogdata.ICharset|null); } - /** Represents a ShardGtid. */ - class ShardGtid implements IShardGtid { + /** Represents a StreamTablesRequest. */ + class StreamTablesRequest implements IStreamTablesRequest { /** - * Constructs a new ShardGtid. + * Constructs a new StreamTablesRequest. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IShardGtid); - - /** ShardGtid keyspace. */ - public keyspace: string; + constructor(properties?: binlogdata.IStreamTablesRequest); - /** ShardGtid shard. */ - public shard: string; + /** StreamTablesRequest position. */ + public position: string; - /** ShardGtid gtid. */ - public gtid: string; + /** StreamTablesRequest tables. */ + public tables: string[]; - /** ShardGtid table_p_ks. */ - public table_p_ks: binlogdata.ITableLastPK[]; + /** StreamTablesRequest charset. */ + public charset?: (binlogdata.ICharset|null); /** - * Creates a new ShardGtid instance using the specified properties. + * Creates a new StreamTablesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ShardGtid instance + * @returns StreamTablesRequest instance */ - public static create(properties?: binlogdata.IShardGtid): binlogdata.ShardGtid; + public static create(properties?: binlogdata.IStreamTablesRequest): binlogdata.StreamTablesRequest; /** - * Encodes the specified ShardGtid message. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. - * @param message ShardGtid message or plain object to encode + * Encodes the specified StreamTablesRequest message. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. + * @param message StreamTablesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IShardGtid, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardGtid message, length delimited. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. - * @param message ShardGtid message or plain object to encode + * Encodes the specified StreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. + * @param message StreamTablesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IShardGtid, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardGtid message from the specified reader or buffer. + * Decodes a StreamTablesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardGtid + * @returns StreamTablesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.ShardGtid; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamTablesRequest; /** - * Decodes a ShardGtid message from the specified reader or buffer, length delimited. + * Decodes a StreamTablesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardGtid + * @returns StreamTablesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.ShardGtid; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamTablesRequest; /** - * Verifies a ShardGtid message. + * Verifies a StreamTablesRequest 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 ShardGtid message from a plain object. Also converts values to their respective internal types. + * Creates a StreamTablesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardGtid + * @returns StreamTablesRequest */ - public static fromObject(object: { [k: string]: any }): binlogdata.ShardGtid; + public static fromObject(object: { [k: string]: any }): binlogdata.StreamTablesRequest; /** - * Creates a plain object from a ShardGtid message. Also converts values to other types if specified. - * @param message ShardGtid + * Creates a plain object from a StreamTablesRequest message. Also converts values to other types if specified. + * @param message StreamTablesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.ShardGtid, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.StreamTablesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardGtid to JSON. + * Converts this StreamTablesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardGtid + * Gets the default type url for StreamTablesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VGtid. */ - interface IVGtid { + /** Properties of a StreamTablesResponse. */ + interface IStreamTablesResponse { - /** VGtid shard_gtids */ - shard_gtids?: (binlogdata.IShardGtid[]|null); + /** StreamTablesResponse binlog_transaction */ + binlog_transaction?: (binlogdata.IBinlogTransaction|null); } - /** Represents a VGtid. */ - class VGtid implements IVGtid { + /** Represents a StreamTablesResponse. */ + class StreamTablesResponse implements IStreamTablesResponse { /** - * Constructs a new VGtid. + * Constructs a new StreamTablesResponse. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVGtid); + constructor(properties?: binlogdata.IStreamTablesResponse); - /** VGtid shard_gtids. */ - public shard_gtids: binlogdata.IShardGtid[]; + /** StreamTablesResponse binlog_transaction. */ + public binlog_transaction?: (binlogdata.IBinlogTransaction|null); /** - * Creates a new VGtid instance using the specified properties. + * Creates a new StreamTablesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns VGtid instance + * @returns StreamTablesResponse instance */ - public static create(properties?: binlogdata.IVGtid): binlogdata.VGtid; + public static create(properties?: binlogdata.IStreamTablesResponse): binlogdata.StreamTablesResponse; /** - * Encodes the specified VGtid message. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. - * @param message VGtid message or plain object to encode + * Encodes the specified StreamTablesResponse message. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. + * @param message StreamTablesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVGtid, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VGtid message, length delimited. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. - * @param message VGtid message or plain object to encode + * Encodes the specified StreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. + * @param message StreamTablesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVGtid, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VGtid message from the specified reader or buffer. + * Decodes a StreamTablesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VGtid + * @returns StreamTablesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VGtid; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.StreamTablesResponse; /** - * Decodes a VGtid message from the specified reader or buffer, length delimited. + * Decodes a StreamTablesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VGtid + * @returns StreamTablesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VGtid; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.StreamTablesResponse; /** - * Verifies a VGtid message. + * Verifies a StreamTablesResponse 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 VGtid message from a plain object. Also converts values to their respective internal types. + * Creates a StreamTablesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VGtid + * @returns StreamTablesResponse */ - public static fromObject(object: { [k: string]: any }): binlogdata.VGtid; + public static fromObject(object: { [k: string]: any }): binlogdata.StreamTablesResponse; /** - * Creates a plain object from a VGtid message. Also converts values to other types if specified. - * @param message VGtid + * Creates a plain object from a StreamTablesResponse message. Also converts values to other types if specified. + * @param message StreamTablesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VGtid, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.StreamTablesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VGtid to JSON. + * Converts this StreamTablesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VGtid + * Gets the default type url for StreamTablesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a KeyspaceShard. */ - interface IKeyspaceShard { + /** Properties of a CharsetConversion. */ + interface ICharsetConversion { - /** KeyspaceShard keyspace */ - keyspace?: (string|null); + /** CharsetConversion from_charset */ + from_charset?: (string|null); - /** KeyspaceShard shard */ - shard?: (string|null); + /** CharsetConversion to_charset */ + to_charset?: (string|null); } - /** Represents a KeyspaceShard. */ - class KeyspaceShard implements IKeyspaceShard { + /** Represents a CharsetConversion. */ + class CharsetConversion implements ICharsetConversion { /** - * Constructs a new KeyspaceShard. + * Constructs a new CharsetConversion. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IKeyspaceShard); + constructor(properties?: binlogdata.ICharsetConversion); - /** KeyspaceShard keyspace. */ - public keyspace: string; + /** CharsetConversion from_charset. */ + public from_charset: string; - /** KeyspaceShard shard. */ - public shard: string; + /** CharsetConversion to_charset. */ + public to_charset: string; /** - * Creates a new KeyspaceShard instance using the specified properties. + * Creates a new CharsetConversion instance using the specified properties. * @param [properties] Properties to set - * @returns KeyspaceShard instance + * @returns CharsetConversion instance */ - public static create(properties?: binlogdata.IKeyspaceShard): binlogdata.KeyspaceShard; + public static create(properties?: binlogdata.ICharsetConversion): binlogdata.CharsetConversion; /** - * Encodes the specified KeyspaceShard message. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. - * @param message KeyspaceShard message or plain object to encode + * Encodes the specified CharsetConversion message. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. + * @param message CharsetConversion message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IKeyspaceShard, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.ICharsetConversion, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified KeyspaceShard message, length delimited. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. - * @param message KeyspaceShard message or plain object to encode + * Encodes the specified CharsetConversion message, length delimited. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. + * @param message CharsetConversion message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IKeyspaceShard, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.ICharsetConversion, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a KeyspaceShard message from the specified reader or buffer. + * Decodes a CharsetConversion message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns KeyspaceShard + * @returns CharsetConversion * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.KeyspaceShard; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.CharsetConversion; /** - * Decodes a KeyspaceShard message from the specified reader or buffer, length delimited. + * Decodes a CharsetConversion message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns KeyspaceShard + * @returns CharsetConversion * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.KeyspaceShard; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.CharsetConversion; /** - * Verifies a KeyspaceShard message. + * Verifies a CharsetConversion 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 KeyspaceShard message from a plain object. Also converts values to their respective internal types. + * Creates a CharsetConversion message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns KeyspaceShard + * @returns CharsetConversion */ - public static fromObject(object: { [k: string]: any }): binlogdata.KeyspaceShard; + public static fromObject(object: { [k: string]: any }): binlogdata.CharsetConversion; /** - * Creates a plain object from a KeyspaceShard message. Also converts values to other types if specified. - * @param message KeyspaceShard + * Creates a plain object from a CharsetConversion message. Also converts values to other types if specified. + * @param message CharsetConversion * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.KeyspaceShard, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.CharsetConversion, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this KeyspaceShard to JSON. + * Converts this CharsetConversion to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for KeyspaceShard + * Gets the default type url for CharsetConversion * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** MigrationType enum. */ - enum MigrationType { - TABLES = 0, - SHARDS = 1 - } + /** Properties of a Rule. */ + interface IRule { - /** Properties of a Journal. */ - interface IJournal { + /** Rule match */ + match?: (string|null); - /** Journal id */ - id?: (number|Long|null); + /** Rule filter */ + filter?: (string|null); - /** Journal migration_type */ - migration_type?: (binlogdata.MigrationType|null); + /** Rule convert_enum_to_text */ + convert_enum_to_text?: ({ [k: string]: string }|null); - /** Journal tables */ - tables?: (string[]|null); + /** Rule convert_charset */ + convert_charset?: ({ [k: string]: binlogdata.ICharsetConversion }|null); - /** Journal local_position */ - local_position?: (string|null); + /** Rule source_unique_key_columns */ + source_unique_key_columns?: (string|null); - /** Journal shard_gtids */ - shard_gtids?: (binlogdata.IShardGtid[]|null); + /** Rule target_unique_key_columns */ + target_unique_key_columns?: (string|null); - /** Journal participants */ - participants?: (binlogdata.IKeyspaceShard[]|null); + /** Rule source_unique_key_target_columns */ + source_unique_key_target_columns?: (string|null); - /** Journal source_workflows */ - source_workflows?: (string[]|null); + /** Rule convert_int_to_enum */ + convert_int_to_enum?: ({ [k: string]: boolean }|null); } - /** Represents a Journal. */ - class Journal implements IJournal { + /** Represents a Rule. */ + class Rule implements IRule { /** - * Constructs a new Journal. + * Constructs a new Rule. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IJournal); + constructor(properties?: binlogdata.IRule); - /** Journal id. */ - public id: (number|Long); + /** Rule match. */ + public match: string; - /** Journal migration_type. */ - public migration_type: binlogdata.MigrationType; + /** Rule filter. */ + public filter: string; - /** Journal tables. */ - public tables: string[]; + /** Rule convert_enum_to_text. */ + public convert_enum_to_text: { [k: string]: string }; - /** Journal local_position. */ - public local_position: string; + /** Rule convert_charset. */ + public convert_charset: { [k: string]: binlogdata.ICharsetConversion }; - /** Journal shard_gtids. */ - public shard_gtids: binlogdata.IShardGtid[]; + /** Rule source_unique_key_columns. */ + public source_unique_key_columns: string; - /** Journal participants. */ - public participants: binlogdata.IKeyspaceShard[]; + /** Rule target_unique_key_columns. */ + public target_unique_key_columns: string; - /** Journal source_workflows. */ - public source_workflows: string[]; + /** Rule source_unique_key_target_columns. */ + public source_unique_key_target_columns: string; + + /** Rule convert_int_to_enum. */ + public convert_int_to_enum: { [k: string]: boolean }; /** - * Creates a new Journal instance using the specified properties. + * Creates a new Rule instance using the specified properties. * @param [properties] Properties to set - * @returns Journal instance + * @returns Rule instance */ - public static create(properties?: binlogdata.IJournal): binlogdata.Journal; + public static create(properties?: binlogdata.IRule): binlogdata.Rule; /** - * Encodes the specified Journal message. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. - * @param message Journal message or plain object to encode + * Encodes the specified Rule message. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. + * @param message Rule message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IJournal, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Journal message, length delimited. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. - * @param message Journal message or plain object to encode + * Encodes the specified Rule message, length delimited. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. + * @param message Rule message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IJournal, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Journal message from the specified reader or buffer. + * Decodes a Rule message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Journal + * @returns Rule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Journal; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Rule; /** - * Decodes a Journal message from the specified reader or buffer, length delimited. + * Decodes a Rule message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Journal + * @returns Rule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Journal; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Rule; /** - * Verifies a Journal message. + * Verifies a Rule 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 Journal message from a plain object. Also converts values to their respective internal types. + * Creates a Rule message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Journal + * @returns Rule */ - public static fromObject(object: { [k: string]: any }): binlogdata.Journal; + public static fromObject(object: { [k: string]: any }): binlogdata.Rule; /** - * Creates a plain object from a Journal message. Also converts values to other types if specified. - * @param message Journal + * Creates a plain object from a Rule message. Also converts values to other types if specified. + * @param message Rule * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.Journal, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.Rule, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Journal to JSON. + * Converts this Rule to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Journal + * Gets the default type url for Rule * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VEvent. */ - interface IVEvent { - - /** VEvent type */ - type?: (binlogdata.VEventType|null); - - /** VEvent timestamp */ - timestamp?: (number|Long|null); - - /** VEvent gtid */ - gtid?: (string|null); - - /** VEvent statement */ - statement?: (string|null); - - /** VEvent row_event */ - row_event?: (binlogdata.IRowEvent|null); - - /** VEvent field_event */ - field_event?: (binlogdata.IFieldEvent|null); - - /** VEvent vgtid */ - vgtid?: (binlogdata.IVGtid|null); - - /** VEvent journal */ - journal?: (binlogdata.IJournal|null); - - /** VEvent dml */ - dml?: (string|null); - - /** VEvent current_time */ - current_time?: (number|Long|null); + /** Properties of a Filter. */ + interface IFilter { - /** VEvent last_p_k_event */ - last_p_k_event?: (binlogdata.ILastPKEvent|null); + /** Filter rules */ + rules?: (binlogdata.IRule[]|null); - /** VEvent keyspace */ - keyspace?: (string|null); + /** Filter field_event_mode */ + field_event_mode?: (binlogdata.Filter.FieldEventMode|null); - /** VEvent shard */ - shard?: (string|null); + /** Filter workflow_type */ + workflow_type?: (number|Long|null); - /** VEvent throttled */ - throttled?: (boolean|null); + /** Filter workflow_name */ + workflow_name?: (string|null); } - /** Represents a VEvent. */ - class VEvent implements IVEvent { + /** Represents a Filter. */ + class Filter implements IFilter { /** - * Constructs a new VEvent. + * Constructs a new Filter. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVEvent); - - /** VEvent type. */ - public type: binlogdata.VEventType; - - /** VEvent timestamp. */ - public timestamp: (number|Long); - - /** VEvent gtid. */ - public gtid: string; - - /** VEvent statement. */ - public statement: string; - - /** VEvent row_event. */ - public row_event?: (binlogdata.IRowEvent|null); - - /** VEvent field_event. */ - public field_event?: (binlogdata.IFieldEvent|null); - - /** VEvent vgtid. */ - public vgtid?: (binlogdata.IVGtid|null); - - /** VEvent journal. */ - public journal?: (binlogdata.IJournal|null); - - /** VEvent dml. */ - public dml: string; - - /** VEvent current_time. */ - public current_time: (number|Long); + constructor(properties?: binlogdata.IFilter); - /** VEvent last_p_k_event. */ - public last_p_k_event?: (binlogdata.ILastPKEvent|null); + /** Filter rules. */ + public rules: binlogdata.IRule[]; - /** VEvent keyspace. */ - public keyspace: string; + /** Filter field_event_mode. */ + public field_event_mode: binlogdata.Filter.FieldEventMode; - /** VEvent shard. */ - public shard: string; + /** Filter workflow_type. */ + public workflow_type: (number|Long); - /** VEvent throttled. */ - public throttled: boolean; + /** Filter workflow_name. */ + public workflow_name: string; /** - * Creates a new VEvent instance using the specified properties. + * Creates a new Filter instance using the specified properties. * @param [properties] Properties to set - * @returns VEvent instance + * @returns Filter instance */ - public static create(properties?: binlogdata.IVEvent): binlogdata.VEvent; + public static create(properties?: binlogdata.IFilter): binlogdata.Filter; /** - * Encodes the specified VEvent message. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. - * @param message VEvent message or plain object to encode + * Encodes the specified Filter message. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. + * @param message Filter message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IFilter, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VEvent message, length delimited. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. - * @param message VEvent message or plain object to encode + * Encodes the specified Filter message, length delimited. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. + * @param message Filter message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IFilter, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VEvent message from the specified reader or buffer. + * Decodes a Filter message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VEvent + * @returns Filter * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VEvent; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Filter; /** - * Decodes a VEvent message from the specified reader or buffer, length delimited. + * Decodes a Filter message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VEvent + * @returns Filter * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VEvent; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Filter; /** - * Verifies a VEvent message. + * Verifies a Filter 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 VEvent message from a plain object. Also converts values to their respective internal types. + * Creates a Filter message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VEvent + * @returns Filter */ - public static fromObject(object: { [k: string]: any }): binlogdata.VEvent; + public static fromObject(object: { [k: string]: any }): binlogdata.Filter; /** - * Creates a plain object from a VEvent message. Also converts values to other types if specified. - * @param message VEvent + * Creates a plain object from a Filter message. Also converts values to other types if specified. + * @param message Filter * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.Filter, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VEvent to JSON. + * Converts this Filter to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VEvent + * Gets the default type url for Filter * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MinimalTable. */ - interface IMinimalTable { + namespace Filter { - /** MinimalTable name */ - name?: (string|null); + /** FieldEventMode enum. */ + enum FieldEventMode { + ERR_ON_MISMATCH = 0, + BEST_EFFORT = 1 + } + } - /** MinimalTable fields */ - fields?: (query.IField[]|null); + /** OnDDLAction enum. */ + enum OnDDLAction { + IGNORE = 0, + STOP = 1, + EXEC = 2, + EXEC_IGNORE = 3 + } - /** MinimalTable p_k_columns */ - p_k_columns?: ((number|Long)[]|null); + /** VReplicationWorkflowType enum. */ + enum VReplicationWorkflowType { + Materialize = 0, + MoveTables = 1, + CreateLookupIndex = 2, + Migrate = 3, + Reshard = 4, + OnlineDDL = 5 } - /** Represents a MinimalTable. */ - class MinimalTable implements IMinimalTable { + /** VReplicationWorkflowSubType enum. */ + enum VReplicationWorkflowSubType { + None = 0, + Partial = 1 + } + + /** VReplicationWorkflowState enum. */ + enum VReplicationWorkflowState { + Unknown = 0, + Init = 1, + Stopped = 2, + Copying = 3, + Running = 4, + Error = 5, + Lagging = 6 + } + + /** Properties of a BinlogSource. */ + interface IBinlogSource { + + /** BinlogSource keyspace */ + keyspace?: (string|null); + + /** BinlogSource shard */ + shard?: (string|null); + + /** BinlogSource tablet_type */ + tablet_type?: (topodata.TabletType|null); + + /** BinlogSource key_range */ + key_range?: (topodata.IKeyRange|null); + + /** BinlogSource tables */ + tables?: (string[]|null); + + /** BinlogSource filter */ + filter?: (binlogdata.IFilter|null); + + /** BinlogSource on_ddl */ + on_ddl?: (binlogdata.OnDDLAction|null); + + /** BinlogSource external_mysql */ + external_mysql?: (string|null); + + /** BinlogSource stop_after_copy */ + stop_after_copy?: (boolean|null); + + /** BinlogSource external_cluster */ + external_cluster?: (string|null); + + /** BinlogSource source_time_zone */ + source_time_zone?: (string|null); + + /** BinlogSource target_time_zone */ + target_time_zone?: (string|null); + } + + /** Represents a BinlogSource. */ + class BinlogSource implements IBinlogSource { /** - * Constructs a new MinimalTable. + * Constructs a new BinlogSource. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IMinimalTable); + constructor(properties?: binlogdata.IBinlogSource); - /** MinimalTable name. */ - public name: string; + /** BinlogSource keyspace. */ + public keyspace: string; - /** MinimalTable fields. */ - public fields: query.IField[]; + /** BinlogSource shard. */ + public shard: string; - /** MinimalTable p_k_columns. */ - public p_k_columns: (number|Long)[]; + /** BinlogSource tablet_type. */ + public tablet_type: topodata.TabletType; + + /** BinlogSource key_range. */ + public key_range?: (topodata.IKeyRange|null); + + /** BinlogSource tables. */ + public tables: string[]; + + /** BinlogSource filter. */ + public filter?: (binlogdata.IFilter|null); + + /** BinlogSource on_ddl. */ + public on_ddl: binlogdata.OnDDLAction; + + /** BinlogSource external_mysql. */ + public external_mysql: string; + + /** BinlogSource stop_after_copy. */ + public stop_after_copy: boolean; + + /** BinlogSource external_cluster. */ + public external_cluster: string; + + /** BinlogSource source_time_zone. */ + public source_time_zone: string; + + /** BinlogSource target_time_zone. */ + public target_time_zone: string; /** - * Creates a new MinimalTable instance using the specified properties. + * Creates a new BinlogSource instance using the specified properties. * @param [properties] Properties to set - * @returns MinimalTable instance + * @returns BinlogSource instance */ - public static create(properties?: binlogdata.IMinimalTable): binlogdata.MinimalTable; + public static create(properties?: binlogdata.IBinlogSource): binlogdata.BinlogSource; /** - * Encodes the specified MinimalTable message. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. - * @param message MinimalTable message or plain object to encode + * Encodes the specified BinlogSource message. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. + * @param message BinlogSource message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IMinimalTable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IBinlogSource, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MinimalTable message, length delimited. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. - * @param message MinimalTable message or plain object to encode + * Encodes the specified BinlogSource message, length delimited. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. + * @param message BinlogSource message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IMinimalTable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IBinlogSource, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MinimalTable message from the specified reader or buffer. + * Decodes a BinlogSource message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MinimalTable + * @returns BinlogSource * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.MinimalTable; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.BinlogSource; /** - * Decodes a MinimalTable message from the specified reader or buffer, length delimited. + * Decodes a BinlogSource message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MinimalTable + * @returns BinlogSource * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.MinimalTable; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.BinlogSource; /** - * Verifies a MinimalTable message. + * Verifies a BinlogSource 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 MinimalTable message from a plain object. Also converts values to their respective internal types. + * Creates a BinlogSource message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MinimalTable + * @returns BinlogSource */ - public static fromObject(object: { [k: string]: any }): binlogdata.MinimalTable; + public static fromObject(object: { [k: string]: any }): binlogdata.BinlogSource; /** - * Creates a plain object from a MinimalTable message. Also converts values to other types if specified. - * @param message MinimalTable + * Creates a plain object from a BinlogSource message. Also converts values to other types if specified. + * @param message BinlogSource * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.MinimalTable, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.BinlogSource, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MinimalTable to JSON. + * Converts this BinlogSource to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MinimalTable + * Gets the default type url for BinlogSource * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MinimalSchema. */ - interface IMinimalSchema { + /** VEventType enum. */ + enum VEventType { + UNKNOWN = 0, + GTID = 1, + BEGIN = 2, + COMMIT = 3, + ROLLBACK = 4, + DDL = 5, + INSERT = 6, + REPLACE = 7, + UPDATE = 8, + DELETE = 9, + SET = 10, + OTHER = 11, + ROW = 12, + FIELD = 13, + HEARTBEAT = 14, + VGTID = 15, + JOURNAL = 16, + VERSION = 17, + LASTPK = 18, + SAVEPOINT = 19, + COPY_COMPLETED = 20 + } - /** MinimalSchema tables */ - tables?: (binlogdata.IMinimalTable[]|null); + /** Properties of a RowChange. */ + interface IRowChange { + + /** RowChange before */ + before?: (query.IRow|null); + + /** RowChange after */ + after?: (query.IRow|null); + + /** RowChange data_columns */ + data_columns?: (binlogdata.RowChange.IBitmap|null); } - /** Represents a MinimalSchema. */ - class MinimalSchema implements IMinimalSchema { + /** Represents a RowChange. */ + class RowChange implements IRowChange { /** - * Constructs a new MinimalSchema. + * Constructs a new RowChange. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IMinimalSchema); + constructor(properties?: binlogdata.IRowChange); - /** MinimalSchema tables. */ - public tables: binlogdata.IMinimalTable[]; + /** RowChange before. */ + public before?: (query.IRow|null); + + /** RowChange after. */ + public after?: (query.IRow|null); + + /** RowChange data_columns. */ + public data_columns?: (binlogdata.RowChange.IBitmap|null); /** - * Creates a new MinimalSchema instance using the specified properties. + * Creates a new RowChange instance using the specified properties. * @param [properties] Properties to set - * @returns MinimalSchema instance + * @returns RowChange instance */ - public static create(properties?: binlogdata.IMinimalSchema): binlogdata.MinimalSchema; + public static create(properties?: binlogdata.IRowChange): binlogdata.RowChange; /** - * Encodes the specified MinimalSchema message. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. - * @param message MinimalSchema message or plain object to encode + * Encodes the specified RowChange message. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. + * @param message RowChange message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IMinimalSchema, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IRowChange, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MinimalSchema message, length delimited. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. - * @param message MinimalSchema message or plain object to encode + * Encodes the specified RowChange message, length delimited. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. + * @param message RowChange message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IMinimalSchema, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IRowChange, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MinimalSchema message from the specified reader or buffer. + * Decodes a RowChange message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MinimalSchema + * @returns RowChange * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.MinimalSchema; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowChange; /** - * Decodes a MinimalSchema message from the specified reader or buffer, length delimited. + * Decodes a RowChange message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MinimalSchema + * @returns RowChange * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.MinimalSchema; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowChange; /** - * Verifies a MinimalSchema message. + * Verifies a RowChange 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 MinimalSchema message from a plain object. Also converts values to their respective internal types. + * Creates a RowChange message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MinimalSchema + * @returns RowChange */ - public static fromObject(object: { [k: string]: any }): binlogdata.MinimalSchema; + public static fromObject(object: { [k: string]: any }): binlogdata.RowChange; /** - * Creates a plain object from a MinimalSchema message. Also converts values to other types if specified. - * @param message MinimalSchema + * Creates a plain object from a RowChange message. Also converts values to other types if specified. + * @param message RowChange * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.MinimalSchema, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.RowChange, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MinimalSchema to JSON. + * Converts this RowChange to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MinimalSchema + * Gets the default type url for RowChange * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamRequest. */ - interface IVStreamRequest { + namespace RowChange { - /** VStreamRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); + /** Properties of a Bitmap. */ + interface IBitmap { - /** VStreamRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** Bitmap count */ + count?: (number|Long|null); - /** VStreamRequest target */ - target?: (query.ITarget|null); + /** Bitmap cols */ + cols?: (Uint8Array|null); + } - /** VStreamRequest position */ - position?: (string|null); + /** Represents a Bitmap. */ + class Bitmap implements IBitmap { - /** VStreamRequest filter */ - filter?: (binlogdata.IFilter|null); + /** + * Constructs a new Bitmap. + * @param [properties] Properties to set + */ + constructor(properties?: binlogdata.RowChange.IBitmap); - /** VStreamRequest table_last_p_ks */ - table_last_p_ks?: (binlogdata.ITableLastPK[]|null); + /** Bitmap count. */ + public count: (number|Long); + + /** Bitmap cols. */ + public cols: Uint8Array; + + /** + * Creates a new Bitmap instance using the specified properties. + * @param [properties] Properties to set + * @returns Bitmap instance + */ + public static create(properties?: binlogdata.RowChange.IBitmap): binlogdata.RowChange.Bitmap; + + /** + * Encodes the specified Bitmap message. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. + * @param message Bitmap message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: binlogdata.RowChange.IBitmap, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Bitmap message, length delimited. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. + * @param message Bitmap message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: binlogdata.RowChange.IBitmap, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Bitmap message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Bitmap + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowChange.Bitmap; + + /** + * Decodes a Bitmap message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Bitmap + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowChange.Bitmap; + + /** + * Verifies a Bitmap 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 Bitmap message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Bitmap + */ + public static fromObject(object: { [k: string]: any }): binlogdata.RowChange.Bitmap; + + /** + * Creates a plain object from a Bitmap message. Also converts values to other types if specified. + * @param message Bitmap + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: binlogdata.RowChange.Bitmap, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Bitmap to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Bitmap + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } } - /** Represents a VStreamRequest. */ - class VStreamRequest implements IVStreamRequest { + /** Properties of a RowEvent. */ + interface IRowEvent { + + /** RowEvent table_name */ + table_name?: (string|null); + + /** RowEvent row_changes */ + row_changes?: (binlogdata.IRowChange[]|null); + + /** RowEvent keyspace */ + keyspace?: (string|null); + + /** RowEvent shard */ + shard?: (string|null); + } + + /** Represents a RowEvent. */ + class RowEvent implements IRowEvent { /** - * Constructs a new VStreamRequest. + * Constructs a new RowEvent. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamRequest); - - /** VStreamRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); - - /** VStreamRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + constructor(properties?: binlogdata.IRowEvent); - /** VStreamRequest target. */ - public target?: (query.ITarget|null); + /** RowEvent table_name. */ + public table_name: string; - /** VStreamRequest position. */ - public position: string; + /** RowEvent row_changes. */ + public row_changes: binlogdata.IRowChange[]; - /** VStreamRequest filter. */ - public filter?: (binlogdata.IFilter|null); + /** RowEvent keyspace. */ + public keyspace: string; - /** VStreamRequest table_last_p_ks. */ - public table_last_p_ks: binlogdata.ITableLastPK[]; + /** RowEvent shard. */ + public shard: string; /** - * Creates a new VStreamRequest instance using the specified properties. + * Creates a new RowEvent instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamRequest instance + * @returns RowEvent instance */ - public static create(properties?: binlogdata.IVStreamRequest): binlogdata.VStreamRequest; + public static create(properties?: binlogdata.IRowEvent): binlogdata.RowEvent; /** - * Encodes the specified VStreamRequest message. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. - * @param message VStreamRequest message or plain object to encode + * Encodes the specified RowEvent message. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. + * @param message RowEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IRowEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. - * @param message VStreamRequest message or plain object to encode + * Encodes the specified RowEvent message, length delimited. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. + * @param message RowEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IRowEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamRequest message from the specified reader or buffer. + * Decodes a RowEvent message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamRequest + * @returns RowEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.RowEvent; /** - * Decodes a VStreamRequest message from the specified reader or buffer, length delimited. + * Decodes a RowEvent message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamRequest + * @returns RowEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.RowEvent; /** - * Verifies a VStreamRequest message. + * Verifies a RowEvent 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 VStreamRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RowEvent message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamRequest + * @returns RowEvent */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRequest; + public static fromObject(object: { [k: string]: any }): binlogdata.RowEvent; /** - * Creates a plain object from a VStreamRequest message. Also converts values to other types if specified. - * @param message VStreamRequest + * Creates a plain object from a RowEvent message. Also converts values to other types if specified. + * @param message RowEvent * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.RowEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamRequest to JSON. + * Converts this RowEvent to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamRequest + * Gets the default type url for RowEvent * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamResponse. */ - interface IVStreamResponse { + /** Properties of a FieldEvent. */ + interface IFieldEvent { - /** VStreamResponse events */ - events?: (binlogdata.IVEvent[]|null); + /** FieldEvent table_name */ + table_name?: (string|null); + + /** FieldEvent fields */ + fields?: (query.IField[]|null); + + /** FieldEvent keyspace */ + keyspace?: (string|null); + + /** FieldEvent shard */ + shard?: (string|null); } - /** Represents a VStreamResponse. */ - class VStreamResponse implements IVStreamResponse { + /** Represents a FieldEvent. */ + class FieldEvent implements IFieldEvent { /** - * Constructs a new VStreamResponse. + * Constructs a new FieldEvent. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamResponse); + constructor(properties?: binlogdata.IFieldEvent); - /** VStreamResponse events. */ - public events: binlogdata.IVEvent[]; + /** FieldEvent table_name. */ + public table_name: string; + + /** FieldEvent fields. */ + public fields: query.IField[]; + + /** FieldEvent keyspace. */ + public keyspace: string; + + /** FieldEvent shard. */ + public shard: string; /** - * Creates a new VStreamResponse instance using the specified properties. + * Creates a new FieldEvent instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamResponse instance + * @returns FieldEvent instance */ - public static create(properties?: binlogdata.IVStreamResponse): binlogdata.VStreamResponse; + public static create(properties?: binlogdata.IFieldEvent): binlogdata.FieldEvent; /** - * Encodes the specified VStreamResponse message. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. - * @param message VStreamResponse message or plain object to encode + * Encodes the specified FieldEvent message. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. + * @param message FieldEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IFieldEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. - * @param message VStreamResponse message or plain object to encode + * Encodes the specified FieldEvent message, length delimited. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. + * @param message FieldEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IFieldEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamResponse message from the specified reader or buffer. + * Decodes a FieldEvent message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamResponse + * @returns FieldEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.FieldEvent; /** - * Decodes a VStreamResponse message from the specified reader or buffer, length delimited. + * Decodes a FieldEvent message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamResponse + * @returns FieldEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.FieldEvent; /** - * Verifies a VStreamResponse message. + * Verifies a FieldEvent 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 VStreamResponse message from a plain object. Also converts values to their respective internal types. + * Creates a FieldEvent message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamResponse + * @returns FieldEvent */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResponse; + public static fromObject(object: { [k: string]: any }): binlogdata.FieldEvent; /** - * Creates a plain object from a VStreamResponse message. Also converts values to other types if specified. - * @param message VStreamResponse + * Creates a plain object from a FieldEvent message. Also converts values to other types if specified. + * @param message FieldEvent * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.FieldEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamResponse to JSON. + * Converts this FieldEvent to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamResponse + * Gets the default type url for FieldEvent * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamRowsRequest. */ - interface IVStreamRowsRequest { - - /** VStreamRowsRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); + /** Properties of a ShardGtid. */ + interface IShardGtid { - /** VStreamRowsRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** ShardGtid keyspace */ + keyspace?: (string|null); - /** VStreamRowsRequest target */ - target?: (query.ITarget|null); + /** ShardGtid shard */ + shard?: (string|null); - /** VStreamRowsRequest query */ - query?: (string|null); + /** ShardGtid gtid */ + gtid?: (string|null); - /** VStreamRowsRequest lastpk */ - lastpk?: (query.IQueryResult|null); + /** ShardGtid table_p_ks */ + table_p_ks?: (binlogdata.ITableLastPK[]|null); } - /** Represents a VStreamRowsRequest. */ - class VStreamRowsRequest implements IVStreamRowsRequest { + /** Represents a ShardGtid. */ + class ShardGtid implements IShardGtid { /** - * Constructs a new VStreamRowsRequest. + * Constructs a new ShardGtid. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamRowsRequest); - - /** VStreamRowsRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); + constructor(properties?: binlogdata.IShardGtid); - /** VStreamRowsRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + /** ShardGtid keyspace. */ + public keyspace: string; - /** VStreamRowsRequest target. */ - public target?: (query.ITarget|null); + /** ShardGtid shard. */ + public shard: string; - /** VStreamRowsRequest query. */ - public query: string; + /** ShardGtid gtid. */ + public gtid: string; - /** VStreamRowsRequest lastpk. */ - public lastpk?: (query.IQueryResult|null); + /** ShardGtid table_p_ks. */ + public table_p_ks: binlogdata.ITableLastPK[]; /** - * Creates a new VStreamRowsRequest instance using the specified properties. + * Creates a new ShardGtid instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamRowsRequest instance + * @returns ShardGtid instance */ - public static create(properties?: binlogdata.IVStreamRowsRequest): binlogdata.VStreamRowsRequest; + public static create(properties?: binlogdata.IShardGtid): binlogdata.ShardGtid; /** - * Encodes the specified VStreamRowsRequest message. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. - * @param message VStreamRowsRequest message or plain object to encode + * Encodes the specified ShardGtid message. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. + * @param message ShardGtid message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamRowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IShardGtid, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamRowsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. - * @param message VStreamRowsRequest message or plain object to encode + * Encodes the specified ShardGtid message, length delimited. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. + * @param message ShardGtid message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamRowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IShardGtid, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamRowsRequest message from the specified reader or buffer. + * Decodes a ShardGtid message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamRowsRequest + * @returns ShardGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRowsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.ShardGtid; /** - * Decodes a VStreamRowsRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardGtid message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamRowsRequest + * @returns ShardGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRowsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.ShardGtid; /** - * Verifies a VStreamRowsRequest message. + * Verifies a ShardGtid 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 VStreamRowsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardGtid message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamRowsRequest + * @returns ShardGtid */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRowsRequest; + public static fromObject(object: { [k: string]: any }): binlogdata.ShardGtid; /** - * Creates a plain object from a VStreamRowsRequest message. Also converts values to other types if specified. - * @param message VStreamRowsRequest + * Creates a plain object from a ShardGtid message. Also converts values to other types if specified. + * @param message ShardGtid * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamRowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.ShardGtid, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamRowsRequest to JSON. + * Converts this ShardGtid to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamRowsRequest + * Gets the default type url for ShardGtid * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamRowsResponse. */ - interface IVStreamRowsResponse { - - /** VStreamRowsResponse fields */ - fields?: (query.IField[]|null); - - /** VStreamRowsResponse pkfields */ - pkfields?: (query.IField[]|null); - - /** VStreamRowsResponse gtid */ - gtid?: (string|null); - - /** VStreamRowsResponse rows */ - rows?: (query.IRow[]|null); - - /** VStreamRowsResponse lastpk */ - lastpk?: (query.IRow|null); - - /** VStreamRowsResponse throttled */ - throttled?: (boolean|null); + /** Properties of a VGtid. */ + interface IVGtid { - /** VStreamRowsResponse heartbeat */ - heartbeat?: (boolean|null); + /** VGtid shard_gtids */ + shard_gtids?: (binlogdata.IShardGtid[]|null); } - /** Represents a VStreamRowsResponse. */ - class VStreamRowsResponse implements IVStreamRowsResponse { + /** Represents a VGtid. */ + class VGtid implements IVGtid { /** - * Constructs a new VStreamRowsResponse. + * Constructs a new VGtid. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamRowsResponse); - - /** VStreamRowsResponse fields. */ - public fields: query.IField[]; - - /** VStreamRowsResponse pkfields. */ - public pkfields: query.IField[]; - - /** VStreamRowsResponse gtid. */ - public gtid: string; - - /** VStreamRowsResponse rows. */ - public rows: query.IRow[]; - - /** VStreamRowsResponse lastpk. */ - public lastpk?: (query.IRow|null); - - /** VStreamRowsResponse throttled. */ - public throttled: boolean; + constructor(properties?: binlogdata.IVGtid); - /** VStreamRowsResponse heartbeat. */ - public heartbeat: boolean; + /** VGtid shard_gtids. */ + public shard_gtids: binlogdata.IShardGtid[]; /** - * Creates a new VStreamRowsResponse instance using the specified properties. + * Creates a new VGtid instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamRowsResponse instance + * @returns VGtid instance */ - public static create(properties?: binlogdata.IVStreamRowsResponse): binlogdata.VStreamRowsResponse; + public static create(properties?: binlogdata.IVGtid): binlogdata.VGtid; /** - * Encodes the specified VStreamRowsResponse message. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. - * @param message VStreamRowsResponse message or plain object to encode + * Encodes the specified VGtid message. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. + * @param message VGtid message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamRowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVGtid, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamRowsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. - * @param message VStreamRowsResponse message or plain object to encode + * Encodes the specified VGtid message, length delimited. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. + * @param message VGtid message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamRowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVGtid, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamRowsResponse message from the specified reader or buffer. + * Decodes a VGtid message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamRowsResponse + * @returns VGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRowsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VGtid; /** - * Decodes a VStreamRowsResponse message from the specified reader or buffer, length delimited. + * Decodes a VGtid message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamRowsResponse + * @returns VGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRowsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VGtid; /** - * Verifies a VStreamRowsResponse message. + * Verifies a VGtid 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 VStreamRowsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VGtid message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamRowsResponse + * @returns VGtid */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRowsResponse; + public static fromObject(object: { [k: string]: any }): binlogdata.VGtid; /** - * Creates a plain object from a VStreamRowsResponse message. Also converts values to other types if specified. - * @param message VStreamRowsResponse + * Creates a plain object from a VGtid message. Also converts values to other types if specified. + * @param message VGtid * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamRowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VGtid, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamRowsResponse to JSON. + * Converts this VGtid to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamRowsResponse + * Gets the default type url for VGtid * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a LastPKEvent. */ - interface ILastPKEvent { + /** Properties of a KeyspaceShard. */ + interface IKeyspaceShard { - /** LastPKEvent table_last_p_k */ - table_last_p_k?: (binlogdata.ITableLastPK|null); + /** KeyspaceShard keyspace */ + keyspace?: (string|null); - /** LastPKEvent completed */ - completed?: (boolean|null); + /** KeyspaceShard shard */ + shard?: (string|null); } - /** Represents a LastPKEvent. */ - class LastPKEvent implements ILastPKEvent { + /** Represents a KeyspaceShard. */ + class KeyspaceShard implements IKeyspaceShard { /** - * Constructs a new LastPKEvent. + * Constructs a new KeyspaceShard. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.ILastPKEvent); + constructor(properties?: binlogdata.IKeyspaceShard); - /** LastPKEvent table_last_p_k. */ - public table_last_p_k?: (binlogdata.ITableLastPK|null); + /** KeyspaceShard keyspace. */ + public keyspace: string; - /** LastPKEvent completed. */ - public completed: boolean; + /** KeyspaceShard shard. */ + public shard: string; /** - * Creates a new LastPKEvent instance using the specified properties. + * Creates a new KeyspaceShard instance using the specified properties. * @param [properties] Properties to set - * @returns LastPKEvent instance + * @returns KeyspaceShard instance */ - public static create(properties?: binlogdata.ILastPKEvent): binlogdata.LastPKEvent; + public static create(properties?: binlogdata.IKeyspaceShard): binlogdata.KeyspaceShard; /** - * Encodes the specified LastPKEvent message. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. - * @param message LastPKEvent message or plain object to encode + * Encodes the specified KeyspaceShard message. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. + * @param message KeyspaceShard message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.ILastPKEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IKeyspaceShard, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified LastPKEvent message, length delimited. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. - * @param message LastPKEvent message or plain object to encode + * Encodes the specified KeyspaceShard message, length delimited. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. + * @param message KeyspaceShard message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.ILastPKEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IKeyspaceShard, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a LastPKEvent message from the specified reader or buffer. + * Decodes a KeyspaceShard message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns LastPKEvent + * @returns KeyspaceShard * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.LastPKEvent; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.KeyspaceShard; /** - * Decodes a LastPKEvent message from the specified reader or buffer, length delimited. + * Decodes a KeyspaceShard message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns LastPKEvent + * @returns KeyspaceShard * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.LastPKEvent; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.KeyspaceShard; /** - * Verifies a LastPKEvent message. + * Verifies a KeyspaceShard 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 LastPKEvent message from a plain object. Also converts values to their respective internal types. + * Creates a KeyspaceShard message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns LastPKEvent + * @returns KeyspaceShard */ - public static fromObject(object: { [k: string]: any }): binlogdata.LastPKEvent; + public static fromObject(object: { [k: string]: any }): binlogdata.KeyspaceShard; /** - * Creates a plain object from a LastPKEvent message. Also converts values to other types if specified. - * @param message LastPKEvent + * Creates a plain object from a KeyspaceShard message. Also converts values to other types if specified. + * @param message KeyspaceShard * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.LastPKEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.KeyspaceShard, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this LastPKEvent to JSON. + * Converts this KeyspaceShard to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for LastPKEvent + * Gets the default type url for KeyspaceShard * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a TableLastPK. */ - interface ITableLastPK { + /** MigrationType enum. */ + enum MigrationType { + TABLES = 0, + SHARDS = 1 + } - /** TableLastPK table_name */ - table_name?: (string|null); + /** Properties of a Journal. */ + interface IJournal { - /** TableLastPK lastpk */ - lastpk?: (query.IQueryResult|null); + /** Journal id */ + id?: (number|Long|null); + + /** Journal migration_type */ + migration_type?: (binlogdata.MigrationType|null); + + /** Journal tables */ + tables?: (string[]|null); + + /** Journal local_position */ + local_position?: (string|null); + + /** Journal shard_gtids */ + shard_gtids?: (binlogdata.IShardGtid[]|null); + + /** Journal participants */ + participants?: (binlogdata.IKeyspaceShard[]|null); + + /** Journal source_workflows */ + source_workflows?: (string[]|null); } - /** Represents a TableLastPK. */ - class TableLastPK implements ITableLastPK { + /** Represents a Journal. */ + class Journal implements IJournal { /** - * Constructs a new TableLastPK. + * Constructs a new Journal. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.ITableLastPK); + constructor(properties?: binlogdata.IJournal); - /** TableLastPK table_name. */ - public table_name: string; + /** Journal id. */ + public id: (number|Long); - /** TableLastPK lastpk. */ - public lastpk?: (query.IQueryResult|null); + /** Journal migration_type. */ + public migration_type: binlogdata.MigrationType; + + /** Journal tables. */ + public tables: string[]; + + /** Journal local_position. */ + public local_position: string; + + /** Journal shard_gtids. */ + public shard_gtids: binlogdata.IShardGtid[]; + + /** Journal participants. */ + public participants: binlogdata.IKeyspaceShard[]; + + /** Journal source_workflows. */ + public source_workflows: string[]; /** - * Creates a new TableLastPK instance using the specified properties. + * Creates a new Journal instance using the specified properties. * @param [properties] Properties to set - * @returns TableLastPK instance + * @returns Journal instance */ - public static create(properties?: binlogdata.ITableLastPK): binlogdata.TableLastPK; + public static create(properties?: binlogdata.IJournal): binlogdata.Journal; /** - * Encodes the specified TableLastPK message. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. - * @param message TableLastPK message or plain object to encode + * Encodes the specified Journal message. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. + * @param message Journal message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.ITableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IJournal, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TableLastPK message, length delimited. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. - * @param message TableLastPK message or plain object to encode + * Encodes the specified Journal message, length delimited. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. + * @param message Journal message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.ITableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IJournal, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TableLastPK message from the specified reader or buffer. + * Decodes a Journal message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TableLastPK + * @returns Journal * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.TableLastPK; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.Journal; /** - * Decodes a TableLastPK message from the specified reader or buffer, length delimited. + * Decodes a Journal message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TableLastPK + * @returns Journal * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.TableLastPK; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.Journal; /** - * Verifies a TableLastPK message. + * Verifies a Journal 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 TableLastPK message from a plain object. Also converts values to their respective internal types. + * Creates a Journal message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TableLastPK + * @returns Journal */ - public static fromObject(object: { [k: string]: any }): binlogdata.TableLastPK; + public static fromObject(object: { [k: string]: any }): binlogdata.Journal; /** - * Creates a plain object from a TableLastPK message. Also converts values to other types if specified. - * @param message TableLastPK + * Creates a plain object from a Journal message. Also converts values to other types if specified. + * @param message Journal * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.TableLastPK, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.Journal, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TableLastPK to JSON. + * Converts this Journal to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TableLastPK + * Gets the default type url for Journal * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamResultsRequest. */ - interface IVStreamResultsRequest { + /** Properties of a VEvent. */ + interface IVEvent { - /** VStreamResultsRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); + /** VEvent type */ + type?: (binlogdata.VEventType|null); - /** VStreamResultsRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** VEvent timestamp */ + timestamp?: (number|Long|null); - /** VStreamResultsRequest target */ - target?: (query.ITarget|null); + /** VEvent gtid */ + gtid?: (string|null); - /** VStreamResultsRequest query */ - query?: (string|null); + /** VEvent statement */ + statement?: (string|null); + + /** VEvent row_event */ + row_event?: (binlogdata.IRowEvent|null); + + /** VEvent field_event */ + field_event?: (binlogdata.IFieldEvent|null); + + /** VEvent vgtid */ + vgtid?: (binlogdata.IVGtid|null); + + /** VEvent journal */ + journal?: (binlogdata.IJournal|null); + + /** VEvent dml */ + dml?: (string|null); + + /** VEvent current_time */ + current_time?: (number|Long|null); + + /** VEvent last_p_k_event */ + last_p_k_event?: (binlogdata.ILastPKEvent|null); + + /** VEvent keyspace */ + keyspace?: (string|null); + + /** VEvent shard */ + shard?: (string|null); + + /** VEvent throttled */ + throttled?: (boolean|null); } - /** Represents a VStreamResultsRequest. */ - class VStreamResultsRequest implements IVStreamResultsRequest { + /** Represents a VEvent. */ + class VEvent implements IVEvent { /** - * Constructs a new VStreamResultsRequest. + * Constructs a new VEvent. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamResultsRequest); + constructor(properties?: binlogdata.IVEvent); - /** VStreamResultsRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); + /** VEvent type. */ + public type: binlogdata.VEventType; - /** VStreamResultsRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + /** VEvent timestamp. */ + public timestamp: (number|Long); - /** VStreamResultsRequest target. */ - public target?: (query.ITarget|null); + /** VEvent gtid. */ + public gtid: string; - /** VStreamResultsRequest query. */ - public query: string; + /** VEvent statement. */ + public statement: string; + + /** VEvent row_event. */ + public row_event?: (binlogdata.IRowEvent|null); + + /** VEvent field_event. */ + public field_event?: (binlogdata.IFieldEvent|null); + + /** VEvent vgtid. */ + public vgtid?: (binlogdata.IVGtid|null); + + /** VEvent journal. */ + public journal?: (binlogdata.IJournal|null); + + /** VEvent dml. */ + public dml: string; + + /** VEvent current_time. */ + public current_time: (number|Long); + + /** VEvent last_p_k_event. */ + public last_p_k_event?: (binlogdata.ILastPKEvent|null); + + /** VEvent keyspace. */ + public keyspace: string; + + /** VEvent shard. */ + public shard: string; + + /** VEvent throttled. */ + public throttled: boolean; /** - * Creates a new VStreamResultsRequest instance using the specified properties. + * Creates a new VEvent instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamResultsRequest instance + * @returns VEvent instance */ - public static create(properties?: binlogdata.IVStreamResultsRequest): binlogdata.VStreamResultsRequest; + public static create(properties?: binlogdata.IVEvent): binlogdata.VEvent; /** - * Encodes the specified VStreamResultsRequest message. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. - * @param message VStreamResultsRequest message or plain object to encode + * Encodes the specified VEvent message. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. + * @param message VEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamResultsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamResultsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. - * @param message VStreamResultsRequest message or plain object to encode + * Encodes the specified VEvent message, length delimited. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. + * @param message VEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamResultsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamResultsRequest message from the specified reader or buffer. + * Decodes a VEvent message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamResultsRequest + * @returns VEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResultsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VEvent; /** - * Decodes a VStreamResultsRequest message from the specified reader or buffer, length delimited. + * Decodes a VEvent message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamResultsRequest + * @returns VEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResultsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VEvent; /** - * Verifies a VStreamResultsRequest message. + * Verifies a VEvent 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 VStreamResultsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VEvent message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamResultsRequest + * @returns VEvent */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResultsRequest; + public static fromObject(object: { [k: string]: any }): binlogdata.VEvent; /** - * Creates a plain object from a VStreamResultsRequest message. Also converts values to other types if specified. - * @param message VStreamResultsRequest + * Creates a plain object from a VEvent message. Also converts values to other types if specified. + * @param message VEvent * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamResultsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamResultsRequest to JSON. + * Converts this VEvent to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamResultsRequest + * Gets the default type url for VEvent * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VStreamResultsResponse. */ - interface IVStreamResultsResponse { + /** Properties of a MinimalTable. */ + interface IMinimalTable { - /** VStreamResultsResponse fields */ - fields?: (query.IField[]|null); + /** MinimalTable name */ + name?: (string|null); - /** VStreamResultsResponse gtid */ - gtid?: (string|null); + /** MinimalTable fields */ + fields?: (query.IField[]|null); - /** VStreamResultsResponse rows */ - rows?: (query.IRow[]|null); + /** MinimalTable p_k_columns */ + p_k_columns?: ((number|Long)[]|null); } - /** Represents a VStreamResultsResponse. */ - class VStreamResultsResponse implements IVStreamResultsResponse { + /** Represents a MinimalTable. */ + class MinimalTable implements IMinimalTable { /** - * Constructs a new VStreamResultsResponse. + * Constructs a new MinimalTable. * @param [properties] Properties to set */ - constructor(properties?: binlogdata.IVStreamResultsResponse); + constructor(properties?: binlogdata.IMinimalTable); - /** VStreamResultsResponse fields. */ - public fields: query.IField[]; + /** MinimalTable name. */ + public name: string; - /** VStreamResultsResponse gtid. */ - public gtid: string; + /** MinimalTable fields. */ + public fields: query.IField[]; - /** VStreamResultsResponse rows. */ - public rows: query.IRow[]; + /** MinimalTable p_k_columns. */ + public p_k_columns: (number|Long)[]; /** - * Creates a new VStreamResultsResponse instance using the specified properties. + * Creates a new MinimalTable instance using the specified properties. * @param [properties] Properties to set - * @returns VStreamResultsResponse instance + * @returns MinimalTable instance */ - public static create(properties?: binlogdata.IVStreamResultsResponse): binlogdata.VStreamResultsResponse; + public static create(properties?: binlogdata.IMinimalTable): binlogdata.MinimalTable; /** - * Encodes the specified VStreamResultsResponse message. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. - * @param message VStreamResultsResponse message or plain object to encode + * Encodes the specified MinimalTable message. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. + * @param message MinimalTable message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: binlogdata.IVStreamResultsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IMinimalTable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VStreamResultsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. - * @param message VStreamResultsResponse message or plain object to encode + * Encodes the specified MinimalTable message, length delimited. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. + * @param message MinimalTable message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: binlogdata.IVStreamResultsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IMinimalTable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VStreamResultsResponse message from the specified reader or buffer. + * Decodes a MinimalTable message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VStreamResultsResponse + * @returns MinimalTable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResultsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.MinimalTable; /** - * Decodes a VStreamResultsResponse message from the specified reader or buffer, length delimited. + * Decodes a MinimalTable message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VStreamResultsResponse + * @returns MinimalTable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResultsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.MinimalTable; /** - * Verifies a VStreamResultsResponse message. + * Verifies a MinimalTable 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 VStreamResultsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MinimalTable message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VStreamResultsResponse + * @returns MinimalTable */ - public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResultsResponse; + public static fromObject(object: { [k: string]: any }): binlogdata.MinimalTable; /** - * Creates a plain object from a VStreamResultsResponse message. Also converts values to other types if specified. - * @param message VStreamResultsResponse + * Creates a plain object from a MinimalTable message. Also converts values to other types if specified. + * @param message MinimalTable * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: binlogdata.VStreamResultsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.MinimalTable, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VStreamResultsResponse to JSON. + * Converts this MinimalTable to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VStreamResultsResponse + * Gets the default type url for MinimalTable * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} - -/** Namespace vtrpc. */ -export namespace vtrpc { - - /** Properties of a CallerID. */ - interface ICallerID { - - /** CallerID principal */ - principal?: (string|null); - - /** CallerID component */ - component?: (string|null); - /** CallerID subcomponent */ - subcomponent?: (string|null); + /** Properties of a MinimalSchema. */ + interface IMinimalSchema { - /** CallerID groups */ - groups?: (string[]|null); + /** MinimalSchema tables */ + tables?: (binlogdata.IMinimalTable[]|null); } - /** Represents a CallerID. */ - class CallerID implements ICallerID { + /** Represents a MinimalSchema. */ + class MinimalSchema implements IMinimalSchema { /** - * Constructs a new CallerID. + * Constructs a new MinimalSchema. * @param [properties] Properties to set */ - constructor(properties?: vtrpc.ICallerID); - - /** CallerID principal. */ - public principal: string; - - /** CallerID component. */ - public component: string; - - /** CallerID subcomponent. */ - public subcomponent: string; + constructor(properties?: binlogdata.IMinimalSchema); - /** CallerID groups. */ - public groups: string[]; + /** MinimalSchema tables. */ + public tables: binlogdata.IMinimalTable[]; /** - * Creates a new CallerID instance using the specified properties. + * Creates a new MinimalSchema instance using the specified properties. * @param [properties] Properties to set - * @returns CallerID instance + * @returns MinimalSchema instance */ - public static create(properties?: vtrpc.ICallerID): vtrpc.CallerID; + public static create(properties?: binlogdata.IMinimalSchema): binlogdata.MinimalSchema; /** - * Encodes the specified CallerID message. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. - * @param message CallerID message or plain object to encode + * Encodes the specified MinimalSchema message. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. + * @param message MinimalSchema message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtrpc.ICallerID, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IMinimalSchema, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CallerID message, length delimited. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. - * @param message CallerID message or plain object to encode + * Encodes the specified MinimalSchema message, length delimited. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. + * @param message MinimalSchema message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtrpc.ICallerID, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IMinimalSchema, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CallerID message from the specified reader or buffer. + * Decodes a MinimalSchema message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CallerID + * @returns MinimalSchema * @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): vtrpc.CallerID; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.MinimalSchema; /** - * Decodes a CallerID message from the specified reader or buffer, length delimited. + * Decodes a MinimalSchema message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CallerID + * @returns MinimalSchema * @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)): vtrpc.CallerID; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.MinimalSchema; /** - * Verifies a CallerID message. + * Verifies a MinimalSchema 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 CallerID message from a plain object. Also converts values to their respective internal types. + * Creates a MinimalSchema message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CallerID + * @returns MinimalSchema */ - public static fromObject(object: { [k: string]: any }): vtrpc.CallerID; + public static fromObject(object: { [k: string]: any }): binlogdata.MinimalSchema; /** - * Creates a plain object from a CallerID message. Also converts values to other types if specified. - * @param message CallerID + * Creates a plain object from a MinimalSchema message. Also converts values to other types if specified. + * @param message MinimalSchema * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtrpc.CallerID, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.MinimalSchema, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CallerID to JSON. + * Converts this MinimalSchema to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CallerID + * Gets the default type url for MinimalSchema * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Code enum. */ - enum Code { - OK = 0, - CANCELED = 1, - UNKNOWN = 2, - INVALID_ARGUMENT = 3, - DEADLINE_EXCEEDED = 4, - NOT_FOUND = 5, - ALREADY_EXISTS = 6, - PERMISSION_DENIED = 7, - RESOURCE_EXHAUSTED = 8, - FAILED_PRECONDITION = 9, - ABORTED = 10, - OUT_OF_RANGE = 11, - UNIMPLEMENTED = 12, - INTERNAL = 13, - UNAVAILABLE = 14, - DATA_LOSS = 15, - UNAUTHENTICATED = 16, - CLUSTER_EVENT = 17, - READ_ONLY = 18 - } - - /** Properties of a RPCError. */ - interface IRPCError { + /** Properties of a VStreamRequest. */ + interface IVStreamRequest { - /** RPCError message */ - message?: (string|null); + /** VStreamRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); - /** RPCError code */ - code?: (vtrpc.Code|null); + /** VStreamRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamRequest target */ + target?: (query.ITarget|null); + + /** VStreamRequest position */ + position?: (string|null); + + /** VStreamRequest filter */ + filter?: (binlogdata.IFilter|null); + + /** VStreamRequest table_last_p_ks */ + table_last_p_ks?: (binlogdata.ITableLastPK[]|null); } - /** Represents a RPCError. */ - class RPCError implements IRPCError { + /** Represents a VStreamRequest. */ + class VStreamRequest implements IVStreamRequest { /** - * Constructs a new RPCError. + * Constructs a new VStreamRequest. * @param [properties] Properties to set */ - constructor(properties?: vtrpc.IRPCError); + constructor(properties?: binlogdata.IVStreamRequest); - /** RPCError message. */ - public message: string; + /** VStreamRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); - /** RPCError code. */ - public code: vtrpc.Code; + /** VStreamRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamRequest target. */ + public target?: (query.ITarget|null); + + /** VStreamRequest position. */ + public position: string; + + /** VStreamRequest filter. */ + public filter?: (binlogdata.IFilter|null); + + /** VStreamRequest table_last_p_ks. */ + public table_last_p_ks: binlogdata.ITableLastPK[]; /** - * Creates a new RPCError instance using the specified properties. + * Creates a new VStreamRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RPCError instance + * @returns VStreamRequest instance */ - public static create(properties?: vtrpc.IRPCError): vtrpc.RPCError; + public static create(properties?: binlogdata.IVStreamRequest): binlogdata.VStreamRequest; /** - * Encodes the specified RPCError message. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. - * @param message RPCError message or plain object to encode + * Encodes the specified VStreamRequest message. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. + * @param message VStreamRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtrpc.IRPCError, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RPCError message, length delimited. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. - * @param message RPCError message or plain object to encode + * Encodes the specified VStreamRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. + * @param message VStreamRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtrpc.IRPCError, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RPCError message from the specified reader or buffer. + * Decodes a VStreamRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RPCError + * @returns VStreamRequest * @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): vtrpc.RPCError; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRequest; /** - * Decodes a RPCError message from the specified reader or buffer, length delimited. + * Decodes a VStreamRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RPCError + * @returns VStreamRequest * @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)): vtrpc.RPCError; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRequest; /** - * Verifies a RPCError message. + * Verifies a VStreamRequest 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 RPCError message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RPCError + * @returns VStreamRequest */ - public static fromObject(object: { [k: string]: any }): vtrpc.RPCError; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRequest; /** - * Creates a plain object from a RPCError message. Also converts values to other types if specified. - * @param message RPCError + * Creates a plain object from a VStreamRequest message. Also converts values to other types if specified. + * @param message VStreamRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtrpc.RPCError, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RPCError to JSON. + * Converts this VStreamRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RPCError + * Gets the default type url for VStreamRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} - -/** Namespace query. */ -export namespace query { - - /** Properties of a Target. */ - interface ITarget { - - /** Target keyspace */ - keyspace?: (string|null); - - /** Target shard */ - shard?: (string|null); - /** Target tablet_type */ - tablet_type?: (topodata.TabletType|null); + /** Properties of a VStreamResponse. */ + interface IVStreamResponse { - /** Target cell */ - cell?: (string|null); + /** VStreamResponse events */ + events?: (binlogdata.IVEvent[]|null); } - /** Represents a Target. */ - class Target implements ITarget { + /** Represents a VStreamResponse. */ + class VStreamResponse implements IVStreamResponse { /** - * Constructs a new Target. + * Constructs a new VStreamResponse. * @param [properties] Properties to set */ - constructor(properties?: query.ITarget); - - /** Target keyspace. */ - public keyspace: string; - - /** Target shard. */ - public shard: string; - - /** Target tablet_type. */ - public tablet_type: topodata.TabletType; + constructor(properties?: binlogdata.IVStreamResponse); - /** Target cell. */ - public cell: string; + /** VStreamResponse events. */ + public events: binlogdata.IVEvent[]; /** - * Creates a new Target instance using the specified properties. + * Creates a new VStreamResponse instance using the specified properties. * @param [properties] Properties to set - * @returns Target instance + * @returns VStreamResponse instance */ - public static create(properties?: query.ITarget): query.Target; + public static create(properties?: binlogdata.IVStreamResponse): binlogdata.VStreamResponse; /** - * Encodes the specified Target message. Does not implicitly {@link query.Target.verify|verify} messages. - * @param message Target message or plain object to encode + * Encodes the specified VStreamResponse message. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. + * @param message VStreamResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ITarget, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Target message, length delimited. Does not implicitly {@link query.Target.verify|verify} messages. - * @param message Target message or plain object to encode + * Encodes the specified VStreamResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. + * @param message VStreamResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ITarget, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Target message from the specified reader or buffer. + * Decodes a VStreamResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Target + * @returns VStreamResponse * @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): query.Target; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResponse; /** - * Decodes a Target message from the specified reader or buffer, length delimited. + * Decodes a VStreamResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Target + * @returns VStreamResponse * @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)): query.Target; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResponse; /** - * Verifies a Target message. + * Verifies a VStreamResponse 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 Target message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Target + * @returns VStreamResponse */ - public static fromObject(object: { [k: string]: any }): query.Target; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResponse; /** - * Creates a plain object from a Target message. Also converts values to other types if specified. - * @param message Target + * Creates a plain object from a VStreamResponse message. Also converts values to other types if specified. + * @param message VStreamResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.Target, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Target to JSON. + * Converts this VStreamResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Target + * Gets the default type url for VStreamResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a VTGateCallerID. */ - interface IVTGateCallerID { + /** Properties of a VStreamRowsRequest. */ + interface IVStreamRowsRequest { - /** VTGateCallerID username */ - username?: (string|null); + /** VStreamRowsRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); - /** VTGateCallerID groups */ - groups?: (string[]|null); + /** VStreamRowsRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamRowsRequest target */ + target?: (query.ITarget|null); + + /** VStreamRowsRequest query */ + query?: (string|null); + + /** VStreamRowsRequest lastpk */ + lastpk?: (query.IQueryResult|null); } - /** Represents a VTGateCallerID. */ - class VTGateCallerID implements IVTGateCallerID { + /** Represents a VStreamRowsRequest. */ + class VStreamRowsRequest implements IVStreamRowsRequest { /** - * Constructs a new VTGateCallerID. + * Constructs a new VStreamRowsRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IVTGateCallerID); + constructor(properties?: binlogdata.IVStreamRowsRequest); - /** VTGateCallerID username. */ - public username: string; + /** VStreamRowsRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); - /** VTGateCallerID groups. */ - public groups: string[]; + /** VStreamRowsRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamRowsRequest target. */ + public target?: (query.ITarget|null); + + /** VStreamRowsRequest query. */ + public query: string; + + /** VStreamRowsRequest lastpk. */ + public lastpk?: (query.IQueryResult|null); /** - * Creates a new VTGateCallerID instance using the specified properties. + * Creates a new VStreamRowsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns VTGateCallerID instance + * @returns VStreamRowsRequest instance */ - public static create(properties?: query.IVTGateCallerID): query.VTGateCallerID; + public static create(properties?: binlogdata.IVStreamRowsRequest): binlogdata.VStreamRowsRequest; /** - * Encodes the specified VTGateCallerID message. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. - * @param message VTGateCallerID message or plain object to encode + * Encodes the specified VStreamRowsRequest message. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. + * @param message VStreamRowsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IVTGateCallerID, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamRowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified VTGateCallerID message, length delimited. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. - * @param message VTGateCallerID message or plain object to encode + * Encodes the specified VStreamRowsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. + * @param message VStreamRowsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IVTGateCallerID, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamRowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a VTGateCallerID message from the specified reader or buffer. + * Decodes a VStreamRowsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns VTGateCallerID + * @returns VStreamRowsRequest * @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): query.VTGateCallerID; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRowsRequest; /** - * Decodes a VTGateCallerID message from the specified reader or buffer, length delimited. + * Decodes a VStreamRowsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns VTGateCallerID + * @returns VStreamRowsRequest * @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)): query.VTGateCallerID; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRowsRequest; /** - * Verifies a VTGateCallerID message. + * Verifies a VStreamRowsRequest 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 VTGateCallerID message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRowsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns VTGateCallerID + * @returns VStreamRowsRequest */ - public static fromObject(object: { [k: string]: any }): query.VTGateCallerID; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRowsRequest; /** - * Creates a plain object from a VTGateCallerID message. Also converts values to other types if specified. - * @param message VTGateCallerID + * Creates a plain object from a VStreamRowsRequest message. Also converts values to other types if specified. + * @param message VStreamRowsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.VTGateCallerID, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamRowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this VTGateCallerID to JSON. + * Converts this VStreamRowsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for VTGateCallerID + * Gets the default type url for VStreamRowsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an EventToken. */ - interface IEventToken { + /** Properties of a VStreamRowsResponse. */ + interface IVStreamRowsResponse { - /** EventToken timestamp */ - timestamp?: (number|Long|null); + /** VStreamRowsResponse fields */ + fields?: (query.IField[]|null); - /** EventToken shard */ - shard?: (string|null); + /** VStreamRowsResponse pkfields */ + pkfields?: (query.IField[]|null); - /** EventToken position */ - position?: (string|null); + /** VStreamRowsResponse gtid */ + gtid?: (string|null); + + /** VStreamRowsResponse rows */ + rows?: (query.IRow[]|null); + + /** VStreamRowsResponse lastpk */ + lastpk?: (query.IRow|null); + + /** VStreamRowsResponse throttled */ + throttled?: (boolean|null); + + /** VStreamRowsResponse heartbeat */ + heartbeat?: (boolean|null); } - /** Represents an EventToken. */ - class EventToken implements IEventToken { + /** Represents a VStreamRowsResponse. */ + class VStreamRowsResponse implements IVStreamRowsResponse { /** - * Constructs a new EventToken. + * Constructs a new VStreamRowsResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IEventToken); + constructor(properties?: binlogdata.IVStreamRowsResponse); - /** EventToken timestamp. */ - public timestamp: (number|Long); + /** VStreamRowsResponse fields. */ + public fields: query.IField[]; - /** EventToken shard. */ - public shard: string; + /** VStreamRowsResponse pkfields. */ + public pkfields: query.IField[]; - /** EventToken position. */ - public position: string; + /** VStreamRowsResponse gtid. */ + public gtid: string; + + /** VStreamRowsResponse rows. */ + public rows: query.IRow[]; + + /** VStreamRowsResponse lastpk. */ + public lastpk?: (query.IRow|null); + + /** VStreamRowsResponse throttled. */ + public throttled: boolean; + + /** VStreamRowsResponse heartbeat. */ + public heartbeat: boolean; /** - * Creates a new EventToken instance using the specified properties. + * Creates a new VStreamRowsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns EventToken instance + * @returns VStreamRowsResponse instance */ - public static create(properties?: query.IEventToken): query.EventToken; + public static create(properties?: binlogdata.IVStreamRowsResponse): binlogdata.VStreamRowsResponse; /** - * Encodes the specified EventToken message. Does not implicitly {@link query.EventToken.verify|verify} messages. - * @param message EventToken message or plain object to encode + * Encodes the specified VStreamRowsResponse message. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. + * @param message VStreamRowsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IEventToken, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamRowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified EventToken message, length delimited. Does not implicitly {@link query.EventToken.verify|verify} messages. - * @param message EventToken message or plain object to encode + * Encodes the specified VStreamRowsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. + * @param message VStreamRowsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IEventToken, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamRowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an EventToken message from the specified reader or buffer. + * Decodes a VStreamRowsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns EventToken + * @returns VStreamRowsResponse * @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): query.EventToken; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamRowsResponse; /** - * Decodes an EventToken message from the specified reader or buffer, length delimited. + * Decodes a VStreamRowsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns EventToken + * @returns VStreamRowsResponse * @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)): query.EventToken; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamRowsResponse; /** - * Verifies an EventToken message. + * Verifies a VStreamRowsResponse 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 EventToken message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRowsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns EventToken + * @returns VStreamRowsResponse */ - public static fromObject(object: { [k: string]: any }): query.EventToken; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamRowsResponse; /** - * Creates a plain object from an EventToken message. Also converts values to other types if specified. - * @param message EventToken + * Creates a plain object from a VStreamRowsResponse message. Also converts values to other types if specified. + * @param message VStreamRowsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.EventToken, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamRowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this EventToken to JSON. + * Converts this VStreamRowsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for EventToken + * Gets the default type url for VStreamRowsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** MySqlFlag enum. */ - enum MySqlFlag { - EMPTY = 0, - NOT_NULL_FLAG = 1, - PRI_KEY_FLAG = 2, - UNIQUE_KEY_FLAG = 4, - MULTIPLE_KEY_FLAG = 8, - BLOB_FLAG = 16, - UNSIGNED_FLAG = 32, - ZEROFILL_FLAG = 64, - BINARY_FLAG = 128, - ENUM_FLAG = 256, - AUTO_INCREMENT_FLAG = 512, - TIMESTAMP_FLAG = 1024, - SET_FLAG = 2048, - NO_DEFAULT_VALUE_FLAG = 4096, - ON_UPDATE_NOW_FLAG = 8192, - NUM_FLAG = 32768, - PART_KEY_FLAG = 16384, - GROUP_FLAG = 32768, - UNIQUE_FLAG = 65536, - BINCMP_FLAG = 131072 - } - - /** Flag enum. */ - enum Flag { - NONE = 0, - ISINTEGRAL = 256, - ISUNSIGNED = 512, - ISFLOAT = 1024, - ISQUOTED = 2048, - ISTEXT = 4096, - ISBINARY = 8192 - } - - /** Type enum. */ - enum Type { - NULL_TYPE = 0, - INT8 = 257, - UINT8 = 770, - INT16 = 259, - UINT16 = 772, - INT24 = 261, - UINT24 = 774, - INT32 = 263, - UINT32 = 776, - INT64 = 265, - UINT64 = 778, - FLOAT32 = 1035, - FLOAT64 = 1036, - TIMESTAMP = 2061, - DATE = 2062, - TIME = 2063, - DATETIME = 2064, - YEAR = 785, - DECIMAL = 18, - TEXT = 6163, - BLOB = 10260, - VARCHAR = 6165, - VARBINARY = 10262, - CHAR = 6167, - BINARY = 10264, - BIT = 2073, - ENUM = 2074, - SET = 2075, - TUPLE = 28, - GEOMETRY = 2077, - JSON = 2078, - EXPRESSION = 31, - HEXNUM = 4128, - HEXVAL = 4129, - BITNUM = 4130 - } - - /** Properties of a Value. */ - interface IValue { + /** Properties of a LastPKEvent. */ + interface ILastPKEvent { - /** Value type */ - type?: (query.Type|null); + /** LastPKEvent table_last_p_k */ + table_last_p_k?: (binlogdata.ITableLastPK|null); - /** Value value */ - value?: (Uint8Array|null); + /** LastPKEvent completed */ + completed?: (boolean|null); } - /** Represents a Value. */ - class Value implements IValue { + /** Represents a LastPKEvent. */ + class LastPKEvent implements ILastPKEvent { /** - * Constructs a new Value. + * Constructs a new LastPKEvent. * @param [properties] Properties to set */ - constructor(properties?: query.IValue); + constructor(properties?: binlogdata.ILastPKEvent); - /** Value type. */ - public type: query.Type; + /** LastPKEvent table_last_p_k. */ + public table_last_p_k?: (binlogdata.ITableLastPK|null); - /** Value value. */ - public value: Uint8Array; + /** LastPKEvent completed. */ + public completed: boolean; /** - * Creates a new Value instance using the specified properties. + * Creates a new LastPKEvent instance using the specified properties. * @param [properties] Properties to set - * @returns Value instance + * @returns LastPKEvent instance */ - public static create(properties?: query.IValue): query.Value; + public static create(properties?: binlogdata.ILastPKEvent): binlogdata.LastPKEvent; /** - * Encodes the specified Value message. Does not implicitly {@link query.Value.verify|verify} messages. - * @param message Value message or plain object to encode + * Encodes the specified LastPKEvent message. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. + * @param message LastPKEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IValue, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.ILastPKEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Value message, length delimited. Does not implicitly {@link query.Value.verify|verify} messages. - * @param message Value message or plain object to encode + * Encodes the specified LastPKEvent message, length delimited. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. + * @param message LastPKEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IValue, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.ILastPKEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Value message from the specified reader or buffer. + * Decodes a LastPKEvent message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Value + * @returns LastPKEvent * @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): query.Value; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.LastPKEvent; /** - * Decodes a Value message from the specified reader or buffer, length delimited. + * Decodes a LastPKEvent message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Value + * @returns LastPKEvent * @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)): query.Value; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.LastPKEvent; /** - * Verifies a Value message. + * Verifies a LastPKEvent 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 Value message from a plain object. Also converts values to their respective internal types. + * Creates a LastPKEvent message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Value + * @returns LastPKEvent */ - public static fromObject(object: { [k: string]: any }): query.Value; + public static fromObject(object: { [k: string]: any }): binlogdata.LastPKEvent; /** - * Creates a plain object from a Value message. Also converts values to other types if specified. - * @param message Value + * Creates a plain object from a LastPKEvent message. Also converts values to other types if specified. + * @param message LastPKEvent * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.Value, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.LastPKEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Value to JSON. + * Converts this LastPKEvent to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Value + * Gets the default type url for LastPKEvent * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BindVariable. */ - interface IBindVariable { - - /** BindVariable type */ - type?: (query.Type|null); + /** Properties of a TableLastPK. */ + interface ITableLastPK { - /** BindVariable value */ - value?: (Uint8Array|null); + /** TableLastPK table_name */ + table_name?: (string|null); - /** BindVariable values */ - values?: (query.IValue[]|null); + /** TableLastPK lastpk */ + lastpk?: (query.IQueryResult|null); } - /** Represents a BindVariable. */ - class BindVariable implements IBindVariable { + /** Represents a TableLastPK. */ + class TableLastPK implements ITableLastPK { /** - * Constructs a new BindVariable. + * Constructs a new TableLastPK. * @param [properties] Properties to set */ - constructor(properties?: query.IBindVariable); - - /** BindVariable type. */ - public type: query.Type; + constructor(properties?: binlogdata.ITableLastPK); - /** BindVariable value. */ - public value: Uint8Array; + /** TableLastPK table_name. */ + public table_name: string; - /** BindVariable values. */ - public values: query.IValue[]; + /** TableLastPK lastpk. */ + public lastpk?: (query.IQueryResult|null); /** - * Creates a new BindVariable instance using the specified properties. + * Creates a new TableLastPK instance using the specified properties. * @param [properties] Properties to set - * @returns BindVariable instance + * @returns TableLastPK instance */ - public static create(properties?: query.IBindVariable): query.BindVariable; + public static create(properties?: binlogdata.ITableLastPK): binlogdata.TableLastPK; /** - * Encodes the specified BindVariable message. Does not implicitly {@link query.BindVariable.verify|verify} messages. - * @param message BindVariable message or plain object to encode + * Encodes the specified TableLastPK message. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. + * @param message TableLastPK message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBindVariable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.ITableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BindVariable message, length delimited. Does not implicitly {@link query.BindVariable.verify|verify} messages. - * @param message BindVariable message or plain object to encode + * Encodes the specified TableLastPK message, length delimited. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. + * @param message TableLastPK message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBindVariable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.ITableLastPK, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BindVariable message from the specified reader or buffer. + * Decodes a TableLastPK message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BindVariable + * @returns TableLastPK * @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): query.BindVariable; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.TableLastPK; /** - * Decodes a BindVariable message from the specified reader or buffer, length delimited. + * Decodes a TableLastPK message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BindVariable + * @returns TableLastPK * @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)): query.BindVariable; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.TableLastPK; /** - * Verifies a BindVariable message. + * Verifies a TableLastPK 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 BindVariable message from a plain object. Also converts values to their respective internal types. + * Creates a TableLastPK message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BindVariable + * @returns TableLastPK */ - public static fromObject(object: { [k: string]: any }): query.BindVariable; + public static fromObject(object: { [k: string]: any }): binlogdata.TableLastPK; /** - * Creates a plain object from a BindVariable message. Also converts values to other types if specified. - * @param message BindVariable + * Creates a plain object from a TableLastPK message. Also converts values to other types if specified. + * @param message TableLastPK * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BindVariable, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.TableLastPK, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BindVariable to JSON. + * Converts this TableLastPK to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BindVariable + * Gets the default type url for TableLastPK * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BoundQuery. */ - interface IBoundQuery { + /** Properties of a VStreamResultsRequest. */ + interface IVStreamResultsRequest { - /** BoundQuery sql */ - sql?: (string|null); + /** VStreamResultsRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); - /** BoundQuery bind_variables */ - bind_variables?: ({ [k: string]: query.IBindVariable }|null); + /** VStreamResultsRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamResultsRequest target */ + target?: (query.ITarget|null); + + /** VStreamResultsRequest query */ + query?: (string|null); } - /** Represents a BoundQuery. */ - class BoundQuery implements IBoundQuery { + /** Represents a VStreamResultsRequest. */ + class VStreamResultsRequest implements IVStreamResultsRequest { /** - * Constructs a new BoundQuery. + * Constructs a new VStreamResultsRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IBoundQuery); + constructor(properties?: binlogdata.IVStreamResultsRequest); - /** BoundQuery sql. */ - public sql: string; + /** VStreamResultsRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); - /** BoundQuery bind_variables. */ - public bind_variables: { [k: string]: query.IBindVariable }; + /** VStreamResultsRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamResultsRequest target. */ + public target?: (query.ITarget|null); + + /** VStreamResultsRequest query. */ + public query: string; /** - * Creates a new BoundQuery instance using the specified properties. + * Creates a new VStreamResultsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns BoundQuery instance + * @returns VStreamResultsRequest instance */ - public static create(properties?: query.IBoundQuery): query.BoundQuery; + public static create(properties?: binlogdata.IVStreamResultsRequest): binlogdata.VStreamResultsRequest; /** - * Encodes the specified BoundQuery message. Does not implicitly {@link query.BoundQuery.verify|verify} messages. - * @param message BoundQuery message or plain object to encode + * Encodes the specified VStreamResultsRequest message. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. + * @param message VStreamResultsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBoundQuery, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamResultsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BoundQuery message, length delimited. Does not implicitly {@link query.BoundQuery.verify|verify} messages. - * @param message BoundQuery message or plain object to encode + * Encodes the specified VStreamResultsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. + * @param message VStreamResultsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBoundQuery, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamResultsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BoundQuery message from the specified reader or buffer. + * Decodes a VStreamResultsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BoundQuery + * @returns VStreamResultsRequest * @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): query.BoundQuery; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResultsRequest; /** - * Decodes a BoundQuery message from the specified reader or buffer, length delimited. + * Decodes a VStreamResultsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BoundQuery + * @returns VStreamResultsRequest * @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)): query.BoundQuery; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResultsRequest; /** - * Verifies a BoundQuery message. + * Verifies a VStreamResultsRequest 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 BoundQuery message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResultsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BoundQuery + * @returns VStreamResultsRequest */ - public static fromObject(object: { [k: string]: any }): query.BoundQuery; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResultsRequest; /** - * Creates a plain object from a BoundQuery message. Also converts values to other types if specified. - * @param message BoundQuery + * Creates a plain object from a VStreamResultsRequest message. Also converts values to other types if specified. + * @param message VStreamResultsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BoundQuery, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamResultsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BoundQuery to JSON. + * Converts this VStreamResultsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BoundQuery + * Gets the default type url for VStreamResultsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteOptions. */ - interface IExecuteOptions { - - /** ExecuteOptions included_fields */ - included_fields?: (query.ExecuteOptions.IncludedFields|null); - - /** ExecuteOptions client_found_rows */ - client_found_rows?: (boolean|null); - - /** ExecuteOptions workload */ - workload?: (query.ExecuteOptions.Workload|null); - - /** ExecuteOptions sql_select_limit */ - sql_select_limit?: (number|Long|null); - - /** ExecuteOptions transaction_isolation */ - transaction_isolation?: (query.ExecuteOptions.TransactionIsolation|null); - - /** ExecuteOptions skip_query_plan_cache */ - skip_query_plan_cache?: (boolean|null); - - /** ExecuteOptions planner_version */ - planner_version?: (query.ExecuteOptions.PlannerVersion|null); - - /** ExecuteOptions has_created_temp_tables */ - has_created_temp_tables?: (boolean|null); - - /** ExecuteOptions consolidator */ - consolidator?: (query.ExecuteOptions.Consolidator|null); + /** Properties of a VStreamResultsResponse. */ + interface IVStreamResultsResponse { - /** ExecuteOptions transaction_access_mode */ - transaction_access_mode?: (query.ExecuteOptions.TransactionAccessMode[]|null); + /** VStreamResultsResponse fields */ + fields?: (query.IField[]|null); - /** ExecuteOptions WorkloadName */ - WorkloadName?: (string|null); + /** VStreamResultsResponse gtid */ + gtid?: (string|null); - /** ExecuteOptions priority */ - priority?: (string|null); + /** VStreamResultsResponse rows */ + rows?: (query.IRow[]|null); } - /** Represents an ExecuteOptions. */ - class ExecuteOptions implements IExecuteOptions { + /** Represents a VStreamResultsResponse. */ + class VStreamResultsResponse implements IVStreamResultsResponse { /** - * Constructs a new ExecuteOptions. + * Constructs a new VStreamResultsResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IExecuteOptions); - - /** ExecuteOptions included_fields. */ - public included_fields: query.ExecuteOptions.IncludedFields; - - /** ExecuteOptions client_found_rows. */ - public client_found_rows: boolean; - - /** ExecuteOptions workload. */ - public workload: query.ExecuteOptions.Workload; - - /** ExecuteOptions sql_select_limit. */ - public sql_select_limit: (number|Long); - - /** ExecuteOptions transaction_isolation. */ - public transaction_isolation: query.ExecuteOptions.TransactionIsolation; - - /** ExecuteOptions skip_query_plan_cache. */ - public skip_query_plan_cache: boolean; - - /** ExecuteOptions planner_version. */ - public planner_version: query.ExecuteOptions.PlannerVersion; - - /** ExecuteOptions has_created_temp_tables. */ - public has_created_temp_tables: boolean; - - /** ExecuteOptions consolidator. */ - public consolidator: query.ExecuteOptions.Consolidator; + constructor(properties?: binlogdata.IVStreamResultsResponse); - /** ExecuteOptions transaction_access_mode. */ - public transaction_access_mode: query.ExecuteOptions.TransactionAccessMode[]; + /** VStreamResultsResponse fields. */ + public fields: query.IField[]; - /** ExecuteOptions WorkloadName. */ - public WorkloadName: string; + /** VStreamResultsResponse gtid. */ + public gtid: string; - /** ExecuteOptions priority. */ - public priority: string; + /** VStreamResultsResponse rows. */ + public rows: query.IRow[]; /** - * Creates a new ExecuteOptions instance using the specified properties. + * Creates a new VStreamResultsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteOptions instance + * @returns VStreamResultsResponse instance */ - public static create(properties?: query.IExecuteOptions): query.ExecuteOptions; + public static create(properties?: binlogdata.IVStreamResultsResponse): binlogdata.VStreamResultsResponse; /** - * Encodes the specified ExecuteOptions message. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. - * @param message ExecuteOptions message or plain object to encode + * Encodes the specified VStreamResultsResponse message. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. + * @param message VStreamResultsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IExecuteOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: binlogdata.IVStreamResultsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteOptions message, length delimited. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. - * @param message ExecuteOptions message or plain object to encode + * Encodes the specified VStreamResultsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. + * @param message VStreamResultsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IExecuteOptions, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: binlogdata.IVStreamResultsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteOptions message from the specified reader or buffer. + * Decodes a VStreamResultsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteOptions + * @returns VStreamResultsResponse * @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): query.ExecuteOptions; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamResultsResponse; /** - * Decodes an ExecuteOptions message from the specified reader or buffer, length delimited. + * Decodes a VStreamResultsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteOptions + * @returns VStreamResultsResponse * @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)): query.ExecuteOptions; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamResultsResponse; /** - * Verifies an ExecuteOptions message. + * Verifies a VStreamResultsResponse 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 ExecuteOptions message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResultsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteOptions + * @returns VStreamResultsResponse */ - public static fromObject(object: { [k: string]: any }): query.ExecuteOptions; + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamResultsResponse; /** - * Creates a plain object from an ExecuteOptions message. Also converts values to other types if specified. - * @param message ExecuteOptions + * Creates a plain object from a VStreamResultsResponse message. Also converts values to other types if specified. + * @param message VStreamResultsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ExecuteOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: binlogdata.VStreamResultsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteOptions to JSON. + * Converts this VStreamResultsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteOptions + * Gets the default type url for VStreamResultsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } +} - namespace ExecuteOptions { - - /** IncludedFields enum. */ - enum IncludedFields { - TYPE_AND_NAME = 0, - TYPE_ONLY = 1, - ALL = 2 - } - - /** Workload enum. */ - enum Workload { - UNSPECIFIED = 0, - OLTP = 1, - OLAP = 2, - DBA = 3 - } - - /** TransactionIsolation enum. */ - enum TransactionIsolation { - DEFAULT = 0, - REPEATABLE_READ = 1, - READ_COMMITTED = 2, - READ_UNCOMMITTED = 3, - SERIALIZABLE = 4, - CONSISTENT_SNAPSHOT_READ_ONLY = 5, - AUTOCOMMIT = 6 - } - - /** PlannerVersion enum. */ - enum PlannerVersion { - DEFAULT_PLANNER = 0, - V3 = 1, - Gen4 = 2, - Gen4Greedy = 3, - Gen4Left2Right = 4, - Gen4WithFallback = 5, - Gen4CompareV3 = 6, - V3Insert = 7 - } - - /** Consolidator enum. */ - enum Consolidator { - CONSOLIDATOR_UNSPECIFIED = 0, - CONSOLIDATOR_DISABLED = 1, - CONSOLIDATOR_ENABLED = 2, - CONSOLIDATOR_ENABLED_REPLICAS = 3 - } - - /** TransactionAccessMode enum. */ - enum TransactionAccessMode { - CONSISTENT_SNAPSHOT = 0, - READ_WRITE = 1, - READ_ONLY = 2 - } - } - - /** Properties of a Field. */ - interface IField { - - /** Field name */ - name?: (string|null); - - /** Field type */ - type?: (query.Type|null); - - /** Field table */ - table?: (string|null); - - /** Field org_table */ - org_table?: (string|null); - - /** Field database */ - database?: (string|null); - - /** Field org_name */ - org_name?: (string|null); +/** Namespace vtrpc. */ +export namespace vtrpc { - /** Field column_length */ - column_length?: (number|null); + /** Properties of a CallerID. */ + interface ICallerID { - /** Field charset */ - charset?: (number|null); + /** CallerID principal */ + principal?: (string|null); - /** Field decimals */ - decimals?: (number|null); + /** CallerID component */ + component?: (string|null); - /** Field flags */ - flags?: (number|null); + /** CallerID subcomponent */ + subcomponent?: (string|null); - /** Field column_type */ - column_type?: (string|null); + /** CallerID groups */ + groups?: (string[]|null); } - /** Represents a Field. */ - class Field implements IField { + /** Represents a CallerID. */ + class CallerID implements ICallerID { /** - * Constructs a new Field. + * Constructs a new CallerID. * @param [properties] Properties to set */ - constructor(properties?: query.IField); - - /** Field name. */ - public name: string; - - /** Field type. */ - public type: query.Type; - - /** Field table. */ - public table: string; - - /** Field org_table. */ - public org_table: string; - - /** Field database. */ - public database: string; - - /** Field org_name. */ - public org_name: string; - - /** Field column_length. */ - public column_length: number; + constructor(properties?: vtrpc.ICallerID); - /** Field charset. */ - public charset: number; + /** CallerID principal. */ + public principal: string; - /** Field decimals. */ - public decimals: number; + /** CallerID component. */ + public component: string; - /** Field flags. */ - public flags: number; + /** CallerID subcomponent. */ + public subcomponent: string; - /** Field column_type. */ - public column_type: string; + /** CallerID groups. */ + public groups: string[]; /** - * Creates a new Field instance using the specified properties. + * Creates a new CallerID instance using the specified properties. * @param [properties] Properties to set - * @returns Field instance + * @returns CallerID instance */ - public static create(properties?: query.IField): query.Field; + public static create(properties?: vtrpc.ICallerID): vtrpc.CallerID; /** - * Encodes the specified Field message. Does not implicitly {@link query.Field.verify|verify} messages. - * @param message Field message or plain object to encode + * Encodes the specified CallerID message. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. + * @param message CallerID message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IField, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtrpc.ICallerID, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Field message, length delimited. Does not implicitly {@link query.Field.verify|verify} messages. - * @param message Field message or plain object to encode + * Encodes the specified CallerID message, length delimited. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. + * @param message CallerID message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IField, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtrpc.ICallerID, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Field message from the specified reader or buffer. + * Decodes a CallerID message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Field + * @returns CallerID * @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): query.Field; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtrpc.CallerID; /** - * Decodes a Field message from the specified reader or buffer, length delimited. + * Decodes a CallerID message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Field + * @returns CallerID * @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)): query.Field; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtrpc.CallerID; /** - * Verifies a Field message. + * Verifies a CallerID 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 Field message from a plain object. Also converts values to their respective internal types. + * Creates a CallerID message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Field + * @returns CallerID */ - public static fromObject(object: { [k: string]: any }): query.Field; + public static fromObject(object: { [k: string]: any }): vtrpc.CallerID; /** - * Creates a plain object from a Field message. Also converts values to other types if specified. - * @param message Field + * Creates a plain object from a CallerID message. Also converts values to other types if specified. + * @param message CallerID * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.Field, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtrpc.CallerID, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Field to JSON. + * Converts this CallerID to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Field + * Gets the default type url for CallerID * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Row. */ - interface IRow { + /** Code enum. */ + enum Code { + OK = 0, + CANCELED = 1, + UNKNOWN = 2, + INVALID_ARGUMENT = 3, + DEADLINE_EXCEEDED = 4, + NOT_FOUND = 5, + ALREADY_EXISTS = 6, + PERMISSION_DENIED = 7, + RESOURCE_EXHAUSTED = 8, + FAILED_PRECONDITION = 9, + ABORTED = 10, + OUT_OF_RANGE = 11, + UNIMPLEMENTED = 12, + INTERNAL = 13, + UNAVAILABLE = 14, + DATA_LOSS = 15, + UNAUTHENTICATED = 16, + CLUSTER_EVENT = 17, + READ_ONLY = 18 + } - /** Row lengths */ - lengths?: ((number|Long)[]|null); + /** Properties of a RPCError. */ + interface IRPCError { - /** Row values */ - values?: (Uint8Array|null); + /** RPCError message */ + message?: (string|null); + + /** RPCError code */ + code?: (vtrpc.Code|null); } - /** Represents a Row. */ - class Row implements IRow { + /** Represents a RPCError. */ + class RPCError implements IRPCError { /** - * Constructs a new Row. + * Constructs a new RPCError. * @param [properties] Properties to set */ - constructor(properties?: query.IRow); + constructor(properties?: vtrpc.IRPCError); - /** Row lengths. */ - public lengths: (number|Long)[]; + /** RPCError message. */ + public message: string; - /** Row values. */ - public values: Uint8Array; + /** RPCError code. */ + public code: vtrpc.Code; /** - * Creates a new Row instance using the specified properties. + * Creates a new RPCError instance using the specified properties. * @param [properties] Properties to set - * @returns Row instance + * @returns RPCError instance */ - public static create(properties?: query.IRow): query.Row; + public static create(properties?: vtrpc.IRPCError): vtrpc.RPCError; /** - * Encodes the specified Row message. Does not implicitly {@link query.Row.verify|verify} messages. - * @param message Row message or plain object to encode + * Encodes the specified RPCError message. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. + * @param message RPCError message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRow, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtrpc.IRPCError, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Row message, length delimited. Does not implicitly {@link query.Row.verify|verify} messages. - * @param message Row message or plain object to encode + * Encodes the specified RPCError message, length delimited. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. + * @param message RPCError message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRow, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtrpc.IRPCError, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Row message from the specified reader or buffer. + * Decodes a RPCError message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Row + * @returns RPCError * @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): query.Row; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtrpc.RPCError; /** - * Decodes a Row message from the specified reader or buffer, length delimited. + * Decodes a RPCError message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Row + * @returns RPCError * @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)): query.Row; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtrpc.RPCError; /** - * Verifies a Row message. + * Verifies a RPCError 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 Row message from a plain object. Also converts values to their respective internal types. + * Creates a RPCError message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Row + * @returns RPCError */ - public static fromObject(object: { [k: string]: any }): query.Row; + public static fromObject(object: { [k: string]: any }): vtrpc.RPCError; /** - * Creates a plain object from a Row message. Also converts values to other types if specified. - * @param message Row + * Creates a plain object from a RPCError message. Also converts values to other types if specified. + * @param message RPCError * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.Row, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtrpc.RPCError, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Row to JSON. + * Converts this RPCError to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Row + * Gets the default type url for RPCError * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } +} - /** Properties of a QueryResult. */ - interface IQueryResult { - - /** QueryResult fields */ - fields?: (query.IField[]|null); +/** Namespace query. */ +export namespace query { - /** QueryResult rows_affected */ - rows_affected?: (number|Long|null); + /** Properties of a Target. */ + interface ITarget { - /** QueryResult insert_id */ - insert_id?: (number|Long|null); + /** Target keyspace */ + keyspace?: (string|null); - /** QueryResult rows */ - rows?: (query.IRow[]|null); + /** Target shard */ + shard?: (string|null); - /** QueryResult info */ - info?: (string|null); + /** Target tablet_type */ + tablet_type?: (topodata.TabletType|null); - /** QueryResult session_state_changes */ - session_state_changes?: (string|null); + /** Target cell */ + cell?: (string|null); } - /** Represents a QueryResult. */ - class QueryResult implements IQueryResult { + /** Represents a Target. */ + class Target implements ITarget { /** - * Constructs a new QueryResult. + * Constructs a new Target. * @param [properties] Properties to set */ - constructor(properties?: query.IQueryResult); - - /** QueryResult fields. */ - public fields: query.IField[]; - - /** QueryResult rows_affected. */ - public rows_affected: (number|Long); + constructor(properties?: query.ITarget); - /** QueryResult insert_id. */ - public insert_id: (number|Long); + /** Target keyspace. */ + public keyspace: string; - /** QueryResult rows. */ - public rows: query.IRow[]; + /** Target shard. */ + public shard: string; - /** QueryResult info. */ - public info: string; + /** Target tablet_type. */ + public tablet_type: topodata.TabletType; - /** QueryResult session_state_changes. */ - public session_state_changes: string; + /** Target cell. */ + public cell: string; /** - * Creates a new QueryResult instance using the specified properties. + * Creates a new Target instance using the specified properties. * @param [properties] Properties to set - * @returns QueryResult instance + * @returns Target instance */ - public static create(properties?: query.IQueryResult): query.QueryResult; + public static create(properties?: query.ITarget): query.Target; /** - * Encodes the specified QueryResult message. Does not implicitly {@link query.QueryResult.verify|verify} messages. - * @param message QueryResult message or plain object to encode + * Encodes the specified Target message. Does not implicitly {@link query.Target.verify|verify} messages. + * @param message Target message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IQueryResult, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ITarget, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified QueryResult message, length delimited. Does not implicitly {@link query.QueryResult.verify|verify} messages. - * @param message QueryResult message or plain object to encode + * Encodes the specified Target message, length delimited. Does not implicitly {@link query.Target.verify|verify} messages. + * @param message Target message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IQueryResult, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ITarget, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a QueryResult message from the specified reader or buffer. + * Decodes a Target message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns QueryResult + * @returns Target * @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): query.QueryResult; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.Target; /** - * Decodes a QueryResult message from the specified reader or buffer, length delimited. + * Decodes a Target message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns QueryResult + * @returns Target * @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)): query.QueryResult; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.Target; /** - * Verifies a QueryResult message. + * Verifies a Target 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 QueryResult message from a plain object. Also converts values to their respective internal types. + * Creates a Target message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns QueryResult + * @returns Target */ - public static fromObject(object: { [k: string]: any }): query.QueryResult; + public static fromObject(object: { [k: string]: any }): query.Target; /** - * Creates a plain object from a QueryResult message. Also converts values to other types if specified. - * @param message QueryResult + * Creates a plain object from a Target message. Also converts values to other types if specified. + * @param message Target * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.QueryResult, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.Target, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this QueryResult to JSON. + * Converts this Target to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for QueryResult + * Gets the default type url for Target * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a QueryWarning. */ - interface IQueryWarning { + /** Properties of a VTGateCallerID. */ + interface IVTGateCallerID { - /** QueryWarning code */ - code?: (number|null); + /** VTGateCallerID username */ + username?: (string|null); - /** QueryWarning message */ - message?: (string|null); + /** VTGateCallerID groups */ + groups?: (string[]|null); } - /** Represents a QueryWarning. */ - class QueryWarning implements IQueryWarning { + /** Represents a VTGateCallerID. */ + class VTGateCallerID implements IVTGateCallerID { /** - * Constructs a new QueryWarning. + * Constructs a new VTGateCallerID. * @param [properties] Properties to set */ - constructor(properties?: query.IQueryWarning); + constructor(properties?: query.IVTGateCallerID); - /** QueryWarning code. */ - public code: number; + /** VTGateCallerID username. */ + public username: string; - /** QueryWarning message. */ - public message: string; + /** VTGateCallerID groups. */ + public groups: string[]; /** - * Creates a new QueryWarning instance using the specified properties. + * Creates a new VTGateCallerID instance using the specified properties. * @param [properties] Properties to set - * @returns QueryWarning instance + * @returns VTGateCallerID instance */ - public static create(properties?: query.IQueryWarning): query.QueryWarning; + public static create(properties?: query.IVTGateCallerID): query.VTGateCallerID; /** - * Encodes the specified QueryWarning message. Does not implicitly {@link query.QueryWarning.verify|verify} messages. - * @param message QueryWarning message or plain object to encode + * Encodes the specified VTGateCallerID message. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. + * @param message VTGateCallerID message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IQueryWarning, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IVTGateCallerID, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified QueryWarning message, length delimited. Does not implicitly {@link query.QueryWarning.verify|verify} messages. - * @param message QueryWarning message or plain object to encode + * Encodes the specified VTGateCallerID message, length delimited. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. + * @param message VTGateCallerID message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IQueryWarning, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IVTGateCallerID, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a QueryWarning message from the specified reader or buffer. + * Decodes a VTGateCallerID message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns QueryWarning + * @returns VTGateCallerID * @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): query.QueryWarning; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.VTGateCallerID; /** - * Decodes a QueryWarning message from the specified reader or buffer, length delimited. + * Decodes a VTGateCallerID message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns QueryWarning + * @returns VTGateCallerID * @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)): query.QueryWarning; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.VTGateCallerID; /** - * Verifies a QueryWarning message. + * Verifies a VTGateCallerID 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 QueryWarning message from a plain object. Also converts values to their respective internal types. + * Creates a VTGateCallerID message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns QueryWarning + * @returns VTGateCallerID */ - public static fromObject(object: { [k: string]: any }): query.QueryWarning; + public static fromObject(object: { [k: string]: any }): query.VTGateCallerID; /** - * Creates a plain object from a QueryWarning message. Also converts values to other types if specified. - * @param message QueryWarning + * Creates a plain object from a VTGateCallerID message. Also converts values to other types if specified. + * @param message VTGateCallerID * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.QueryWarning, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.VTGateCallerID, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this QueryWarning to JSON. + * Converts this VTGateCallerID to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for QueryWarning + * Gets the default type url for VTGateCallerID * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamEvent. */ - interface IStreamEvent { + /** Properties of an EventToken. */ + interface IEventToken { - /** StreamEvent statements */ - statements?: (query.StreamEvent.IStatement[]|null); + /** EventToken timestamp */ + timestamp?: (number|Long|null); - /** StreamEvent event_token */ - event_token?: (query.IEventToken|null); + /** EventToken shard */ + shard?: (string|null); + + /** EventToken position */ + position?: (string|null); } - /** Represents a StreamEvent. */ - class StreamEvent implements IStreamEvent { + /** Represents an EventToken. */ + class EventToken implements IEventToken { /** - * Constructs a new StreamEvent. + * Constructs a new EventToken. * @param [properties] Properties to set */ - constructor(properties?: query.IStreamEvent); + constructor(properties?: query.IEventToken); - /** StreamEvent statements. */ - public statements: query.StreamEvent.IStatement[]; + /** EventToken timestamp. */ + public timestamp: (number|Long); - /** StreamEvent event_token. */ - public event_token?: (query.IEventToken|null); + /** EventToken shard. */ + public shard: string; + + /** EventToken position. */ + public position: string; /** - * Creates a new StreamEvent instance using the specified properties. + * Creates a new EventToken instance using the specified properties. * @param [properties] Properties to set - * @returns StreamEvent instance + * @returns EventToken instance */ - public static create(properties?: query.IStreamEvent): query.StreamEvent; + public static create(properties?: query.IEventToken): query.EventToken; /** - * Encodes the specified StreamEvent message. Does not implicitly {@link query.StreamEvent.verify|verify} messages. - * @param message StreamEvent message or plain object to encode + * Encodes the specified EventToken message. Does not implicitly {@link query.EventToken.verify|verify} messages. + * @param message EventToken message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStreamEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IEventToken, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamEvent message, length delimited. Does not implicitly {@link query.StreamEvent.verify|verify} messages. - * @param message StreamEvent message or plain object to encode + * Encodes the specified EventToken message, length delimited. Does not implicitly {@link query.EventToken.verify|verify} messages. + * @param message EventToken message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStreamEvent, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IEventToken, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamEvent message from the specified reader or buffer. + * Decodes an EventToken message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamEvent + * @returns EventToken * @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): query.StreamEvent; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.EventToken; /** - * Decodes a StreamEvent message from the specified reader or buffer, length delimited. + * Decodes an EventToken message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamEvent + * @returns EventToken * @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)): query.StreamEvent; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.EventToken; /** - * Verifies a StreamEvent message. + * Verifies an EventToken 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 StreamEvent message from a plain object. Also converts values to their respective internal types. + * Creates an EventToken message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamEvent + * @returns EventToken */ - public static fromObject(object: { [k: string]: any }): query.StreamEvent; + public static fromObject(object: { [k: string]: any }): query.EventToken; /** - * Creates a plain object from a StreamEvent message. Also converts values to other types if specified. - * @param message StreamEvent + * Creates a plain object from an EventToken message. Also converts values to other types if specified. + * @param message EventToken * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StreamEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.EventToken, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamEvent to JSON. + * Converts this EventToken to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamEvent + * Gets the default type url for EventToken * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace StreamEvent { - - /** Properties of a Statement. */ - interface IStatement { - - /** Statement category */ - category?: (query.StreamEvent.Statement.Category|null); + /** MySqlFlag enum. */ + enum MySqlFlag { + EMPTY = 0, + NOT_NULL_FLAG = 1, + PRI_KEY_FLAG = 2, + UNIQUE_KEY_FLAG = 4, + MULTIPLE_KEY_FLAG = 8, + BLOB_FLAG = 16, + UNSIGNED_FLAG = 32, + ZEROFILL_FLAG = 64, + BINARY_FLAG = 128, + ENUM_FLAG = 256, + AUTO_INCREMENT_FLAG = 512, + TIMESTAMP_FLAG = 1024, + SET_FLAG = 2048, + NO_DEFAULT_VALUE_FLAG = 4096, + ON_UPDATE_NOW_FLAG = 8192, + NUM_FLAG = 32768, + PART_KEY_FLAG = 16384, + GROUP_FLAG = 32768, + UNIQUE_FLAG = 65536, + BINCMP_FLAG = 131072 + } - /** Statement table_name */ - table_name?: (string|null); + /** Flag enum. */ + enum Flag { + NONE = 0, + ISINTEGRAL = 256, + ISUNSIGNED = 512, + ISFLOAT = 1024, + ISQUOTED = 2048, + ISTEXT = 4096, + ISBINARY = 8192 + } - /** Statement primary_key_fields */ - primary_key_fields?: (query.IField[]|null); + /** Type enum. */ + enum Type { + NULL_TYPE = 0, + INT8 = 257, + UINT8 = 770, + INT16 = 259, + UINT16 = 772, + INT24 = 261, + UINT24 = 774, + INT32 = 263, + UINT32 = 776, + INT64 = 265, + UINT64 = 778, + FLOAT32 = 1035, + FLOAT64 = 1036, + TIMESTAMP = 2061, + DATE = 2062, + TIME = 2063, + DATETIME = 2064, + YEAR = 785, + DECIMAL = 18, + TEXT = 6163, + BLOB = 10260, + VARCHAR = 6165, + VARBINARY = 10262, + CHAR = 6167, + BINARY = 10264, + BIT = 2073, + ENUM = 2074, + SET = 2075, + TUPLE = 28, + GEOMETRY = 2077, + JSON = 2078, + EXPRESSION = 31, + HEXNUM = 4128, + HEXVAL = 4129, + BITNUM = 4130 + } - /** Statement primary_key_values */ - primary_key_values?: (query.IRow[]|null); + /** Properties of a Value. */ + interface IValue { - /** Statement sql */ - sql?: (Uint8Array|null); - } + /** Value type */ + type?: (query.Type|null); - /** Represents a Statement. */ - class Statement implements IStatement { + /** Value value */ + value?: (Uint8Array|null); + } - /** - * Constructs a new Statement. - * @param [properties] Properties to set - */ - constructor(properties?: query.StreamEvent.IStatement); + /** Represents a Value. */ + class Value implements IValue { - /** Statement category. */ - public category: query.StreamEvent.Statement.Category; + /** + * Constructs a new Value. + * @param [properties] Properties to set + */ + constructor(properties?: query.IValue); - /** Statement table_name. */ - public table_name: string; + /** Value type. */ + public type: query.Type; - /** Statement primary_key_fields. */ - public primary_key_fields: query.IField[]; + /** Value value. */ + public value: Uint8Array; - /** Statement primary_key_values. */ - public primary_key_values: query.IRow[]; + /** + * Creates a new Value instance using the specified properties. + * @param [properties] Properties to set + * @returns Value instance + */ + public static create(properties?: query.IValue): query.Value; - /** Statement sql. */ - public sql: Uint8Array; - - /** - * Creates a new Statement instance using the specified properties. - * @param [properties] Properties to set - * @returns Statement instance - */ - public static create(properties?: query.StreamEvent.IStatement): query.StreamEvent.Statement; - - /** - * Encodes the specified Statement message. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. - * @param message Statement message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: query.StreamEvent.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified Statement message, length delimited. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. - * @param message Statement message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: query.StreamEvent.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Decodes a Statement message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns Statement - * @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): query.StreamEvent.Statement; - - /** - * Decodes a Statement message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns Statement - * @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)): query.StreamEvent.Statement; - - /** - * Verifies a Statement 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 Statement message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns Statement - */ - public static fromObject(object: { [k: string]: any }): query.StreamEvent.Statement; - - /** - * Creates a plain object from a Statement message. Also converts values to other types if specified. - * @param message Statement - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: query.StreamEvent.Statement, options?: $protobuf.IConversionOptions): { [k: string]: any }; - - /** - * Converts this Statement to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; - - /** - * Gets the default type url for Statement - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } - - namespace Statement { - - /** Category enum. */ - enum Category { - Error = 0, - DML = 1, - DDL = 2 - } - } - } - - /** Properties of an ExecuteRequest. */ - interface IExecuteRequest { - - /** ExecuteRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); - - /** ExecuteRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); - - /** ExecuteRequest target */ - target?: (query.ITarget|null); - - /** ExecuteRequest query */ - query?: (query.IBoundQuery|null); - - /** ExecuteRequest transaction_id */ - transaction_id?: (number|Long|null); - - /** ExecuteRequest options */ - options?: (query.IExecuteOptions|null); - - /** ExecuteRequest reserved_id */ - reserved_id?: (number|Long|null); - } - - /** Represents an ExecuteRequest. */ - class ExecuteRequest implements IExecuteRequest { - - /** - * Constructs a new ExecuteRequest. - * @param [properties] Properties to set - */ - constructor(properties?: query.IExecuteRequest); - - /** ExecuteRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); - - /** ExecuteRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); - - /** ExecuteRequest target. */ - public target?: (query.ITarget|null); - - /** ExecuteRequest query. */ - public query?: (query.IBoundQuery|null); - - /** ExecuteRequest transaction_id. */ - public transaction_id: (number|Long); - - /** ExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); - - /** ExecuteRequest reserved_id. */ - public reserved_id: (number|Long); + /** + * Encodes the specified Value message. Does not implicitly {@link query.Value.verify|verify} messages. + * @param message Value message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.IValue, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Creates a new ExecuteRequest instance using the specified properties. - * @param [properties] Properties to set - * @returns ExecuteRequest instance - */ - public static create(properties?: query.IExecuteRequest): query.ExecuteRequest; - - /** - * Encodes the specified ExecuteRequest message. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. - * @param message ExecuteRequest message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: query.IExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified ExecuteRequest message, length delimited. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. - * @param message ExecuteRequest message or plain object to encode + * Encodes the specified Value message, length delimited. Does not implicitly {@link query.Value.verify|verify} messages. + * @param message Value message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IValue, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteRequest message from the specified reader or buffer. + * Decodes a Value message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteRequest + * @returns Value * @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): query.ExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.Value; /** - * Decodes an ExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a Value message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteRequest + * @returns Value * @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)): query.ExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.Value; /** - * Verifies an ExecuteRequest message. + * Verifies a Value 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 ExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a Value message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteRequest + * @returns Value */ - public static fromObject(object: { [k: string]: any }): query.ExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.Value; /** - * Creates a plain object from an ExecuteRequest message. Also converts values to other types if specified. - * @param message ExecuteRequest + * Creates a plain object from a Value message. Also converts values to other types if specified. + * @param message Value * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.Value, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteRequest to JSON. + * Converts this Value to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteRequest + * Gets the default type url for Value * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteResponse. */ - interface IExecuteResponse { + /** Properties of a BindVariable. */ + interface IBindVariable { - /** ExecuteResponse result */ - result?: (query.IQueryResult|null); + /** BindVariable type */ + type?: (query.Type|null); + + /** BindVariable value */ + value?: (Uint8Array|null); + + /** BindVariable values */ + values?: (query.IValue[]|null); } - /** Represents an ExecuteResponse. */ - class ExecuteResponse implements IExecuteResponse { + /** Represents a BindVariable. */ + class BindVariable implements IBindVariable { /** - * Constructs a new ExecuteResponse. + * Constructs a new BindVariable. * @param [properties] Properties to set */ - constructor(properties?: query.IExecuteResponse); + constructor(properties?: query.IBindVariable); - /** ExecuteResponse result. */ - public result?: (query.IQueryResult|null); + /** BindVariable type. */ + public type: query.Type; + + /** BindVariable value. */ + public value: Uint8Array; + + /** BindVariable values. */ + public values: query.IValue[]; /** - * Creates a new ExecuteResponse instance using the specified properties. + * Creates a new BindVariable instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteResponse instance + * @returns BindVariable instance */ - public static create(properties?: query.IExecuteResponse): query.ExecuteResponse; + public static create(properties?: query.IBindVariable): query.BindVariable; /** - * Encodes the specified ExecuteResponse message. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. - * @param message ExecuteResponse message or plain object to encode + * Encodes the specified BindVariable message. Does not implicitly {@link query.BindVariable.verify|verify} messages. + * @param message BindVariable message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBindVariable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteResponse message, length delimited. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. - * @param message ExecuteResponse message or plain object to encode + * Encodes the specified BindVariable message, length delimited. Does not implicitly {@link query.BindVariable.verify|verify} messages. + * @param message BindVariable message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBindVariable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteResponse message from the specified reader or buffer. + * Decodes a BindVariable message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteResponse + * @returns BindVariable * @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): query.ExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BindVariable; /** - * Decodes an ExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a BindVariable message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteResponse + * @returns BindVariable * @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)): query.ExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BindVariable; /** - * Verifies an ExecuteResponse message. + * Verifies a BindVariable 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 ExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BindVariable message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteResponse + * @returns BindVariable */ - public static fromObject(object: { [k: string]: any }): query.ExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.BindVariable; /** - * Creates a plain object from an ExecuteResponse message. Also converts values to other types if specified. - * @param message ExecuteResponse + * Creates a plain object from a BindVariable message. Also converts values to other types if specified. + * @param message BindVariable * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BindVariable, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteResponse to JSON. + * Converts this BindVariable to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteResponse + * Gets the default type url for BindVariable * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ResultWithError. */ - interface IResultWithError { + /** Properties of a BoundQuery. */ + interface IBoundQuery { - /** ResultWithError error */ - error?: (vtrpc.IRPCError|null); + /** BoundQuery sql */ + sql?: (string|null); - /** ResultWithError result */ - result?: (query.IQueryResult|null); + /** BoundQuery bind_variables */ + bind_variables?: ({ [k: string]: query.IBindVariable }|null); } - /** Represents a ResultWithError. */ - class ResultWithError implements IResultWithError { + /** Represents a BoundQuery. */ + class BoundQuery implements IBoundQuery { /** - * Constructs a new ResultWithError. + * Constructs a new BoundQuery. * @param [properties] Properties to set */ - constructor(properties?: query.IResultWithError); + constructor(properties?: query.IBoundQuery); - /** ResultWithError error. */ - public error?: (vtrpc.IRPCError|null); + /** BoundQuery sql. */ + public sql: string; - /** ResultWithError result. */ - public result?: (query.IQueryResult|null); + /** BoundQuery bind_variables. */ + public bind_variables: { [k: string]: query.IBindVariable }; /** - * Creates a new ResultWithError instance using the specified properties. + * Creates a new BoundQuery instance using the specified properties. * @param [properties] Properties to set - * @returns ResultWithError instance + * @returns BoundQuery instance */ - public static create(properties?: query.IResultWithError): query.ResultWithError; + public static create(properties?: query.IBoundQuery): query.BoundQuery; /** - * Encodes the specified ResultWithError message. Does not implicitly {@link query.ResultWithError.verify|verify} messages. - * @param message ResultWithError message or plain object to encode + * Encodes the specified BoundQuery message. Does not implicitly {@link query.BoundQuery.verify|verify} messages. + * @param message BoundQuery message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IResultWithError, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBoundQuery, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ResultWithError message, length delimited. Does not implicitly {@link query.ResultWithError.verify|verify} messages. - * @param message ResultWithError message or plain object to encode + * Encodes the specified BoundQuery message, length delimited. Does not implicitly {@link query.BoundQuery.verify|verify} messages. + * @param message BoundQuery message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IResultWithError, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBoundQuery, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ResultWithError message from the specified reader or buffer. + * Decodes a BoundQuery message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ResultWithError + * @returns BoundQuery * @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): query.ResultWithError; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BoundQuery; /** - * Decodes a ResultWithError message from the specified reader or buffer, length delimited. + * Decodes a BoundQuery message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ResultWithError + * @returns BoundQuery * @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)): query.ResultWithError; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BoundQuery; /** - * Verifies a ResultWithError message. + * Verifies a BoundQuery 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 ResultWithError message from a plain object. Also converts values to their respective internal types. + * Creates a BoundQuery message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ResultWithError + * @returns BoundQuery */ - public static fromObject(object: { [k: string]: any }): query.ResultWithError; + public static fromObject(object: { [k: string]: any }): query.BoundQuery; /** - * Creates a plain object from a ResultWithError message. Also converts values to other types if specified. - * @param message ResultWithError + * Creates a plain object from a BoundQuery message. Also converts values to other types if specified. + * @param message BoundQuery * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ResultWithError, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BoundQuery, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ResultWithError to JSON. + * Converts this BoundQuery to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ResultWithError + * Gets the default type url for BoundQuery * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamExecuteRequest. */ - interface IStreamExecuteRequest { + /** Properties of an ExecuteOptions. */ + interface IExecuteOptions { - /** StreamExecuteRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); + /** ExecuteOptions included_fields */ + included_fields?: (query.ExecuteOptions.IncludedFields|null); - /** StreamExecuteRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** ExecuteOptions client_found_rows */ + client_found_rows?: (boolean|null); - /** StreamExecuteRequest target */ - target?: (query.ITarget|null); + /** ExecuteOptions workload */ + workload?: (query.ExecuteOptions.Workload|null); - /** StreamExecuteRequest query */ - query?: (query.IBoundQuery|null); + /** ExecuteOptions sql_select_limit */ + sql_select_limit?: (number|Long|null); - /** StreamExecuteRequest options */ - options?: (query.IExecuteOptions|null); + /** ExecuteOptions transaction_isolation */ + transaction_isolation?: (query.ExecuteOptions.TransactionIsolation|null); - /** StreamExecuteRequest transaction_id */ - transaction_id?: (number|Long|null); + /** ExecuteOptions skip_query_plan_cache */ + skip_query_plan_cache?: (boolean|null); - /** StreamExecuteRequest reserved_id */ - reserved_id?: (number|Long|null); + /** ExecuteOptions planner_version */ + planner_version?: (query.ExecuteOptions.PlannerVersion|null); + + /** ExecuteOptions has_created_temp_tables */ + has_created_temp_tables?: (boolean|null); + + /** ExecuteOptions consolidator */ + consolidator?: (query.ExecuteOptions.Consolidator|null); + + /** ExecuteOptions transaction_access_mode */ + transaction_access_mode?: (query.ExecuteOptions.TransactionAccessMode[]|null); + + /** ExecuteOptions WorkloadName */ + WorkloadName?: (string|null); + + /** ExecuteOptions priority */ + priority?: (string|null); } - /** Represents a StreamExecuteRequest. */ - class StreamExecuteRequest implements IStreamExecuteRequest { + /** Represents an ExecuteOptions. */ + class ExecuteOptions implements IExecuteOptions { /** - * Constructs a new StreamExecuteRequest. + * Constructs a new ExecuteOptions. * @param [properties] Properties to set */ - constructor(properties?: query.IStreamExecuteRequest); + constructor(properties?: query.IExecuteOptions); - /** StreamExecuteRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); + /** ExecuteOptions included_fields. */ + public included_fields: query.ExecuteOptions.IncludedFields; - /** StreamExecuteRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + /** ExecuteOptions client_found_rows. */ + public client_found_rows: boolean; - /** StreamExecuteRequest target. */ - public target?: (query.ITarget|null); + /** ExecuteOptions workload. */ + public workload: query.ExecuteOptions.Workload; - /** StreamExecuteRequest query. */ - public query?: (query.IBoundQuery|null); + /** ExecuteOptions sql_select_limit. */ + public sql_select_limit: (number|Long); - /** StreamExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); + /** ExecuteOptions transaction_isolation. */ + public transaction_isolation: query.ExecuteOptions.TransactionIsolation; - /** StreamExecuteRequest transaction_id. */ - public transaction_id: (number|Long); + /** ExecuteOptions skip_query_plan_cache. */ + public skip_query_plan_cache: boolean; - /** StreamExecuteRequest reserved_id. */ - public reserved_id: (number|Long); + /** ExecuteOptions planner_version. */ + public planner_version: query.ExecuteOptions.PlannerVersion; + + /** ExecuteOptions has_created_temp_tables. */ + public has_created_temp_tables: boolean; + + /** ExecuteOptions consolidator. */ + public consolidator: query.ExecuteOptions.Consolidator; + + /** ExecuteOptions transaction_access_mode. */ + public transaction_access_mode: query.ExecuteOptions.TransactionAccessMode[]; + + /** ExecuteOptions WorkloadName. */ + public WorkloadName: string; + + /** ExecuteOptions priority. */ + public priority: string; /** - * Creates a new StreamExecuteRequest instance using the specified properties. + * Creates a new ExecuteOptions instance using the specified properties. * @param [properties] Properties to set - * @returns StreamExecuteRequest instance + * @returns ExecuteOptions instance */ - public static create(properties?: query.IStreamExecuteRequest): query.StreamExecuteRequest; + public static create(properties?: query.IExecuteOptions): query.ExecuteOptions; /** - * Encodes the specified StreamExecuteRequest message. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. - * @param message StreamExecuteRequest message or plain object to encode + * Encodes the specified ExecuteOptions message. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. + * @param message ExecuteOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IExecuteOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamExecuteRequest message, length delimited. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. - * @param message StreamExecuteRequest message or plain object to encode + * Encodes the specified ExecuteOptions message, length delimited. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. + * @param message ExecuteOptions message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IExecuteOptions, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamExecuteRequest message from the specified reader or buffer. + * Decodes an ExecuteOptions message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamExecuteRequest + * @returns ExecuteOptions * @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): query.StreamExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ExecuteOptions; /** - * Decodes a StreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteOptions message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamExecuteRequest + * @returns ExecuteOptions * @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)): query.StreamExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ExecuteOptions; /** - * Verifies a StreamExecuteRequest message. + * Verifies an ExecuteOptions 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 StreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteOptions message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamExecuteRequest + * @returns ExecuteOptions */ - public static fromObject(object: { [k: string]: any }): query.StreamExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.ExecuteOptions; /** - * Creates a plain object from a StreamExecuteRequest message. Also converts values to other types if specified. - * @param message StreamExecuteRequest + * Creates a plain object from an ExecuteOptions message. Also converts values to other types if specified. + * @param message ExecuteOptions * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ExecuteOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamExecuteRequest to JSON. + * Converts this ExecuteOptions to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamExecuteRequest + * Gets the default type url for ExecuteOptions * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamExecuteResponse. */ - interface IStreamExecuteResponse { - - /** StreamExecuteResponse result */ - result?: (query.IQueryResult|null); - } + namespace ExecuteOptions { - /** Represents a StreamExecuteResponse. */ - class StreamExecuteResponse implements IStreamExecuteResponse { + /** IncludedFields enum. */ + enum IncludedFields { + TYPE_AND_NAME = 0, + TYPE_ONLY = 1, + ALL = 2 + } + + /** Workload enum. */ + enum Workload { + UNSPECIFIED = 0, + OLTP = 1, + OLAP = 2, + DBA = 3 + } + + /** TransactionIsolation enum. */ + enum TransactionIsolation { + DEFAULT = 0, + REPEATABLE_READ = 1, + READ_COMMITTED = 2, + READ_UNCOMMITTED = 3, + SERIALIZABLE = 4, + CONSISTENT_SNAPSHOT_READ_ONLY = 5, + AUTOCOMMIT = 6 + } + + /** PlannerVersion enum. */ + enum PlannerVersion { + DEFAULT_PLANNER = 0, + V3 = 1, + Gen4 = 2, + Gen4Greedy = 3, + Gen4Left2Right = 4, + Gen4WithFallback = 5, + Gen4CompareV3 = 6, + V3Insert = 7 + } + + /** Consolidator enum. */ + enum Consolidator { + CONSOLIDATOR_UNSPECIFIED = 0, + CONSOLIDATOR_DISABLED = 1, + CONSOLIDATOR_ENABLED = 2, + CONSOLIDATOR_ENABLED_REPLICAS = 3 + } + + /** TransactionAccessMode enum. */ + enum TransactionAccessMode { + CONSISTENT_SNAPSHOT = 0, + READ_WRITE = 1, + READ_ONLY = 2 + } + } + + /** Properties of a Field. */ + interface IField { + + /** Field name */ + name?: (string|null); + + /** Field type */ + type?: (query.Type|null); + + /** Field table */ + table?: (string|null); + + /** Field org_table */ + org_table?: (string|null); + + /** Field database */ + database?: (string|null); + + /** Field org_name */ + org_name?: (string|null); + + /** Field column_length */ + column_length?: (number|null); + + /** Field charset */ + charset?: (number|null); + + /** Field decimals */ + decimals?: (number|null); + + /** Field flags */ + flags?: (number|null); + + /** Field column_type */ + column_type?: (string|null); + } + + /** Represents a Field. */ + class Field implements IField { /** - * Constructs a new StreamExecuteResponse. + * Constructs a new Field. * @param [properties] Properties to set */ - constructor(properties?: query.IStreamExecuteResponse); + constructor(properties?: query.IField); - /** StreamExecuteResponse result. */ - public result?: (query.IQueryResult|null); + /** Field name. */ + public name: string; + + /** Field type. */ + public type: query.Type; + + /** Field table. */ + public table: string; + + /** Field org_table. */ + public org_table: string; + + /** Field database. */ + public database: string; + + /** Field org_name. */ + public org_name: string; + + /** Field column_length. */ + public column_length: number; + + /** Field charset. */ + public charset: number; + + /** Field decimals. */ + public decimals: number; + + /** Field flags. */ + public flags: number; + + /** Field column_type. */ + public column_type: string; /** - * Creates a new StreamExecuteResponse instance using the specified properties. + * Creates a new Field instance using the specified properties. * @param [properties] Properties to set - * @returns StreamExecuteResponse instance + * @returns Field instance */ - public static create(properties?: query.IStreamExecuteResponse): query.StreamExecuteResponse; + public static create(properties?: query.IField): query.Field; /** - * Encodes the specified StreamExecuteResponse message. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. - * @param message StreamExecuteResponse message or plain object to encode + * Encodes the specified Field message. Does not implicitly {@link query.Field.verify|verify} messages. + * @param message Field message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IField, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamExecuteResponse message, length delimited. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. - * @param message StreamExecuteResponse message or plain object to encode + * Encodes the specified Field message, length delimited. Does not implicitly {@link query.Field.verify|verify} messages. + * @param message Field message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IField, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamExecuteResponse message from the specified reader or buffer. + * Decodes a Field message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamExecuteResponse + * @returns Field * @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): query.StreamExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.Field; /** - * Decodes a StreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a Field message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamExecuteResponse + * @returns Field * @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)): query.StreamExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.Field; /** - * Verifies a StreamExecuteResponse message. + * Verifies a Field 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 StreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a Field message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamExecuteResponse + * @returns Field */ - public static fromObject(object: { [k: string]: any }): query.StreamExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.Field; /** - * Creates a plain object from a StreamExecuteResponse message. Also converts values to other types if specified. - * @param message StreamExecuteResponse + * Creates a plain object from a Field message. Also converts values to other types if specified. + * @param message Field * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.Field, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamExecuteResponse to JSON. + * Converts this Field to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamExecuteResponse + * Gets the default type url for Field * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginRequest. */ - interface IBeginRequest { - - /** BeginRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); - - /** BeginRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** Properties of a Row. */ + interface IRow { - /** BeginRequest target */ - target?: (query.ITarget|null); + /** Row lengths */ + lengths?: ((number|Long)[]|null); - /** BeginRequest options */ - options?: (query.IExecuteOptions|null); + /** Row values */ + values?: (Uint8Array|null); } - /** Represents a BeginRequest. */ - class BeginRequest implements IBeginRequest { + /** Represents a Row. */ + class Row implements IRow { /** - * Constructs a new BeginRequest. + * Constructs a new Row. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginRequest); - - /** BeginRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); - - /** BeginRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + constructor(properties?: query.IRow); - /** BeginRequest target. */ - public target?: (query.ITarget|null); + /** Row lengths. */ + public lengths: (number|Long)[]; - /** BeginRequest options. */ - public options?: (query.IExecuteOptions|null); + /** Row values. */ + public values: Uint8Array; /** - * Creates a new BeginRequest instance using the specified properties. + * Creates a new Row instance using the specified properties. * @param [properties] Properties to set - * @returns BeginRequest instance + * @returns Row instance */ - public static create(properties?: query.IBeginRequest): query.BeginRequest; + public static create(properties?: query.IRow): query.Row; /** - * Encodes the specified BeginRequest message. Does not implicitly {@link query.BeginRequest.verify|verify} messages. - * @param message BeginRequest message or plain object to encode + * Encodes the specified Row message. Does not implicitly {@link query.Row.verify|verify} messages. + * @param message Row message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRow, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginRequest message, length delimited. Does not implicitly {@link query.BeginRequest.verify|verify} messages. - * @param message BeginRequest message or plain object to encode + * Encodes the specified Row message, length delimited. Does not implicitly {@link query.Row.verify|verify} messages. + * @param message Row message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRow, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginRequest message from the specified reader or buffer. + * Decodes a Row message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginRequest + * @returns Row * @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): query.BeginRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.Row; /** - * Decodes a BeginRequest message from the specified reader or buffer, length delimited. + * Decodes a Row message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginRequest + * @returns Row * @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)): query.BeginRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.Row; /** - * Verifies a BeginRequest message. + * Verifies a Row 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 BeginRequest message from a plain object. Also converts values to their respective internal types. + * Creates a Row message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginRequest + * @returns Row */ - public static fromObject(object: { [k: string]: any }): query.BeginRequest; + public static fromObject(object: { [k: string]: any }): query.Row; /** - * Creates a plain object from a BeginRequest message. Also converts values to other types if specified. - * @param message BeginRequest + * Creates a plain object from a Row message. Also converts values to other types if specified. + * @param message Row * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.Row, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginRequest to JSON. + * Converts this Row to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginRequest + * Gets the default type url for Row * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginResponse. */ - interface IBeginResponse { + /** Properties of a QueryResult. */ + interface IQueryResult { - /** BeginResponse transaction_id */ - transaction_id?: (number|Long|null); + /** QueryResult fields */ + fields?: (query.IField[]|null); - /** BeginResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** QueryResult rows_affected */ + rows_affected?: (number|Long|null); - /** BeginResponse session_state_changes */ + /** QueryResult insert_id */ + insert_id?: (number|Long|null); + + /** QueryResult rows */ + rows?: (query.IRow[]|null); + + /** QueryResult info */ + info?: (string|null); + + /** QueryResult session_state_changes */ session_state_changes?: (string|null); } - /** Represents a BeginResponse. */ - class BeginResponse implements IBeginResponse { + /** Represents a QueryResult. */ + class QueryResult implements IQueryResult { /** - * Constructs a new BeginResponse. + * Constructs a new QueryResult. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginResponse); + constructor(properties?: query.IQueryResult); - /** BeginResponse transaction_id. */ - public transaction_id: (number|Long); + /** QueryResult fields. */ + public fields: query.IField[]; - /** BeginResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** QueryResult rows_affected. */ + public rows_affected: (number|Long); - /** BeginResponse session_state_changes. */ + /** QueryResult insert_id. */ + public insert_id: (number|Long); + + /** QueryResult rows. */ + public rows: query.IRow[]; + + /** QueryResult info. */ + public info: string; + + /** QueryResult session_state_changes. */ public session_state_changes: string; /** - * Creates a new BeginResponse instance using the specified properties. + * Creates a new QueryResult instance using the specified properties. * @param [properties] Properties to set - * @returns BeginResponse instance + * @returns QueryResult instance */ - public static create(properties?: query.IBeginResponse): query.BeginResponse; + public static create(properties?: query.IQueryResult): query.QueryResult; /** - * Encodes the specified BeginResponse message. Does not implicitly {@link query.BeginResponse.verify|verify} messages. - * @param message BeginResponse message or plain object to encode + * Encodes the specified QueryResult message. Does not implicitly {@link query.QueryResult.verify|verify} messages. + * @param message QueryResult message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IQueryResult, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginResponse message, length delimited. Does not implicitly {@link query.BeginResponse.verify|verify} messages. - * @param message BeginResponse message or plain object to encode + * Encodes the specified QueryResult message, length delimited. Does not implicitly {@link query.QueryResult.verify|verify} messages. + * @param message QueryResult message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IQueryResult, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginResponse message from the specified reader or buffer. + * Decodes a QueryResult message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginResponse + * @returns QueryResult * @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): query.BeginResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.QueryResult; /** - * Decodes a BeginResponse message from the specified reader or buffer, length delimited. + * Decodes a QueryResult message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginResponse + * @returns QueryResult * @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)): query.BeginResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.QueryResult; /** - * Verifies a BeginResponse message. + * Verifies a QueryResult 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 BeginResponse message from a plain object. Also converts values to their respective internal types. + * Creates a QueryResult message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginResponse + * @returns QueryResult */ - public static fromObject(object: { [k: string]: any }): query.BeginResponse; + public static fromObject(object: { [k: string]: any }): query.QueryResult; /** - * Creates a plain object from a BeginResponse message. Also converts values to other types if specified. - * @param message BeginResponse + * Creates a plain object from a QueryResult message. Also converts values to other types if specified. + * @param message QueryResult * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.QueryResult, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginResponse to JSON. + * Converts this QueryResult to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginResponse + * Gets the default type url for QueryResult * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CommitRequest. */ - interface ICommitRequest { - - /** CommitRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); - - /** CommitRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** Properties of a QueryWarning. */ + interface IQueryWarning { - /** CommitRequest target */ - target?: (query.ITarget|null); + /** QueryWarning code */ + code?: (number|null); - /** CommitRequest transaction_id */ - transaction_id?: (number|Long|null); + /** QueryWarning message */ + message?: (string|null); } - /** Represents a CommitRequest. */ - class CommitRequest implements ICommitRequest { + /** Represents a QueryWarning. */ + class QueryWarning implements IQueryWarning { /** - * Constructs a new CommitRequest. + * Constructs a new QueryWarning. * @param [properties] Properties to set */ - constructor(properties?: query.ICommitRequest); - - /** CommitRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); - - /** CommitRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); + constructor(properties?: query.IQueryWarning); - /** CommitRequest target. */ - public target?: (query.ITarget|null); + /** QueryWarning code. */ + public code: number; - /** CommitRequest transaction_id. */ - public transaction_id: (number|Long); + /** QueryWarning message. */ + public message: string; /** - * Creates a new CommitRequest instance using the specified properties. + * Creates a new QueryWarning instance using the specified properties. * @param [properties] Properties to set - * @returns CommitRequest instance + * @returns QueryWarning instance */ - public static create(properties?: query.ICommitRequest): query.CommitRequest; + public static create(properties?: query.IQueryWarning): query.QueryWarning; /** - * Encodes the specified CommitRequest message. Does not implicitly {@link query.CommitRequest.verify|verify} messages. - * @param message CommitRequest message or plain object to encode + * Encodes the specified QueryWarning message. Does not implicitly {@link query.QueryWarning.verify|verify} messages. + * @param message QueryWarning message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IQueryWarning, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CommitRequest message, length delimited. Does not implicitly {@link query.CommitRequest.verify|verify} messages. - * @param message CommitRequest message or plain object to encode + * Encodes the specified QueryWarning message, length delimited. Does not implicitly {@link query.QueryWarning.verify|verify} messages. + * @param message QueryWarning message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IQueryWarning, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CommitRequest message from the specified reader or buffer. + * Decodes a QueryWarning message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CommitRequest + * @returns QueryWarning * @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): query.CommitRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.QueryWarning; /** - * Decodes a CommitRequest message from the specified reader or buffer, length delimited. + * Decodes a QueryWarning message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CommitRequest + * @returns QueryWarning * @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)): query.CommitRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.QueryWarning; /** - * Verifies a CommitRequest message. + * Verifies a QueryWarning 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 CommitRequest message from a plain object. Also converts values to their respective internal types. + * Creates a QueryWarning message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CommitRequest + * @returns QueryWarning */ - public static fromObject(object: { [k: string]: any }): query.CommitRequest; + public static fromObject(object: { [k: string]: any }): query.QueryWarning; /** - * Creates a plain object from a CommitRequest message. Also converts values to other types if specified. - * @param message CommitRequest + * Creates a plain object from a QueryWarning message. Also converts values to other types if specified. + * @param message QueryWarning * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CommitRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.QueryWarning, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CommitRequest to JSON. + * Converts this QueryWarning to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CommitRequest + * Gets the default type url for QueryWarning * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CommitResponse. */ - interface ICommitResponse { + /** Properties of a StreamEvent. */ + interface IStreamEvent { - /** CommitResponse reserved_id */ - reserved_id?: (number|Long|null); + /** StreamEvent statements */ + statements?: (query.StreamEvent.IStatement[]|null); + + /** StreamEvent event_token */ + event_token?: (query.IEventToken|null); } - /** Represents a CommitResponse. */ - class CommitResponse implements ICommitResponse { + /** Represents a StreamEvent. */ + class StreamEvent implements IStreamEvent { /** - * Constructs a new CommitResponse. + * Constructs a new StreamEvent. * @param [properties] Properties to set */ - constructor(properties?: query.ICommitResponse); + constructor(properties?: query.IStreamEvent); - /** CommitResponse reserved_id. */ - public reserved_id: (number|Long); + /** StreamEvent statements. */ + public statements: query.StreamEvent.IStatement[]; + + /** StreamEvent event_token. */ + public event_token?: (query.IEventToken|null); /** - * Creates a new CommitResponse instance using the specified properties. + * Creates a new StreamEvent instance using the specified properties. * @param [properties] Properties to set - * @returns CommitResponse instance + * @returns StreamEvent instance */ - public static create(properties?: query.ICommitResponse): query.CommitResponse; + public static create(properties?: query.IStreamEvent): query.StreamEvent; /** - * Encodes the specified CommitResponse message. Does not implicitly {@link query.CommitResponse.verify|verify} messages. - * @param message CommitResponse message or plain object to encode + * Encodes the specified StreamEvent message. Does not implicitly {@link query.StreamEvent.verify|verify} messages. + * @param message StreamEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IStreamEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CommitResponse message, length delimited. Does not implicitly {@link query.CommitResponse.verify|verify} messages. - * @param message CommitResponse message or plain object to encode + * Encodes the specified StreamEvent message, length delimited. Does not implicitly {@link query.StreamEvent.verify|verify} messages. + * @param message StreamEvent message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IStreamEvent, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CommitResponse message from the specified reader or buffer. + * Decodes a StreamEvent message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CommitResponse + * @returns StreamEvent * @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): query.CommitResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.StreamEvent; /** - * Decodes a CommitResponse message from the specified reader or buffer, length delimited. + * Decodes a StreamEvent message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CommitResponse + * @returns StreamEvent * @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)): query.CommitResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.StreamEvent; /** - * Verifies a CommitResponse message. + * Verifies a StreamEvent 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 CommitResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StreamEvent message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CommitResponse + * @returns StreamEvent */ - public static fromObject(object: { [k: string]: any }): query.CommitResponse; + public static fromObject(object: { [k: string]: any }): query.StreamEvent; /** - * Creates a plain object from a CommitResponse message. Also converts values to other types if specified. - * @param message CommitResponse + * Creates a plain object from a StreamEvent message. Also converts values to other types if specified. + * @param message StreamEvent * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CommitResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.StreamEvent, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CommitResponse to JSON. + * Converts this StreamEvent to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CommitResponse + * Gets the default type url for StreamEvent * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RollbackRequest. */ - interface IRollbackRequest { + namespace StreamEvent { - /** RollbackRequest effective_caller_id */ + /** Properties of a Statement. */ + interface IStatement { + + /** Statement category */ + category?: (query.StreamEvent.Statement.Category|null); + + /** Statement table_name */ + table_name?: (string|null); + + /** Statement primary_key_fields */ + primary_key_fields?: (query.IField[]|null); + + /** Statement primary_key_values */ + primary_key_values?: (query.IRow[]|null); + + /** Statement sql */ + sql?: (Uint8Array|null); + } + + /** Represents a Statement. */ + class Statement implements IStatement { + + /** + * Constructs a new Statement. + * @param [properties] Properties to set + */ + constructor(properties?: query.StreamEvent.IStatement); + + /** Statement category. */ + public category: query.StreamEvent.Statement.Category; + + /** Statement table_name. */ + public table_name: string; + + /** Statement primary_key_fields. */ + public primary_key_fields: query.IField[]; + + /** Statement primary_key_values. */ + public primary_key_values: query.IRow[]; + + /** Statement sql. */ + public sql: Uint8Array; + + /** + * Creates a new Statement instance using the specified properties. + * @param [properties] Properties to set + * @returns Statement instance + */ + public static create(properties?: query.StreamEvent.IStatement): query.StreamEvent.Statement; + + /** + * Encodes the specified Statement message. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. + * @param message Statement message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.StreamEvent.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Statement message, length delimited. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. + * @param message Statement message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: query.StreamEvent.IStatement, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Statement message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Statement + * @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): query.StreamEvent.Statement; + + /** + * Decodes a Statement message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Statement + * @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)): query.StreamEvent.Statement; + + /** + * Verifies a Statement 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 Statement message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Statement + */ + public static fromObject(object: { [k: string]: any }): query.StreamEvent.Statement; + + /** + * Creates a plain object from a Statement message. Also converts values to other types if specified. + * @param message Statement + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: query.StreamEvent.Statement, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Statement to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Statement + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + namespace Statement { + + /** Category enum. */ + enum Category { + Error = 0, + DML = 1, + DDL = 2 + } + } + } + + /** Properties of an ExecuteRequest. */ + interface IExecuteRequest { + + /** ExecuteRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** RollbackRequest immediate_caller_id */ + /** ExecuteRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** RollbackRequest target */ + /** ExecuteRequest target */ target?: (query.ITarget|null); - /** RollbackRequest transaction_id */ + /** ExecuteRequest query */ + query?: (query.IBoundQuery|null); + + /** ExecuteRequest transaction_id */ transaction_id?: (number|Long|null); + + /** ExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** ExecuteRequest reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a RollbackRequest. */ - class RollbackRequest implements IRollbackRequest { + /** Represents an ExecuteRequest. */ + class ExecuteRequest implements IExecuteRequest { /** - * Constructs a new RollbackRequest. + * Constructs a new ExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IRollbackRequest); + constructor(properties?: query.IExecuteRequest); - /** RollbackRequest effective_caller_id. */ + /** ExecuteRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** RollbackRequest immediate_caller_id. */ + /** ExecuteRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** RollbackRequest target. */ + /** ExecuteRequest target. */ public target?: (query.ITarget|null); - /** RollbackRequest transaction_id. */ + /** ExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** ExecuteRequest transaction_id. */ public transaction_id: (number|Long); + /** ExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** ExecuteRequest reserved_id. */ + public reserved_id: (number|Long); + /** - * Creates a new RollbackRequest instance using the specified properties. + * Creates a new ExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RollbackRequest instance + * @returns ExecuteRequest instance */ - public static create(properties?: query.IRollbackRequest): query.RollbackRequest; + public static create(properties?: query.IExecuteRequest): query.ExecuteRequest; /** - * Encodes the specified RollbackRequest message. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. - * @param message RollbackRequest message or plain object to encode + * Encodes the specified ExecuteRequest message. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. + * @param message ExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RollbackRequest message, length delimited. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. - * @param message RollbackRequest message or plain object to encode + * Encodes the specified ExecuteRequest message, length delimited. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. + * @param message ExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RollbackRequest message from the specified reader or buffer. + * Decodes an ExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RollbackRequest + * @returns ExecuteRequest * @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): query.RollbackRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ExecuteRequest; /** - * Decodes a RollbackRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RollbackRequest + * @returns ExecuteRequest * @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)): query.RollbackRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ExecuteRequest; /** - * Verifies a RollbackRequest message. + * Verifies an ExecuteRequest 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 RollbackRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RollbackRequest + * @returns ExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.RollbackRequest; + public static fromObject(object: { [k: string]: any }): query.ExecuteRequest; /** - * Creates a plain object from a RollbackRequest message. Also converts values to other types if specified. - * @param message RollbackRequest + * Creates a plain object from an ExecuteRequest message. Also converts values to other types if specified. + * @param message ExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.RollbackRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RollbackRequest to JSON. + * Converts this ExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RollbackRequest + * Gets the default type url for ExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RollbackResponse. */ - interface IRollbackResponse { + /** Properties of an ExecuteResponse. */ + interface IExecuteResponse { - /** RollbackResponse reserved_id */ - reserved_id?: (number|Long|null); + /** ExecuteResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a RollbackResponse. */ - class RollbackResponse implements IRollbackResponse { + /** Represents an ExecuteResponse. */ + class ExecuteResponse implements IExecuteResponse { /** - * Constructs a new RollbackResponse. + * Constructs a new ExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IRollbackResponse); + constructor(properties?: query.IExecuteResponse); - /** RollbackResponse reserved_id. */ - public reserved_id: (number|Long); + /** ExecuteResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new RollbackResponse instance using the specified properties. + * Creates a new ExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RollbackResponse instance + * @returns ExecuteResponse instance */ - public static create(properties?: query.IRollbackResponse): query.RollbackResponse; + public static create(properties?: query.IExecuteResponse): query.ExecuteResponse; /** - * Encodes the specified RollbackResponse message. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. - * @param message RollbackResponse message or plain object to encode + * Encodes the specified ExecuteResponse message. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. + * @param message ExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RollbackResponse message, length delimited. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. - * @param message RollbackResponse message or plain object to encode + * Encodes the specified ExecuteResponse message, length delimited. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. + * @param message ExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RollbackResponse message from the specified reader or buffer. + * Decodes an ExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RollbackResponse + * @returns ExecuteResponse * @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): query.RollbackResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ExecuteResponse; /** - * Decodes a RollbackResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RollbackResponse + * @returns ExecuteResponse * @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)): query.RollbackResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ExecuteResponse; /** - * Verifies a RollbackResponse message. + * Verifies an ExecuteResponse 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 RollbackResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RollbackResponse + * @returns ExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.RollbackResponse; + public static fromObject(object: { [k: string]: any }): query.ExecuteResponse; /** - * Creates a plain object from a RollbackResponse message. Also converts values to other types if specified. - * @param message RollbackResponse + * Creates a plain object from an ExecuteResponse message. Also converts values to other types if specified. + * @param message ExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.RollbackResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RollbackResponse to JSON. + * Converts this ExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RollbackResponse + * Gets the default type url for ExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PrepareRequest. */ - interface IPrepareRequest { - - /** PrepareRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); - - /** PrepareRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); - - /** PrepareRequest target */ - target?: (query.ITarget|null); + /** Properties of a ResultWithError. */ + interface IResultWithError { - /** PrepareRequest transaction_id */ - transaction_id?: (number|Long|null); + /** ResultWithError error */ + error?: (vtrpc.IRPCError|null); - /** PrepareRequest dtid */ - dtid?: (string|null); + /** ResultWithError result */ + result?: (query.IQueryResult|null); } - /** Represents a PrepareRequest. */ - class PrepareRequest implements IPrepareRequest { + /** Represents a ResultWithError. */ + class ResultWithError implements IResultWithError { /** - * Constructs a new PrepareRequest. + * Constructs a new ResultWithError. * @param [properties] Properties to set */ - constructor(properties?: query.IPrepareRequest); - - /** PrepareRequest effective_caller_id. */ - public effective_caller_id?: (vtrpc.ICallerID|null); - - /** PrepareRequest immediate_caller_id. */ - public immediate_caller_id?: (query.IVTGateCallerID|null); - - /** PrepareRequest target. */ - public target?: (query.ITarget|null); + constructor(properties?: query.IResultWithError); - /** PrepareRequest transaction_id. */ - public transaction_id: (number|Long); + /** ResultWithError error. */ + public error?: (vtrpc.IRPCError|null); - /** PrepareRequest dtid. */ - public dtid: string; + /** ResultWithError result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new PrepareRequest instance using the specified properties. + * Creates a new ResultWithError instance using the specified properties. * @param [properties] Properties to set - * @returns PrepareRequest instance + * @returns ResultWithError instance */ - public static create(properties?: query.IPrepareRequest): query.PrepareRequest; + public static create(properties?: query.IResultWithError): query.ResultWithError; /** - * Encodes the specified PrepareRequest message. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. - * @param message PrepareRequest message or plain object to encode + * Encodes the specified ResultWithError message. Does not implicitly {@link query.ResultWithError.verify|verify} messages. + * @param message ResultWithError message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IPrepareRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IResultWithError, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PrepareRequest message, length delimited. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. - * @param message PrepareRequest message or plain object to encode + * Encodes the specified ResultWithError message, length delimited. Does not implicitly {@link query.ResultWithError.verify|verify} messages. + * @param message ResultWithError message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IPrepareRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IResultWithError, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PrepareRequest message from the specified reader or buffer. + * Decodes a ResultWithError message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PrepareRequest + * @returns ResultWithError * @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): query.PrepareRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ResultWithError; /** - * Decodes a PrepareRequest message from the specified reader or buffer, length delimited. + * Decodes a ResultWithError message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PrepareRequest + * @returns ResultWithError * @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)): query.PrepareRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ResultWithError; /** - * Verifies a PrepareRequest message. + * Verifies a ResultWithError 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 PrepareRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ResultWithError message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PrepareRequest + * @returns ResultWithError */ - public static fromObject(object: { [k: string]: any }): query.PrepareRequest; + public static fromObject(object: { [k: string]: any }): query.ResultWithError; /** - * Creates a plain object from a PrepareRequest message. Also converts values to other types if specified. - * @param message PrepareRequest + * Creates a plain object from a ResultWithError message. Also converts values to other types if specified. + * @param message ResultWithError * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.PrepareRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ResultWithError, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PrepareRequest to JSON. + * Converts this ResultWithError to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PrepareRequest + * Gets the default type url for ResultWithError * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PrepareResponse. */ - interface IPrepareResponse { + /** Properties of a StreamExecuteRequest. */ + interface IStreamExecuteRequest { + + /** StreamExecuteRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); + + /** StreamExecuteRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** StreamExecuteRequest target */ + target?: (query.ITarget|null); + + /** StreamExecuteRequest query */ + query?: (query.IBoundQuery|null); + + /** StreamExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** StreamExecuteRequest transaction_id */ + transaction_id?: (number|Long|null); + + /** StreamExecuteRequest reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a PrepareResponse. */ - class PrepareResponse implements IPrepareResponse { + /** Represents a StreamExecuteRequest. */ + class StreamExecuteRequest implements IStreamExecuteRequest { /** - * Constructs a new PrepareResponse. + * Constructs a new StreamExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IPrepareResponse); + constructor(properties?: query.IStreamExecuteRequest); + + /** StreamExecuteRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); + + /** StreamExecuteRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** StreamExecuteRequest target. */ + public target?: (query.ITarget|null); + + /** StreamExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** StreamExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** StreamExecuteRequest transaction_id. */ + public transaction_id: (number|Long); + + /** StreamExecuteRequest reserved_id. */ + public reserved_id: (number|Long); /** - * Creates a new PrepareResponse instance using the specified properties. + * Creates a new StreamExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns PrepareResponse instance + * @returns StreamExecuteRequest instance */ - public static create(properties?: query.IPrepareResponse): query.PrepareResponse; + public static create(properties?: query.IStreamExecuteRequest): query.StreamExecuteRequest; /** - * Encodes the specified PrepareResponse message. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. - * @param message PrepareResponse message or plain object to encode + * Encodes the specified StreamExecuteRequest message. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. + * @param message StreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IPrepareResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PrepareResponse message, length delimited. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. - * @param message PrepareResponse message or plain object to encode + * Encodes the specified StreamExecuteRequest message, length delimited. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. + * @param message StreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IPrepareResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PrepareResponse message from the specified reader or buffer. + * Decodes a StreamExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PrepareResponse + * @returns StreamExecuteRequest * @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): query.PrepareResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.StreamExecuteRequest; /** - * Decodes a PrepareResponse message from the specified reader or buffer, length delimited. + * Decodes a StreamExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PrepareResponse + * @returns StreamExecuteRequest * @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)): query.PrepareResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.StreamExecuteRequest; /** - * Verifies a PrepareResponse message. + * Verifies a StreamExecuteRequest 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 PrepareResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PrepareResponse + * @returns StreamExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.PrepareResponse; + public static fromObject(object: { [k: string]: any }): query.StreamExecuteRequest; /** - * Creates a plain object from a PrepareResponse message. Also converts values to other types if specified. - * @param message PrepareResponse + * Creates a plain object from a StreamExecuteRequest message. Also converts values to other types if specified. + * @param message StreamExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.PrepareResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.StreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PrepareResponse to JSON. + * Converts this StreamExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PrepareResponse + * Gets the default type url for StreamExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CommitPreparedRequest. */ - interface ICommitPreparedRequest { + /** Properties of a StreamExecuteResponse. */ + interface IStreamExecuteResponse { - /** CommitPreparedRequest effective_caller_id */ - effective_caller_id?: (vtrpc.ICallerID|null); + /** StreamExecuteResponse result */ + result?: (query.IQueryResult|null); + } - /** CommitPreparedRequest immediate_caller_id */ - immediate_caller_id?: (query.IVTGateCallerID|null); + /** Represents a StreamExecuteResponse. */ + class StreamExecuteResponse implements IStreamExecuteResponse { - /** CommitPreparedRequest target */ + /** + * Constructs a new StreamExecuteResponse. + * @param [properties] Properties to set + */ + constructor(properties?: query.IStreamExecuteResponse); + + /** StreamExecuteResponse result. */ + public result?: (query.IQueryResult|null); + + /** + * Creates a new StreamExecuteResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns StreamExecuteResponse instance + */ + public static create(properties?: query.IStreamExecuteResponse): query.StreamExecuteResponse; + + /** + * Encodes the specified StreamExecuteResponse message. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. + * @param message StreamExecuteResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.IStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified StreamExecuteResponse message, length delimited. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. + * @param message StreamExecuteResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: query.IStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a StreamExecuteResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns StreamExecuteResponse + * @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): query.StreamExecuteResponse; + + /** + * Decodes a StreamExecuteResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns StreamExecuteResponse + * @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)): query.StreamExecuteResponse; + + /** + * Verifies a StreamExecuteResponse 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 StreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns StreamExecuteResponse + */ + public static fromObject(object: { [k: string]: any }): query.StreamExecuteResponse; + + /** + * Creates a plain object from a StreamExecuteResponse message. Also converts values to other types if specified. + * @param message StreamExecuteResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: query.StreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this StreamExecuteResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for StreamExecuteResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a BeginRequest. */ + interface IBeginRequest { + + /** BeginRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); + + /** BeginRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** BeginRequest target */ target?: (query.ITarget|null); - /** CommitPreparedRequest dtid */ - dtid?: (string|null); + /** BeginRequest options */ + options?: (query.IExecuteOptions|null); } - /** Represents a CommitPreparedRequest. */ - class CommitPreparedRequest implements ICommitPreparedRequest { + /** Represents a BeginRequest. */ + class BeginRequest implements IBeginRequest { /** - * Constructs a new CommitPreparedRequest. + * Constructs a new BeginRequest. * @param [properties] Properties to set */ - constructor(properties?: query.ICommitPreparedRequest); + constructor(properties?: query.IBeginRequest); - /** CommitPreparedRequest effective_caller_id. */ + /** BeginRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** CommitPreparedRequest immediate_caller_id. */ + /** BeginRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** CommitPreparedRequest target. */ + /** BeginRequest target. */ public target?: (query.ITarget|null); - /** CommitPreparedRequest dtid. */ - public dtid: string; + /** BeginRequest options. */ + public options?: (query.IExecuteOptions|null); /** - * Creates a new CommitPreparedRequest instance using the specified properties. + * Creates a new BeginRequest instance using the specified properties. * @param [properties] Properties to set - * @returns CommitPreparedRequest instance + * @returns BeginRequest instance */ - public static create(properties?: query.ICommitPreparedRequest): query.CommitPreparedRequest; + public static create(properties?: query.IBeginRequest): query.BeginRequest; /** - * Encodes the specified CommitPreparedRequest message. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. - * @param message CommitPreparedRequest message or plain object to encode + * Encodes the specified BeginRequest message. Does not implicitly {@link query.BeginRequest.verify|verify} messages. + * @param message BeginRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICommitPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CommitPreparedRequest message, length delimited. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. - * @param message CommitPreparedRequest message or plain object to encode + * Encodes the specified BeginRequest message, length delimited. Does not implicitly {@link query.BeginRequest.verify|verify} messages. + * @param message BeginRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICommitPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CommitPreparedRequest message from the specified reader or buffer. + * Decodes a BeginRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CommitPreparedRequest + * @returns BeginRequest * @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): query.CommitPreparedRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginRequest; /** - * Decodes a CommitPreparedRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CommitPreparedRequest + * @returns BeginRequest * @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)): query.CommitPreparedRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginRequest; /** - * Verifies a CommitPreparedRequest message. + * Verifies a BeginRequest 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 CommitPreparedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CommitPreparedRequest + * @returns BeginRequest */ - public static fromObject(object: { [k: string]: any }): query.CommitPreparedRequest; + public static fromObject(object: { [k: string]: any }): query.BeginRequest; /** - * Creates a plain object from a CommitPreparedRequest message. Also converts values to other types if specified. - * @param message CommitPreparedRequest + * Creates a plain object from a BeginRequest message. Also converts values to other types if specified. + * @param message BeginRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CommitPreparedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CommitPreparedRequest to JSON. + * Converts this BeginRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CommitPreparedRequest + * Gets the default type url for BeginRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CommitPreparedResponse. */ - interface ICommitPreparedResponse { + /** Properties of a BeginResponse. */ + interface IBeginResponse { + + /** BeginResponse transaction_id */ + transaction_id?: (number|Long|null); + + /** BeginResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** BeginResponse session_state_changes */ + session_state_changes?: (string|null); } - /** Represents a CommitPreparedResponse. */ - class CommitPreparedResponse implements ICommitPreparedResponse { + /** Represents a BeginResponse. */ + class BeginResponse implements IBeginResponse { /** - * Constructs a new CommitPreparedResponse. + * Constructs a new BeginResponse. * @param [properties] Properties to set */ - constructor(properties?: query.ICommitPreparedResponse); + constructor(properties?: query.IBeginResponse); + + /** BeginResponse transaction_id. */ + public transaction_id: (number|Long); + + /** BeginResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** BeginResponse session_state_changes. */ + public session_state_changes: string; /** - * Creates a new CommitPreparedResponse instance using the specified properties. + * Creates a new BeginResponse instance using the specified properties. * @param [properties] Properties to set - * @returns CommitPreparedResponse instance + * @returns BeginResponse instance */ - public static create(properties?: query.ICommitPreparedResponse): query.CommitPreparedResponse; + public static create(properties?: query.IBeginResponse): query.BeginResponse; /** - * Encodes the specified CommitPreparedResponse message. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. - * @param message CommitPreparedResponse message or plain object to encode + * Encodes the specified BeginResponse message. Does not implicitly {@link query.BeginResponse.verify|verify} messages. + * @param message BeginResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICommitPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CommitPreparedResponse message, length delimited. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. - * @param message CommitPreparedResponse message or plain object to encode + * Encodes the specified BeginResponse message, length delimited. Does not implicitly {@link query.BeginResponse.verify|verify} messages. + * @param message BeginResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICommitPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CommitPreparedResponse message from the specified reader or buffer. + * Decodes a BeginResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CommitPreparedResponse + * @returns BeginResponse * @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): query.CommitPreparedResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginResponse; /** - * Decodes a CommitPreparedResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CommitPreparedResponse + * @returns BeginResponse * @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)): query.CommitPreparedResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginResponse; /** - * Verifies a CommitPreparedResponse message. + * Verifies a BeginResponse 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 CommitPreparedResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CommitPreparedResponse + * @returns BeginResponse */ - public static fromObject(object: { [k: string]: any }): query.CommitPreparedResponse; + public static fromObject(object: { [k: string]: any }): query.BeginResponse; /** - * Creates a plain object from a CommitPreparedResponse message. Also converts values to other types if specified. - * @param message CommitPreparedResponse + * Creates a plain object from a BeginResponse message. Also converts values to other types if specified. + * @param message BeginResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CommitPreparedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CommitPreparedResponse to JSON. + * Converts this BeginResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CommitPreparedResponse + * Gets the default type url for BeginResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RollbackPreparedRequest. */ - interface IRollbackPreparedRequest { + /** Properties of a CommitRequest. */ + interface ICommitRequest { - /** RollbackPreparedRequest effective_caller_id */ + /** CommitRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** RollbackPreparedRequest immediate_caller_id */ + /** CommitRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** RollbackPreparedRequest target */ + /** CommitRequest target */ target?: (query.ITarget|null); - /** RollbackPreparedRequest transaction_id */ + /** CommitRequest transaction_id */ transaction_id?: (number|Long|null); - - /** RollbackPreparedRequest dtid */ - dtid?: (string|null); } - /** Represents a RollbackPreparedRequest. */ - class RollbackPreparedRequest implements IRollbackPreparedRequest { + /** Represents a CommitRequest. */ + class CommitRequest implements ICommitRequest { /** - * Constructs a new RollbackPreparedRequest. + * Constructs a new CommitRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IRollbackPreparedRequest); + constructor(properties?: query.ICommitRequest); - /** RollbackPreparedRequest effective_caller_id. */ + /** CommitRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** RollbackPreparedRequest immediate_caller_id. */ + /** CommitRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** RollbackPreparedRequest target. */ + /** CommitRequest target. */ public target?: (query.ITarget|null); - /** RollbackPreparedRequest transaction_id. */ + /** CommitRequest transaction_id. */ public transaction_id: (number|Long); - /** RollbackPreparedRequest dtid. */ - public dtid: string; - /** - * Creates a new RollbackPreparedRequest instance using the specified properties. + * Creates a new CommitRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RollbackPreparedRequest instance + * @returns CommitRequest instance */ - public static create(properties?: query.IRollbackPreparedRequest): query.RollbackPreparedRequest; + public static create(properties?: query.ICommitRequest): query.CommitRequest; /** - * Encodes the specified RollbackPreparedRequest message. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. - * @param message RollbackPreparedRequest message or plain object to encode + * Encodes the specified CommitRequest message. Does not implicitly {@link query.CommitRequest.verify|verify} messages. + * @param message CommitRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRollbackPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RollbackPreparedRequest message, length delimited. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. - * @param message RollbackPreparedRequest message or plain object to encode + * Encodes the specified CommitRequest message, length delimited. Does not implicitly {@link query.CommitRequest.verify|verify} messages. + * @param message CommitRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRollbackPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RollbackPreparedRequest message from the specified reader or buffer. + * Decodes a CommitRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RollbackPreparedRequest + * @returns CommitRequest * @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): query.RollbackPreparedRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CommitRequest; /** - * Decodes a RollbackPreparedRequest message from the specified reader or buffer, length delimited. + * Decodes a CommitRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RollbackPreparedRequest + * @returns CommitRequest * @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)): query.RollbackPreparedRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CommitRequest; /** - * Verifies a RollbackPreparedRequest message. + * Verifies a CommitRequest 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 RollbackPreparedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CommitRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RollbackPreparedRequest + * @returns CommitRequest */ - public static fromObject(object: { [k: string]: any }): query.RollbackPreparedRequest; + public static fromObject(object: { [k: string]: any }): query.CommitRequest; /** - * Creates a plain object from a RollbackPreparedRequest message. Also converts values to other types if specified. - * @param message RollbackPreparedRequest + * Creates a plain object from a CommitRequest message. Also converts values to other types if specified. + * @param message CommitRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.RollbackPreparedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CommitRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RollbackPreparedRequest to JSON. + * Converts this CommitRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RollbackPreparedRequest + * Gets the default type url for CommitRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RollbackPreparedResponse. */ - interface IRollbackPreparedResponse { + /** Properties of a CommitResponse. */ + interface ICommitResponse { + + /** CommitResponse reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a RollbackPreparedResponse. */ - class RollbackPreparedResponse implements IRollbackPreparedResponse { + /** Represents a CommitResponse. */ + class CommitResponse implements ICommitResponse { /** - * Constructs a new RollbackPreparedResponse. + * Constructs a new CommitResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IRollbackPreparedResponse); + constructor(properties?: query.ICommitResponse); + + /** CommitResponse reserved_id. */ + public reserved_id: (number|Long); /** - * Creates a new RollbackPreparedResponse instance using the specified properties. + * Creates a new CommitResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RollbackPreparedResponse instance + * @returns CommitResponse instance */ - public static create(properties?: query.IRollbackPreparedResponse): query.RollbackPreparedResponse; + public static create(properties?: query.ICommitResponse): query.CommitResponse; /** - * Encodes the specified RollbackPreparedResponse message. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. - * @param message RollbackPreparedResponse message or plain object to encode + * Encodes the specified CommitResponse message. Does not implicitly {@link query.CommitResponse.verify|verify} messages. + * @param message CommitResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRollbackPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RollbackPreparedResponse message, length delimited. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. - * @param message RollbackPreparedResponse message or plain object to encode + * Encodes the specified CommitResponse message, length delimited. Does not implicitly {@link query.CommitResponse.verify|verify} messages. + * @param message CommitResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRollbackPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RollbackPreparedResponse message from the specified reader or buffer. + * Decodes a CommitResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RollbackPreparedResponse + * @returns CommitResponse * @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): query.RollbackPreparedResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CommitResponse; /** - * Decodes a RollbackPreparedResponse message from the specified reader or buffer, length delimited. + * Decodes a CommitResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RollbackPreparedResponse + * @returns CommitResponse * @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)): query.RollbackPreparedResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CommitResponse; /** - * Verifies a RollbackPreparedResponse message. + * Verifies a CommitResponse 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 RollbackPreparedResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CommitResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RollbackPreparedResponse + * @returns CommitResponse */ - public static fromObject(object: { [k: string]: any }): query.RollbackPreparedResponse; + public static fromObject(object: { [k: string]: any }): query.CommitResponse; /** - * Creates a plain object from a RollbackPreparedResponse message. Also converts values to other types if specified. - * @param message RollbackPreparedResponse + * Creates a plain object from a CommitResponse message. Also converts values to other types if specified. + * @param message CommitResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.RollbackPreparedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CommitResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RollbackPreparedResponse to JSON. + * Converts this CommitResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RollbackPreparedResponse + * Gets the default type url for CommitResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateTransactionRequest. */ - interface ICreateTransactionRequest { + /** Properties of a RollbackRequest. */ + interface IRollbackRequest { - /** CreateTransactionRequest effective_caller_id */ + /** RollbackRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** CreateTransactionRequest immediate_caller_id */ + /** RollbackRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** CreateTransactionRequest target */ + /** RollbackRequest target */ target?: (query.ITarget|null); - /** CreateTransactionRequest dtid */ - dtid?: (string|null); - - /** CreateTransactionRequest participants */ - participants?: (query.ITarget[]|null); + /** RollbackRequest transaction_id */ + transaction_id?: (number|Long|null); } - /** Represents a CreateTransactionRequest. */ - class CreateTransactionRequest implements ICreateTransactionRequest { + /** Represents a RollbackRequest. */ + class RollbackRequest implements IRollbackRequest { /** - * Constructs a new CreateTransactionRequest. + * Constructs a new RollbackRequest. * @param [properties] Properties to set */ - constructor(properties?: query.ICreateTransactionRequest); + constructor(properties?: query.IRollbackRequest); - /** CreateTransactionRequest effective_caller_id. */ + /** RollbackRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** CreateTransactionRequest immediate_caller_id. */ + /** RollbackRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** CreateTransactionRequest target. */ + /** RollbackRequest target. */ public target?: (query.ITarget|null); - /** CreateTransactionRequest dtid. */ - public dtid: string; - - /** CreateTransactionRequest participants. */ - public participants: query.ITarget[]; + /** RollbackRequest transaction_id. */ + public transaction_id: (number|Long); /** - * Creates a new CreateTransactionRequest instance using the specified properties. + * Creates a new RollbackRequest instance using the specified properties. * @param [properties] Properties to set - * @returns CreateTransactionRequest instance + * @returns RollbackRequest instance */ - public static create(properties?: query.ICreateTransactionRequest): query.CreateTransactionRequest; + public static create(properties?: query.IRollbackRequest): query.RollbackRequest; /** - * Encodes the specified CreateTransactionRequest message. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. - * @param message CreateTransactionRequest message or plain object to encode + * Encodes the specified RollbackRequest message. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. + * @param message RollbackRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICreateTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateTransactionRequest message, length delimited. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. - * @param message CreateTransactionRequest message or plain object to encode + * Encodes the specified RollbackRequest message, length delimited. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. + * @param message RollbackRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICreateTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateTransactionRequest message from the specified reader or buffer. + * Decodes a RollbackRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateTransactionRequest + * @returns RollbackRequest * @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): query.CreateTransactionRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.RollbackRequest; /** - * Decodes a CreateTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a RollbackRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateTransactionRequest + * @returns RollbackRequest * @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)): query.CreateTransactionRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.RollbackRequest; /** - * Verifies a CreateTransactionRequest message. + * Verifies a RollbackRequest 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 CreateTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateTransactionRequest + * @returns RollbackRequest */ - public static fromObject(object: { [k: string]: any }): query.CreateTransactionRequest; + public static fromObject(object: { [k: string]: any }): query.RollbackRequest; /** - * Creates a plain object from a CreateTransactionRequest message. Also converts values to other types if specified. - * @param message CreateTransactionRequest + * Creates a plain object from a RollbackRequest message. Also converts values to other types if specified. + * @param message RollbackRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CreateTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.RollbackRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateTransactionRequest to JSON. + * Converts this RollbackRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateTransactionRequest + * Gets the default type url for RollbackRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateTransactionResponse. */ - interface ICreateTransactionResponse { + /** Properties of a RollbackResponse. */ + interface IRollbackResponse { + + /** RollbackResponse reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a CreateTransactionResponse. */ - class CreateTransactionResponse implements ICreateTransactionResponse { + /** Represents a RollbackResponse. */ + class RollbackResponse implements IRollbackResponse { /** - * Constructs a new CreateTransactionResponse. + * Constructs a new RollbackResponse. * @param [properties] Properties to set */ - constructor(properties?: query.ICreateTransactionResponse); + constructor(properties?: query.IRollbackResponse); + + /** RollbackResponse reserved_id. */ + public reserved_id: (number|Long); /** - * Creates a new CreateTransactionResponse instance using the specified properties. + * Creates a new RollbackResponse instance using the specified properties. * @param [properties] Properties to set - * @returns CreateTransactionResponse instance + * @returns RollbackResponse instance */ - public static create(properties?: query.ICreateTransactionResponse): query.CreateTransactionResponse; + public static create(properties?: query.IRollbackResponse): query.RollbackResponse; /** - * Encodes the specified CreateTransactionResponse message. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. - * @param message CreateTransactionResponse message or plain object to encode + * Encodes the specified RollbackResponse message. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. + * @param message RollbackResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ICreateTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateTransactionResponse message, length delimited. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. - * @param message CreateTransactionResponse message or plain object to encode + * Encodes the specified RollbackResponse message, length delimited. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. + * @param message RollbackResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ICreateTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateTransactionResponse message from the specified reader or buffer. + * Decodes a RollbackResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateTransactionResponse + * @returns RollbackResponse * @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): query.CreateTransactionResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.RollbackResponse; /** - * Decodes a CreateTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a RollbackResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateTransactionResponse + * @returns RollbackResponse * @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)): query.CreateTransactionResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.RollbackResponse; /** - * Verifies a CreateTransactionResponse message. + * Verifies a RollbackResponse 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 CreateTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateTransactionResponse + * @returns RollbackResponse */ - public static fromObject(object: { [k: string]: any }): query.CreateTransactionResponse; + public static fromObject(object: { [k: string]: any }): query.RollbackResponse; /** - * Creates a plain object from a CreateTransactionResponse message. Also converts values to other types if specified. - * @param message CreateTransactionResponse + * Creates a plain object from a RollbackResponse message. Also converts values to other types if specified. + * @param message RollbackResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.CreateTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.RollbackResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateTransactionResponse to JSON. + * Converts this RollbackResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateTransactionResponse + * Gets the default type url for RollbackResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StartCommitRequest. */ - interface IStartCommitRequest { + /** Properties of a PrepareRequest. */ + interface IPrepareRequest { - /** StartCommitRequest effective_caller_id */ + /** PrepareRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** StartCommitRequest immediate_caller_id */ + /** PrepareRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** StartCommitRequest target */ + /** PrepareRequest target */ target?: (query.ITarget|null); - /** StartCommitRequest transaction_id */ + /** PrepareRequest transaction_id */ transaction_id?: (number|Long|null); - /** StartCommitRequest dtid */ + /** PrepareRequest dtid */ dtid?: (string|null); } - /** Represents a StartCommitRequest. */ - class StartCommitRequest implements IStartCommitRequest { + /** Represents a PrepareRequest. */ + class PrepareRequest implements IPrepareRequest { /** - * Constructs a new StartCommitRequest. + * Constructs a new PrepareRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IStartCommitRequest); + constructor(properties?: query.IPrepareRequest); - /** StartCommitRequest effective_caller_id. */ + /** PrepareRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** StartCommitRequest immediate_caller_id. */ + /** PrepareRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** StartCommitRequest target. */ + /** PrepareRequest target. */ public target?: (query.ITarget|null); - /** StartCommitRequest transaction_id. */ + /** PrepareRequest transaction_id. */ public transaction_id: (number|Long); - /** StartCommitRequest dtid. */ + /** PrepareRequest dtid. */ public dtid: string; /** - * Creates a new StartCommitRequest instance using the specified properties. + * Creates a new PrepareRequest instance using the specified properties. * @param [properties] Properties to set - * @returns StartCommitRequest instance + * @returns PrepareRequest instance */ - public static create(properties?: query.IStartCommitRequest): query.StartCommitRequest; + public static create(properties?: query.IPrepareRequest): query.PrepareRequest; /** - * Encodes the specified StartCommitRequest message. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. - * @param message StartCommitRequest message or plain object to encode + * Encodes the specified PrepareRequest message. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. + * @param message PrepareRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStartCommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IPrepareRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StartCommitRequest message, length delimited. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. - * @param message StartCommitRequest message or plain object to encode + * Encodes the specified PrepareRequest message, length delimited. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. + * @param message PrepareRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStartCommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IPrepareRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StartCommitRequest message from the specified reader or buffer. + * Decodes a PrepareRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StartCommitRequest + * @returns PrepareRequest * @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): query.StartCommitRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.PrepareRequest; /** - * Decodes a StartCommitRequest message from the specified reader or buffer, length delimited. + * Decodes a PrepareRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StartCommitRequest + * @returns PrepareRequest * @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)): query.StartCommitRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.PrepareRequest; /** - * Verifies a StartCommitRequest message. + * Verifies a PrepareRequest 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 StartCommitRequest message from a plain object. Also converts values to their respective internal types. + * Creates a PrepareRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StartCommitRequest + * @returns PrepareRequest */ - public static fromObject(object: { [k: string]: any }): query.StartCommitRequest; + public static fromObject(object: { [k: string]: any }): query.PrepareRequest; /** - * Creates a plain object from a StartCommitRequest message. Also converts values to other types if specified. - * @param message StartCommitRequest + * Creates a plain object from a PrepareRequest message. Also converts values to other types if specified. + * @param message PrepareRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StartCommitRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.PrepareRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StartCommitRequest to JSON. + * Converts this PrepareRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StartCommitRequest + * Gets the default type url for PrepareRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StartCommitResponse. */ - interface IStartCommitResponse { + /** Properties of a PrepareResponse. */ + interface IPrepareResponse { } - /** Represents a StartCommitResponse. */ - class StartCommitResponse implements IStartCommitResponse { + /** Represents a PrepareResponse. */ + class PrepareResponse implements IPrepareResponse { /** - * Constructs a new StartCommitResponse. + * Constructs a new PrepareResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IStartCommitResponse); + constructor(properties?: query.IPrepareResponse); /** - * Creates a new StartCommitResponse instance using the specified properties. + * Creates a new PrepareResponse instance using the specified properties. * @param [properties] Properties to set - * @returns StartCommitResponse instance + * @returns PrepareResponse instance */ - public static create(properties?: query.IStartCommitResponse): query.StartCommitResponse; + public static create(properties?: query.IPrepareResponse): query.PrepareResponse; /** - * Encodes the specified StartCommitResponse message. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. - * @param message StartCommitResponse message or plain object to encode + * Encodes the specified PrepareResponse message. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. + * @param message PrepareResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStartCommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IPrepareResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StartCommitResponse message, length delimited. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. - * @param message StartCommitResponse message or plain object to encode + * Encodes the specified PrepareResponse message, length delimited. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. + * @param message PrepareResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStartCommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IPrepareResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StartCommitResponse message from the specified reader or buffer. + * Decodes a PrepareResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StartCommitResponse + * @returns PrepareResponse * @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): query.StartCommitResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.PrepareResponse; /** - * Decodes a StartCommitResponse message from the specified reader or buffer, length delimited. + * Decodes a PrepareResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StartCommitResponse + * @returns PrepareResponse * @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)): query.StartCommitResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.PrepareResponse; /** - * Verifies a StartCommitResponse message. + * Verifies a PrepareResponse 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 StartCommitResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PrepareResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StartCommitResponse + * @returns PrepareResponse */ - public static fromObject(object: { [k: string]: any }): query.StartCommitResponse; + public static fromObject(object: { [k: string]: any }): query.PrepareResponse; /** - * Creates a plain object from a StartCommitResponse message. Also converts values to other types if specified. - * @param message StartCommitResponse + * Creates a plain object from a PrepareResponse message. Also converts values to other types if specified. + * @param message PrepareResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StartCommitResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.PrepareResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StartCommitResponse to JSON. + * Converts this PrepareResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StartCommitResponse + * Gets the default type url for PrepareResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetRollbackRequest. */ - interface ISetRollbackRequest { + /** Properties of a CommitPreparedRequest. */ + interface ICommitPreparedRequest { - /** SetRollbackRequest effective_caller_id */ + /** CommitPreparedRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** SetRollbackRequest immediate_caller_id */ + /** CommitPreparedRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** SetRollbackRequest target */ + /** CommitPreparedRequest target */ target?: (query.ITarget|null); - /** SetRollbackRequest transaction_id */ - transaction_id?: (number|Long|null); - - /** SetRollbackRequest dtid */ + /** CommitPreparedRequest dtid */ dtid?: (string|null); } - /** Represents a SetRollbackRequest. */ - class SetRollbackRequest implements ISetRollbackRequest { + /** Represents a CommitPreparedRequest. */ + class CommitPreparedRequest implements ICommitPreparedRequest { /** - * Constructs a new SetRollbackRequest. + * Constructs a new CommitPreparedRequest. * @param [properties] Properties to set */ - constructor(properties?: query.ISetRollbackRequest); + constructor(properties?: query.ICommitPreparedRequest); - /** SetRollbackRequest effective_caller_id. */ + /** CommitPreparedRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** SetRollbackRequest immediate_caller_id. */ + /** CommitPreparedRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** SetRollbackRequest target. */ + /** CommitPreparedRequest target. */ public target?: (query.ITarget|null); - /** SetRollbackRequest transaction_id. */ - public transaction_id: (number|Long); - - /** SetRollbackRequest dtid. */ + /** CommitPreparedRequest dtid. */ public dtid: string; /** - * Creates a new SetRollbackRequest instance using the specified properties. + * Creates a new CommitPreparedRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetRollbackRequest instance + * @returns CommitPreparedRequest instance */ - public static create(properties?: query.ISetRollbackRequest): query.SetRollbackRequest; + public static create(properties?: query.ICommitPreparedRequest): query.CommitPreparedRequest; /** - * Encodes the specified SetRollbackRequest message. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. - * @param message SetRollbackRequest message or plain object to encode + * Encodes the specified CommitPreparedRequest message. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. + * @param message CommitPreparedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ISetRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICommitPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetRollbackRequest message, length delimited. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. - * @param message SetRollbackRequest message or plain object to encode + * Encodes the specified CommitPreparedRequest message, length delimited. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. + * @param message CommitPreparedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ISetRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICommitPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetRollbackRequest message from the specified reader or buffer. + * Decodes a CommitPreparedRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetRollbackRequest + * @returns CommitPreparedRequest * @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): query.SetRollbackRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CommitPreparedRequest; /** - * Decodes a SetRollbackRequest message from the specified reader or buffer, length delimited. + * Decodes a CommitPreparedRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetRollbackRequest + * @returns CommitPreparedRequest * @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)): query.SetRollbackRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CommitPreparedRequest; /** - * Verifies a SetRollbackRequest message. + * Verifies a CommitPreparedRequest 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 SetRollbackRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CommitPreparedRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetRollbackRequest + * @returns CommitPreparedRequest */ - public static fromObject(object: { [k: string]: any }): query.SetRollbackRequest; + public static fromObject(object: { [k: string]: any }): query.CommitPreparedRequest; /** - * Creates a plain object from a SetRollbackRequest message. Also converts values to other types if specified. - * @param message SetRollbackRequest + * Creates a plain object from a CommitPreparedRequest message. Also converts values to other types if specified. + * @param message CommitPreparedRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.SetRollbackRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CommitPreparedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetRollbackRequest to JSON. + * Converts this CommitPreparedRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetRollbackRequest + * Gets the default type url for CommitPreparedRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetRollbackResponse. */ - interface ISetRollbackResponse { + /** Properties of a CommitPreparedResponse. */ + interface ICommitPreparedResponse { } - /** Represents a SetRollbackResponse. */ - class SetRollbackResponse implements ISetRollbackResponse { + /** Represents a CommitPreparedResponse. */ + class CommitPreparedResponse implements ICommitPreparedResponse { /** - * Constructs a new SetRollbackResponse. + * Constructs a new CommitPreparedResponse. * @param [properties] Properties to set */ - constructor(properties?: query.ISetRollbackResponse); + constructor(properties?: query.ICommitPreparedResponse); /** - * Creates a new SetRollbackResponse instance using the specified properties. + * Creates a new CommitPreparedResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetRollbackResponse instance + * @returns CommitPreparedResponse instance */ - public static create(properties?: query.ISetRollbackResponse): query.SetRollbackResponse; + public static create(properties?: query.ICommitPreparedResponse): query.CommitPreparedResponse; /** - * Encodes the specified SetRollbackResponse message. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. - * @param message SetRollbackResponse message or plain object to encode + * Encodes the specified CommitPreparedResponse message. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. + * @param message CommitPreparedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ISetRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICommitPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetRollbackResponse message, length delimited. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. - * @param message SetRollbackResponse message or plain object to encode + * Encodes the specified CommitPreparedResponse message, length delimited. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. + * @param message CommitPreparedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ISetRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICommitPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetRollbackResponse message from the specified reader or buffer. + * Decodes a CommitPreparedResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetRollbackResponse + * @returns CommitPreparedResponse * @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): query.SetRollbackResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CommitPreparedResponse; /** - * Decodes a SetRollbackResponse message from the specified reader or buffer, length delimited. + * Decodes a CommitPreparedResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetRollbackResponse + * @returns CommitPreparedResponse * @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)): query.SetRollbackResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CommitPreparedResponse; /** - * Verifies a SetRollbackResponse message. + * Verifies a CommitPreparedResponse 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 SetRollbackResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CommitPreparedResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetRollbackResponse + * @returns CommitPreparedResponse */ - public static fromObject(object: { [k: string]: any }): query.SetRollbackResponse; + public static fromObject(object: { [k: string]: any }): query.CommitPreparedResponse; /** - * Creates a plain object from a SetRollbackResponse message. Also converts values to other types if specified. - * @param message SetRollbackResponse + * Creates a plain object from a CommitPreparedResponse message. Also converts values to other types if specified. + * @param message CommitPreparedResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.SetRollbackResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CommitPreparedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetRollbackResponse to JSON. + * Converts this CommitPreparedResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetRollbackResponse + * Gets the default type url for CommitPreparedResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ConcludeTransactionRequest. */ - interface IConcludeTransactionRequest { + /** Properties of a RollbackPreparedRequest. */ + interface IRollbackPreparedRequest { - /** ConcludeTransactionRequest effective_caller_id */ + /** RollbackPreparedRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ConcludeTransactionRequest immediate_caller_id */ + /** RollbackPreparedRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ConcludeTransactionRequest target */ + /** RollbackPreparedRequest target */ target?: (query.ITarget|null); - /** ConcludeTransactionRequest dtid */ + /** RollbackPreparedRequest transaction_id */ + transaction_id?: (number|Long|null); + + /** RollbackPreparedRequest dtid */ dtid?: (string|null); } - /** Represents a ConcludeTransactionRequest. */ - class ConcludeTransactionRequest implements IConcludeTransactionRequest { + /** Represents a RollbackPreparedRequest. */ + class RollbackPreparedRequest implements IRollbackPreparedRequest { /** - * Constructs a new ConcludeTransactionRequest. + * Constructs a new RollbackPreparedRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IConcludeTransactionRequest); + constructor(properties?: query.IRollbackPreparedRequest); - /** ConcludeTransactionRequest effective_caller_id. */ + /** RollbackPreparedRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ConcludeTransactionRequest immediate_caller_id. */ + /** RollbackPreparedRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ConcludeTransactionRequest target. */ + /** RollbackPreparedRequest target. */ public target?: (query.ITarget|null); - /** ConcludeTransactionRequest dtid. */ + /** RollbackPreparedRequest transaction_id. */ + public transaction_id: (number|Long); + + /** RollbackPreparedRequest dtid. */ public dtid: string; /** - * Creates a new ConcludeTransactionRequest instance using the specified properties. + * Creates a new RollbackPreparedRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ConcludeTransactionRequest instance + * @returns RollbackPreparedRequest instance */ - public static create(properties?: query.IConcludeTransactionRequest): query.ConcludeTransactionRequest; + public static create(properties?: query.IRollbackPreparedRequest): query.RollbackPreparedRequest; /** - * Encodes the specified ConcludeTransactionRequest message. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. - * @param message ConcludeTransactionRequest message or plain object to encode + * Encodes the specified RollbackPreparedRequest message. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. + * @param message RollbackPreparedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IConcludeTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRollbackPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ConcludeTransactionRequest message, length delimited. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. - * @param message ConcludeTransactionRequest message or plain object to encode + * Encodes the specified RollbackPreparedRequest message, length delimited. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. + * @param message RollbackPreparedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IConcludeTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRollbackPreparedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ConcludeTransactionRequest message from the specified reader or buffer. + * Decodes a RollbackPreparedRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ConcludeTransactionRequest + * @returns RollbackPreparedRequest * @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): query.ConcludeTransactionRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.RollbackPreparedRequest; /** - * Decodes a ConcludeTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a RollbackPreparedRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ConcludeTransactionRequest + * @returns RollbackPreparedRequest * @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)): query.ConcludeTransactionRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.RollbackPreparedRequest; /** - * Verifies a ConcludeTransactionRequest message. + * Verifies a RollbackPreparedRequest 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 ConcludeTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackPreparedRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ConcludeTransactionRequest + * @returns RollbackPreparedRequest */ - public static fromObject(object: { [k: string]: any }): query.ConcludeTransactionRequest; + public static fromObject(object: { [k: string]: any }): query.RollbackPreparedRequest; /** - * Creates a plain object from a ConcludeTransactionRequest message. Also converts values to other types if specified. - * @param message ConcludeTransactionRequest + * Creates a plain object from a RollbackPreparedRequest message. Also converts values to other types if specified. + * @param message RollbackPreparedRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ConcludeTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.RollbackPreparedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ConcludeTransactionRequest to JSON. + * Converts this RollbackPreparedRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ConcludeTransactionRequest + * Gets the default type url for RollbackPreparedRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ConcludeTransactionResponse. */ - interface IConcludeTransactionResponse { + /** Properties of a RollbackPreparedResponse. */ + interface IRollbackPreparedResponse { } - /** Represents a ConcludeTransactionResponse. */ - class ConcludeTransactionResponse implements IConcludeTransactionResponse { + /** Represents a RollbackPreparedResponse. */ + class RollbackPreparedResponse implements IRollbackPreparedResponse { /** - * Constructs a new ConcludeTransactionResponse. + * Constructs a new RollbackPreparedResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IConcludeTransactionResponse); + constructor(properties?: query.IRollbackPreparedResponse); /** - * Creates a new ConcludeTransactionResponse instance using the specified properties. + * Creates a new RollbackPreparedResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ConcludeTransactionResponse instance + * @returns RollbackPreparedResponse instance */ - public static create(properties?: query.IConcludeTransactionResponse): query.ConcludeTransactionResponse; + public static create(properties?: query.IRollbackPreparedResponse): query.RollbackPreparedResponse; /** - * Encodes the specified ConcludeTransactionResponse message. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. - * @param message ConcludeTransactionResponse message or plain object to encode + * Encodes the specified RollbackPreparedResponse message. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. + * @param message RollbackPreparedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IConcludeTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRollbackPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ConcludeTransactionResponse message, length delimited. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. - * @param message ConcludeTransactionResponse message or plain object to encode + * Encodes the specified RollbackPreparedResponse message, length delimited. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. + * @param message RollbackPreparedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IConcludeTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRollbackPreparedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ConcludeTransactionResponse message from the specified reader or buffer. + * Decodes a RollbackPreparedResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ConcludeTransactionResponse + * @returns RollbackPreparedResponse * @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): query.ConcludeTransactionResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.RollbackPreparedResponse; /** - * Decodes a ConcludeTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a RollbackPreparedResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ConcludeTransactionResponse + * @returns RollbackPreparedResponse * @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)): query.ConcludeTransactionResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.RollbackPreparedResponse; /** - * Verifies a ConcludeTransactionResponse message. + * Verifies a RollbackPreparedResponse 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 ConcludeTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackPreparedResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ConcludeTransactionResponse + * @returns RollbackPreparedResponse */ - public static fromObject(object: { [k: string]: any }): query.ConcludeTransactionResponse; + public static fromObject(object: { [k: string]: any }): query.RollbackPreparedResponse; /** - * Creates a plain object from a ConcludeTransactionResponse message. Also converts values to other types if specified. - * @param message ConcludeTransactionResponse + * Creates a plain object from a RollbackPreparedResponse message. Also converts values to other types if specified. + * @param message RollbackPreparedResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ConcludeTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.RollbackPreparedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ConcludeTransactionResponse to JSON. + * Converts this RollbackPreparedResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ConcludeTransactionResponse + * Gets the default type url for RollbackPreparedResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReadTransactionRequest. */ - interface IReadTransactionRequest { + /** Properties of a CreateTransactionRequest. */ + interface ICreateTransactionRequest { - /** ReadTransactionRequest effective_caller_id */ + /** CreateTransactionRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReadTransactionRequest immediate_caller_id */ + /** CreateTransactionRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReadTransactionRequest target */ + /** CreateTransactionRequest target */ target?: (query.ITarget|null); - /** ReadTransactionRequest dtid */ + /** CreateTransactionRequest dtid */ dtid?: (string|null); + + /** CreateTransactionRequest participants */ + participants?: (query.ITarget[]|null); } - /** Represents a ReadTransactionRequest. */ - class ReadTransactionRequest implements IReadTransactionRequest { + /** Represents a CreateTransactionRequest. */ + class CreateTransactionRequest implements ICreateTransactionRequest { /** - * Constructs a new ReadTransactionRequest. + * Constructs a new CreateTransactionRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReadTransactionRequest); + constructor(properties?: query.ICreateTransactionRequest); - /** ReadTransactionRequest effective_caller_id. */ + /** CreateTransactionRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReadTransactionRequest immediate_caller_id. */ + /** CreateTransactionRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReadTransactionRequest target. */ + /** CreateTransactionRequest target. */ public target?: (query.ITarget|null); - /** ReadTransactionRequest dtid. */ + /** CreateTransactionRequest dtid. */ public dtid: string; + /** CreateTransactionRequest participants. */ + public participants: query.ITarget[]; + /** - * Creates a new ReadTransactionRequest instance using the specified properties. + * Creates a new CreateTransactionRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReadTransactionRequest instance + * @returns CreateTransactionRequest instance */ - public static create(properties?: query.IReadTransactionRequest): query.ReadTransactionRequest; + public static create(properties?: query.ICreateTransactionRequest): query.CreateTransactionRequest; /** - * Encodes the specified ReadTransactionRequest message. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. - * @param message ReadTransactionRequest message or plain object to encode + * Encodes the specified CreateTransactionRequest message. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. + * @param message CreateTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReadTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICreateTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReadTransactionRequest message, length delimited. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. - * @param message ReadTransactionRequest message or plain object to encode + * Encodes the specified CreateTransactionRequest message, length delimited. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. + * @param message CreateTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReadTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICreateTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReadTransactionRequest message from the specified reader or buffer. + * Decodes a CreateTransactionRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReadTransactionRequest + * @returns CreateTransactionRequest * @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): query.ReadTransactionRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CreateTransactionRequest; /** - * Decodes a ReadTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateTransactionRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReadTransactionRequest + * @returns CreateTransactionRequest * @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)): query.ReadTransactionRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CreateTransactionRequest; /** - * Verifies a ReadTransactionRequest message. + * Verifies a CreateTransactionRequest 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 ReadTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateTransactionRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReadTransactionRequest + * @returns CreateTransactionRequest */ - public static fromObject(object: { [k: string]: any }): query.ReadTransactionRequest; + public static fromObject(object: { [k: string]: any }): query.CreateTransactionRequest; /** - * Creates a plain object from a ReadTransactionRequest message. Also converts values to other types if specified. - * @param message ReadTransactionRequest + * Creates a plain object from a CreateTransactionRequest message. Also converts values to other types if specified. + * @param message CreateTransactionRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReadTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CreateTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReadTransactionRequest to JSON. + * Converts this CreateTransactionRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReadTransactionRequest + * Gets the default type url for CreateTransactionRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReadTransactionResponse. */ - interface IReadTransactionResponse { - - /** ReadTransactionResponse metadata */ - metadata?: (query.ITransactionMetadata|null); + /** Properties of a CreateTransactionResponse. */ + interface ICreateTransactionResponse { } - /** Represents a ReadTransactionResponse. */ - class ReadTransactionResponse implements IReadTransactionResponse { + /** Represents a CreateTransactionResponse. */ + class CreateTransactionResponse implements ICreateTransactionResponse { /** - * Constructs a new ReadTransactionResponse. + * Constructs a new CreateTransactionResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReadTransactionResponse); - - /** ReadTransactionResponse metadata. */ - public metadata?: (query.ITransactionMetadata|null); + constructor(properties?: query.ICreateTransactionResponse); /** - * Creates a new ReadTransactionResponse instance using the specified properties. + * Creates a new CreateTransactionResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReadTransactionResponse instance + * @returns CreateTransactionResponse instance */ - public static create(properties?: query.IReadTransactionResponse): query.ReadTransactionResponse; + public static create(properties?: query.ICreateTransactionResponse): query.CreateTransactionResponse; /** - * Encodes the specified ReadTransactionResponse message. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. - * @param message ReadTransactionResponse message or plain object to encode + * Encodes the specified CreateTransactionResponse message. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. + * @param message CreateTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReadTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ICreateTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReadTransactionResponse message, length delimited. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. - * @param message ReadTransactionResponse message or plain object to encode + * Encodes the specified CreateTransactionResponse message, length delimited. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. + * @param message CreateTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReadTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ICreateTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReadTransactionResponse message from the specified reader or buffer. + * Decodes a CreateTransactionResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReadTransactionResponse + * @returns CreateTransactionResponse * @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): query.ReadTransactionResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.CreateTransactionResponse; /** - * Decodes a ReadTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateTransactionResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReadTransactionResponse + * @returns CreateTransactionResponse * @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)): query.ReadTransactionResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.CreateTransactionResponse; /** - * Verifies a ReadTransactionResponse message. + * Verifies a CreateTransactionResponse 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 ReadTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateTransactionResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReadTransactionResponse + * @returns CreateTransactionResponse */ - public static fromObject(object: { [k: string]: any }): query.ReadTransactionResponse; + public static fromObject(object: { [k: string]: any }): query.CreateTransactionResponse; /** - * Creates a plain object from a ReadTransactionResponse message. Also converts values to other types if specified. - * @param message ReadTransactionResponse + * Creates a plain object from a CreateTransactionResponse message. Also converts values to other types if specified. + * @param message CreateTransactionResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReadTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.CreateTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReadTransactionResponse to JSON. + * Converts this CreateTransactionResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReadTransactionResponse + * Gets the default type url for CreateTransactionResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginExecuteRequest. */ - interface IBeginExecuteRequest { + /** Properties of a StartCommitRequest. */ + interface IStartCommitRequest { - /** BeginExecuteRequest effective_caller_id */ + /** StartCommitRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** BeginExecuteRequest immediate_caller_id */ + /** StartCommitRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** BeginExecuteRequest target */ + /** StartCommitRequest target */ target?: (query.ITarget|null); - /** BeginExecuteRequest query */ - query?: (query.IBoundQuery|null); - - /** BeginExecuteRequest options */ - options?: (query.IExecuteOptions|null); - - /** BeginExecuteRequest reserved_id */ - reserved_id?: (number|Long|null); + /** StartCommitRequest transaction_id */ + transaction_id?: (number|Long|null); - /** BeginExecuteRequest pre_queries */ - pre_queries?: (string[]|null); + /** StartCommitRequest dtid */ + dtid?: (string|null); } - /** Represents a BeginExecuteRequest. */ - class BeginExecuteRequest implements IBeginExecuteRequest { + /** Represents a StartCommitRequest. */ + class StartCommitRequest implements IStartCommitRequest { /** - * Constructs a new BeginExecuteRequest. + * Constructs a new StartCommitRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginExecuteRequest); + constructor(properties?: query.IStartCommitRequest); - /** BeginExecuteRequest effective_caller_id. */ + /** StartCommitRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** BeginExecuteRequest immediate_caller_id. */ + /** StartCommitRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** BeginExecuteRequest target. */ + /** StartCommitRequest target. */ public target?: (query.ITarget|null); - /** BeginExecuteRequest query. */ - public query?: (query.IBoundQuery|null); - - /** BeginExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); - - /** BeginExecuteRequest reserved_id. */ - public reserved_id: (number|Long); + /** StartCommitRequest transaction_id. */ + public transaction_id: (number|Long); - /** BeginExecuteRequest pre_queries. */ - public pre_queries: string[]; + /** StartCommitRequest dtid. */ + public dtid: string; /** - * Creates a new BeginExecuteRequest instance using the specified properties. + * Creates a new StartCommitRequest instance using the specified properties. * @param [properties] Properties to set - * @returns BeginExecuteRequest instance + * @returns StartCommitRequest instance */ - public static create(properties?: query.IBeginExecuteRequest): query.BeginExecuteRequest; + public static create(properties?: query.IStartCommitRequest): query.StartCommitRequest; /** - * Encodes the specified BeginExecuteRequest message. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. - * @param message BeginExecuteRequest message or plain object to encode + * Encodes the specified StartCommitRequest message. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. + * @param message StartCommitRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IStartCommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginExecuteRequest message, length delimited. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. - * @param message BeginExecuteRequest message or plain object to encode + * Encodes the specified StartCommitRequest message, length delimited. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. + * @param message StartCommitRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IStartCommitRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginExecuteRequest message from the specified reader or buffer. + * Decodes a StartCommitRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginExecuteRequest + * @returns StartCommitRequest * @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): query.BeginExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.StartCommitRequest; /** - * Decodes a BeginExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a StartCommitRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginExecuteRequest + * @returns StartCommitRequest * @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)): query.BeginExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.StartCommitRequest; /** - * Verifies a BeginExecuteRequest message. + * Verifies a StartCommitRequest 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 BeginExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StartCommitRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginExecuteRequest + * @returns StartCommitRequest */ - public static fromObject(object: { [k: string]: any }): query.BeginExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.StartCommitRequest; /** - * Creates a plain object from a BeginExecuteRequest message. Also converts values to other types if specified. - * @param message BeginExecuteRequest + * Creates a plain object from a StartCommitRequest message. Also converts values to other types if specified. + * @param message StartCommitRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.StartCommitRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginExecuteRequest to JSON. + * Converts this StartCommitRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginExecuteRequest + * Gets the default type url for StartCommitRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginExecuteResponse. */ - interface IBeginExecuteResponse { - - /** BeginExecuteResponse error */ - error?: (vtrpc.IRPCError|null); - - /** BeginExecuteResponse result */ - result?: (query.IQueryResult|null); - - /** BeginExecuteResponse transaction_id */ - transaction_id?: (number|Long|null); - - /** BeginExecuteResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** BeginExecuteResponse session_state_changes */ - session_state_changes?: (string|null); + /** Properties of a StartCommitResponse. */ + interface IStartCommitResponse { } - /** Represents a BeginExecuteResponse. */ - class BeginExecuteResponse implements IBeginExecuteResponse { + /** Represents a StartCommitResponse. */ + class StartCommitResponse implements IStartCommitResponse { /** - * Constructs a new BeginExecuteResponse. + * Constructs a new StartCommitResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginExecuteResponse); - - /** BeginExecuteResponse error. */ - public error?: (vtrpc.IRPCError|null); - - /** BeginExecuteResponse result. */ - public result?: (query.IQueryResult|null); - - /** BeginExecuteResponse transaction_id. */ - public transaction_id: (number|Long); - - /** BeginExecuteResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** BeginExecuteResponse session_state_changes. */ - public session_state_changes: string; + constructor(properties?: query.IStartCommitResponse); /** - * Creates a new BeginExecuteResponse instance using the specified properties. + * Creates a new StartCommitResponse instance using the specified properties. * @param [properties] Properties to set - * @returns BeginExecuteResponse instance + * @returns StartCommitResponse instance */ - public static create(properties?: query.IBeginExecuteResponse): query.BeginExecuteResponse; + public static create(properties?: query.IStartCommitResponse): query.StartCommitResponse; /** - * Encodes the specified BeginExecuteResponse message. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. - * @param message BeginExecuteResponse message or plain object to encode + * Encodes the specified StartCommitResponse message. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. + * @param message StartCommitResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IStartCommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginExecuteResponse message, length delimited. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. - * @param message BeginExecuteResponse message or plain object to encode + * Encodes the specified StartCommitResponse message, length delimited. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. + * @param message StartCommitResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IStartCommitResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginExecuteResponse message from the specified reader or buffer. + * Decodes a StartCommitResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginExecuteResponse + * @returns StartCommitResponse * @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): query.BeginExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.StartCommitResponse; /** - * Decodes a BeginExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a StartCommitResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginExecuteResponse + * @returns StartCommitResponse * @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)): query.BeginExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.StartCommitResponse; /** - * Verifies a BeginExecuteResponse message. + * Verifies a StartCommitResponse 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 BeginExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StartCommitResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginExecuteResponse + * @returns StartCommitResponse */ - public static fromObject(object: { [k: string]: any }): query.BeginExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.StartCommitResponse; /** - * Creates a plain object from a BeginExecuteResponse message. Also converts values to other types if specified. - * @param message BeginExecuteResponse + * Creates a plain object from a StartCommitResponse message. Also converts values to other types if specified. + * @param message StartCommitResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.StartCommitResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginExecuteResponse to JSON. + * Converts this StartCommitResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginExecuteResponse + * Gets the default type url for StartCommitResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginStreamExecuteRequest. */ - interface IBeginStreamExecuteRequest { + /** Properties of a SetRollbackRequest. */ + interface ISetRollbackRequest { - /** BeginStreamExecuteRequest effective_caller_id */ + /** SetRollbackRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** BeginStreamExecuteRequest immediate_caller_id */ + /** SetRollbackRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** BeginStreamExecuteRequest target */ + /** SetRollbackRequest target */ target?: (query.ITarget|null); - /** BeginStreamExecuteRequest query */ - query?: (query.IBoundQuery|null); - - /** BeginStreamExecuteRequest options */ - options?: (query.IExecuteOptions|null); - - /** BeginStreamExecuteRequest pre_queries */ - pre_queries?: (string[]|null); + /** SetRollbackRequest transaction_id */ + transaction_id?: (number|Long|null); - /** BeginStreamExecuteRequest reserved_id */ - reserved_id?: (number|Long|null); + /** SetRollbackRequest dtid */ + dtid?: (string|null); } - /** Represents a BeginStreamExecuteRequest. */ - class BeginStreamExecuteRequest implements IBeginStreamExecuteRequest { + /** Represents a SetRollbackRequest. */ + class SetRollbackRequest implements ISetRollbackRequest { /** - * Constructs a new BeginStreamExecuteRequest. + * Constructs a new SetRollbackRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginStreamExecuteRequest); + constructor(properties?: query.ISetRollbackRequest); - /** BeginStreamExecuteRequest effective_caller_id. */ + /** SetRollbackRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** BeginStreamExecuteRequest immediate_caller_id. */ + /** SetRollbackRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** BeginStreamExecuteRequest target. */ + /** SetRollbackRequest target. */ public target?: (query.ITarget|null); - /** BeginStreamExecuteRequest query. */ - public query?: (query.IBoundQuery|null); - - /** BeginStreamExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); - - /** BeginStreamExecuteRequest pre_queries. */ - public pre_queries: string[]; + /** SetRollbackRequest transaction_id. */ + public transaction_id: (number|Long); - /** BeginStreamExecuteRequest reserved_id. */ - public reserved_id: (number|Long); + /** SetRollbackRequest dtid. */ + public dtid: string; /** - * Creates a new BeginStreamExecuteRequest instance using the specified properties. + * Creates a new SetRollbackRequest instance using the specified properties. * @param [properties] Properties to set - * @returns BeginStreamExecuteRequest instance + * @returns SetRollbackRequest instance */ - public static create(properties?: query.IBeginStreamExecuteRequest): query.BeginStreamExecuteRequest; + public static create(properties?: query.ISetRollbackRequest): query.SetRollbackRequest; /** - * Encodes the specified BeginStreamExecuteRequest message. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. - * @param message BeginStreamExecuteRequest message or plain object to encode + * Encodes the specified SetRollbackRequest message. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. + * @param message SetRollbackRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ISetRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. - * @param message BeginStreamExecuteRequest message or plain object to encode + * Encodes the specified SetRollbackRequest message, length delimited. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. + * @param message SetRollbackRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ISetRollbackRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer. + * Decodes a SetRollbackRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginStreamExecuteRequest + * @returns SetRollbackRequest * @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): query.BeginStreamExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.SetRollbackRequest; /** - * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a SetRollbackRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginStreamExecuteRequest + * @returns SetRollbackRequest * @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)): query.BeginStreamExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.SetRollbackRequest; /** - * Verifies a BeginStreamExecuteRequest message. + * Verifies a SetRollbackRequest 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 BeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetRollbackRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginStreamExecuteRequest + * @returns SetRollbackRequest */ - public static fromObject(object: { [k: string]: any }): query.BeginStreamExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.SetRollbackRequest; /** - * Creates a plain object from a BeginStreamExecuteRequest message. Also converts values to other types if specified. - * @param message BeginStreamExecuteRequest + * Creates a plain object from a SetRollbackRequest message. Also converts values to other types if specified. + * @param message SetRollbackRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.SetRollbackRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginStreamExecuteRequest to JSON. + * Converts this SetRollbackRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginStreamExecuteRequest + * Gets the default type url for SetRollbackRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BeginStreamExecuteResponse. */ - interface IBeginStreamExecuteResponse { - - /** BeginStreamExecuteResponse error */ - error?: (vtrpc.IRPCError|null); - - /** BeginStreamExecuteResponse result */ - result?: (query.IQueryResult|null); - - /** BeginStreamExecuteResponse transaction_id */ - transaction_id?: (number|Long|null); - - /** BeginStreamExecuteResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** BeginStreamExecuteResponse session_state_changes */ - session_state_changes?: (string|null); + /** Properties of a SetRollbackResponse. */ + interface ISetRollbackResponse { } - /** Represents a BeginStreamExecuteResponse. */ - class BeginStreamExecuteResponse implements IBeginStreamExecuteResponse { + /** Represents a SetRollbackResponse. */ + class SetRollbackResponse implements ISetRollbackResponse { /** - * Constructs a new BeginStreamExecuteResponse. + * Constructs a new SetRollbackResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IBeginStreamExecuteResponse); - - /** BeginStreamExecuteResponse error. */ - public error?: (vtrpc.IRPCError|null); - - /** BeginStreamExecuteResponse result. */ - public result?: (query.IQueryResult|null); - - /** BeginStreamExecuteResponse transaction_id. */ - public transaction_id: (number|Long); - - /** BeginStreamExecuteResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** BeginStreamExecuteResponse session_state_changes. */ - public session_state_changes: string; + constructor(properties?: query.ISetRollbackResponse); /** - * Creates a new BeginStreamExecuteResponse instance using the specified properties. + * Creates a new SetRollbackResponse instance using the specified properties. * @param [properties] Properties to set - * @returns BeginStreamExecuteResponse instance + * @returns SetRollbackResponse instance */ - public static create(properties?: query.IBeginStreamExecuteResponse): query.BeginStreamExecuteResponse; + public static create(properties?: query.ISetRollbackResponse): query.SetRollbackResponse; /** - * Encodes the specified BeginStreamExecuteResponse message. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. - * @param message BeginStreamExecuteResponse message or plain object to encode + * Encodes the specified SetRollbackResponse message. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. + * @param message SetRollbackResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.ISetRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. - * @param message BeginStreamExecuteResponse message or plain object to encode + * Encodes the specified SetRollbackResponse message, length delimited. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. + * @param message SetRollbackResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.ISetRollbackResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer. + * Decodes a SetRollbackResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BeginStreamExecuteResponse + * @returns SetRollbackResponse * @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): query.BeginStreamExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.SetRollbackResponse; /** - * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a SetRollbackResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BeginStreamExecuteResponse + * @returns SetRollbackResponse * @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)): query.BeginStreamExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.SetRollbackResponse; /** - * Verifies a BeginStreamExecuteResponse message. + * Verifies a SetRollbackResponse 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 BeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetRollbackResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BeginStreamExecuteResponse + * @returns SetRollbackResponse */ - public static fromObject(object: { [k: string]: any }): query.BeginStreamExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.SetRollbackResponse; /** - * Creates a plain object from a BeginStreamExecuteResponse message. Also converts values to other types if specified. - * @param message BeginStreamExecuteResponse + * Creates a plain object from a SetRollbackResponse message. Also converts values to other types if specified. + * @param message SetRollbackResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.BeginStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.SetRollbackResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BeginStreamExecuteResponse to JSON. + * Converts this SetRollbackResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BeginStreamExecuteResponse + * Gets the default type url for SetRollbackResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MessageStreamRequest. */ - interface IMessageStreamRequest { + /** Properties of a ConcludeTransactionRequest. */ + interface IConcludeTransactionRequest { - /** MessageStreamRequest effective_caller_id */ + /** ConcludeTransactionRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** MessageStreamRequest immediate_caller_id */ + /** ConcludeTransactionRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** MessageStreamRequest target */ + /** ConcludeTransactionRequest target */ target?: (query.ITarget|null); - /** MessageStreamRequest name */ - name?: (string|null); + /** ConcludeTransactionRequest dtid */ + dtid?: (string|null); } - /** Represents a MessageStreamRequest. */ - class MessageStreamRequest implements IMessageStreamRequest { + /** Represents a ConcludeTransactionRequest. */ + class ConcludeTransactionRequest implements IConcludeTransactionRequest { /** - * Constructs a new MessageStreamRequest. + * Constructs a new ConcludeTransactionRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IMessageStreamRequest); + constructor(properties?: query.IConcludeTransactionRequest); - /** MessageStreamRequest effective_caller_id. */ + /** ConcludeTransactionRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** MessageStreamRequest immediate_caller_id. */ + /** ConcludeTransactionRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** MessageStreamRequest target. */ + /** ConcludeTransactionRequest target. */ public target?: (query.ITarget|null); - /** MessageStreamRequest name. */ - public name: string; + /** ConcludeTransactionRequest dtid. */ + public dtid: string; /** - * Creates a new MessageStreamRequest instance using the specified properties. + * Creates a new ConcludeTransactionRequest instance using the specified properties. * @param [properties] Properties to set - * @returns MessageStreamRequest instance + * @returns ConcludeTransactionRequest instance */ - public static create(properties?: query.IMessageStreamRequest): query.MessageStreamRequest; + public static create(properties?: query.IConcludeTransactionRequest): query.ConcludeTransactionRequest; /** - * Encodes the specified MessageStreamRequest message. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. - * @param message MessageStreamRequest message or plain object to encode + * Encodes the specified ConcludeTransactionRequest message. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. + * @param message ConcludeTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IMessageStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IConcludeTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MessageStreamRequest message, length delimited. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. - * @param message MessageStreamRequest message or plain object to encode + * Encodes the specified ConcludeTransactionRequest message, length delimited. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. + * @param message ConcludeTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IMessageStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IConcludeTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MessageStreamRequest message from the specified reader or buffer. + * Decodes a ConcludeTransactionRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MessageStreamRequest + * @returns ConcludeTransactionRequest * @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): query.MessageStreamRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ConcludeTransactionRequest; /** - * Decodes a MessageStreamRequest message from the specified reader or buffer, length delimited. + * Decodes a ConcludeTransactionRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MessageStreamRequest + * @returns ConcludeTransactionRequest * @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)): query.MessageStreamRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ConcludeTransactionRequest; /** - * Verifies a MessageStreamRequest message. + * Verifies a ConcludeTransactionRequest 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 MessageStreamRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ConcludeTransactionRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MessageStreamRequest + * @returns ConcludeTransactionRequest */ - public static fromObject(object: { [k: string]: any }): query.MessageStreamRequest; + public static fromObject(object: { [k: string]: any }): query.ConcludeTransactionRequest; /** - * Creates a plain object from a MessageStreamRequest message. Also converts values to other types if specified. - * @param message MessageStreamRequest + * Creates a plain object from a ConcludeTransactionRequest message. Also converts values to other types if specified. + * @param message ConcludeTransactionRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.MessageStreamRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ConcludeTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MessageStreamRequest to JSON. + * Converts this ConcludeTransactionRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MessageStreamRequest + * Gets the default type url for ConcludeTransactionRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MessageStreamResponse. */ - interface IMessageStreamResponse { - - /** MessageStreamResponse result */ - result?: (query.IQueryResult|null); + /** Properties of a ConcludeTransactionResponse. */ + interface IConcludeTransactionResponse { } - /** Represents a MessageStreamResponse. */ - class MessageStreamResponse implements IMessageStreamResponse { + /** Represents a ConcludeTransactionResponse. */ + class ConcludeTransactionResponse implements IConcludeTransactionResponse { /** - * Constructs a new MessageStreamResponse. + * Constructs a new ConcludeTransactionResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IMessageStreamResponse); - - /** MessageStreamResponse result. */ - public result?: (query.IQueryResult|null); + constructor(properties?: query.IConcludeTransactionResponse); /** - * Creates a new MessageStreamResponse instance using the specified properties. + * Creates a new ConcludeTransactionResponse instance using the specified properties. * @param [properties] Properties to set - * @returns MessageStreamResponse instance + * @returns ConcludeTransactionResponse instance */ - public static create(properties?: query.IMessageStreamResponse): query.MessageStreamResponse; + public static create(properties?: query.IConcludeTransactionResponse): query.ConcludeTransactionResponse; /** - * Encodes the specified MessageStreamResponse message. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. - * @param message MessageStreamResponse message or plain object to encode + * Encodes the specified ConcludeTransactionResponse message. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. + * @param message ConcludeTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IMessageStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IConcludeTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MessageStreamResponse message, length delimited. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. - * @param message MessageStreamResponse message or plain object to encode + * Encodes the specified ConcludeTransactionResponse message, length delimited. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. + * @param message ConcludeTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IMessageStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IConcludeTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MessageStreamResponse message from the specified reader or buffer. + * Decodes a ConcludeTransactionResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MessageStreamResponse + * @returns ConcludeTransactionResponse * @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): query.MessageStreamResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ConcludeTransactionResponse; /** - * Decodes a MessageStreamResponse message from the specified reader or buffer, length delimited. + * Decodes a ConcludeTransactionResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MessageStreamResponse + * @returns ConcludeTransactionResponse * @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)): query.MessageStreamResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ConcludeTransactionResponse; /** - * Verifies a MessageStreamResponse message. + * Verifies a ConcludeTransactionResponse 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 MessageStreamResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ConcludeTransactionResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MessageStreamResponse + * @returns ConcludeTransactionResponse */ - public static fromObject(object: { [k: string]: any }): query.MessageStreamResponse; + public static fromObject(object: { [k: string]: any }): query.ConcludeTransactionResponse; /** - * Creates a plain object from a MessageStreamResponse message. Also converts values to other types if specified. - * @param message MessageStreamResponse + * Creates a plain object from a ConcludeTransactionResponse message. Also converts values to other types if specified. + * @param message ConcludeTransactionResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.MessageStreamResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ConcludeTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MessageStreamResponse to JSON. + * Converts this ConcludeTransactionResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MessageStreamResponse + * Gets the default type url for ConcludeTransactionResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MessageAckRequest. */ - interface IMessageAckRequest { + /** Properties of a ReadTransactionRequest. */ + interface IReadTransactionRequest { - /** MessageAckRequest effective_caller_id */ + /** ReadTransactionRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** MessageAckRequest immediate_caller_id */ + /** ReadTransactionRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** MessageAckRequest target */ + /** ReadTransactionRequest target */ target?: (query.ITarget|null); - /** MessageAckRequest name */ - name?: (string|null); - - /** MessageAckRequest ids */ - ids?: (query.IValue[]|null); + /** ReadTransactionRequest dtid */ + dtid?: (string|null); } - /** Represents a MessageAckRequest. */ - class MessageAckRequest implements IMessageAckRequest { + /** Represents a ReadTransactionRequest. */ + class ReadTransactionRequest implements IReadTransactionRequest { /** - * Constructs a new MessageAckRequest. + * Constructs a new ReadTransactionRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IMessageAckRequest); + constructor(properties?: query.IReadTransactionRequest); - /** MessageAckRequest effective_caller_id. */ + /** ReadTransactionRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** MessageAckRequest immediate_caller_id. */ + /** ReadTransactionRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** MessageAckRequest target. */ + /** ReadTransactionRequest target. */ public target?: (query.ITarget|null); - /** MessageAckRequest name. */ - public name: string; - - /** MessageAckRequest ids. */ - public ids: query.IValue[]; + /** ReadTransactionRequest dtid. */ + public dtid: string; /** - * Creates a new MessageAckRequest instance using the specified properties. + * Creates a new ReadTransactionRequest instance using the specified properties. * @param [properties] Properties to set - * @returns MessageAckRequest instance + * @returns ReadTransactionRequest instance */ - public static create(properties?: query.IMessageAckRequest): query.MessageAckRequest; + public static create(properties?: query.IReadTransactionRequest): query.ReadTransactionRequest; /** - * Encodes the specified MessageAckRequest message. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. - * @param message MessageAckRequest message or plain object to encode + * Encodes the specified ReadTransactionRequest message. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. + * @param message ReadTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IMessageAckRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReadTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MessageAckRequest message, length delimited. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. - * @param message MessageAckRequest message or plain object to encode + * Encodes the specified ReadTransactionRequest message, length delimited. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. + * @param message ReadTransactionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IMessageAckRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReadTransactionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MessageAckRequest message from the specified reader or buffer. + * Decodes a ReadTransactionRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MessageAckRequest + * @returns ReadTransactionRequest * @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): query.MessageAckRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReadTransactionRequest; /** - * Decodes a MessageAckRequest message from the specified reader or buffer, length delimited. + * Decodes a ReadTransactionRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MessageAckRequest + * @returns ReadTransactionRequest * @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)): query.MessageAckRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReadTransactionRequest; /** - * Verifies a MessageAckRequest message. + * Verifies a ReadTransactionRequest 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 MessageAckRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReadTransactionRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MessageAckRequest + * @returns ReadTransactionRequest */ - public static fromObject(object: { [k: string]: any }): query.MessageAckRequest; + public static fromObject(object: { [k: string]: any }): query.ReadTransactionRequest; /** - * Creates a plain object from a MessageAckRequest message. Also converts values to other types if specified. - * @param message MessageAckRequest + * Creates a plain object from a ReadTransactionRequest message. Also converts values to other types if specified. + * @param message ReadTransactionRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.MessageAckRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReadTransactionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MessageAckRequest to JSON. + * Converts this ReadTransactionRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MessageAckRequest + * Gets the default type url for ReadTransactionRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MessageAckResponse. */ - interface IMessageAckResponse { + /** Properties of a ReadTransactionResponse. */ + interface IReadTransactionResponse { - /** MessageAckResponse result */ - result?: (query.IQueryResult|null); + /** ReadTransactionResponse metadata */ + metadata?: (query.ITransactionMetadata|null); } - /** Represents a MessageAckResponse. */ - class MessageAckResponse implements IMessageAckResponse { + /** Represents a ReadTransactionResponse. */ + class ReadTransactionResponse implements IReadTransactionResponse { /** - * Constructs a new MessageAckResponse. + * Constructs a new ReadTransactionResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IMessageAckResponse); + constructor(properties?: query.IReadTransactionResponse); - /** MessageAckResponse result. */ - public result?: (query.IQueryResult|null); + /** ReadTransactionResponse metadata. */ + public metadata?: (query.ITransactionMetadata|null); /** - * Creates a new MessageAckResponse instance using the specified properties. + * Creates a new ReadTransactionResponse instance using the specified properties. * @param [properties] Properties to set - * @returns MessageAckResponse instance + * @returns ReadTransactionResponse instance */ - public static create(properties?: query.IMessageAckResponse): query.MessageAckResponse; + public static create(properties?: query.IReadTransactionResponse): query.ReadTransactionResponse; /** - * Encodes the specified MessageAckResponse message. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. - * @param message MessageAckResponse message or plain object to encode + * Encodes the specified ReadTransactionResponse message. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. + * @param message ReadTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IMessageAckResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReadTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MessageAckResponse message, length delimited. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. - * @param message MessageAckResponse message or plain object to encode + * Encodes the specified ReadTransactionResponse message, length delimited. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. + * @param message ReadTransactionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IMessageAckResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReadTransactionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MessageAckResponse message from the specified reader or buffer. + * Decodes a ReadTransactionResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MessageAckResponse + * @returns ReadTransactionResponse * @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): query.MessageAckResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReadTransactionResponse; /** - * Decodes a MessageAckResponse message from the specified reader or buffer, length delimited. + * Decodes a ReadTransactionResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MessageAckResponse + * @returns ReadTransactionResponse * @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)): query.MessageAckResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReadTransactionResponse; /** - * Verifies a MessageAckResponse message. + * Verifies a ReadTransactionResponse 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 MessageAckResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReadTransactionResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MessageAckResponse + * @returns ReadTransactionResponse */ - public static fromObject(object: { [k: string]: any }): query.MessageAckResponse; + public static fromObject(object: { [k: string]: any }): query.ReadTransactionResponse; /** - * Creates a plain object from a MessageAckResponse message. Also converts values to other types if specified. - * @param message MessageAckResponse + * Creates a plain object from a ReadTransactionResponse message. Also converts values to other types if specified. + * @param message ReadTransactionResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.MessageAckResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReadTransactionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MessageAckResponse to JSON. + * Converts this ReadTransactionResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MessageAckResponse + * Gets the default type url for ReadTransactionResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveExecuteRequest. */ - interface IReserveExecuteRequest { + /** Properties of a BeginExecuteRequest. */ + interface IBeginExecuteRequest { - /** ReserveExecuteRequest effective_caller_id */ + /** BeginExecuteRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveExecuteRequest immediate_caller_id */ + /** BeginExecuteRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveExecuteRequest target */ + /** BeginExecuteRequest target */ target?: (query.ITarget|null); - /** ReserveExecuteRequest query */ + /** BeginExecuteRequest query */ query?: (query.IBoundQuery|null); - /** ReserveExecuteRequest transaction_id */ - transaction_id?: (number|Long|null); - - /** ReserveExecuteRequest options */ + /** BeginExecuteRequest options */ options?: (query.IExecuteOptions|null); - /** ReserveExecuteRequest pre_queries */ + /** BeginExecuteRequest reserved_id */ + reserved_id?: (number|Long|null); + + /** BeginExecuteRequest pre_queries */ pre_queries?: (string[]|null); } - /** Represents a ReserveExecuteRequest. */ - class ReserveExecuteRequest implements IReserveExecuteRequest { + /** Represents a BeginExecuteRequest. */ + class BeginExecuteRequest implements IBeginExecuteRequest { /** - * Constructs a new ReserveExecuteRequest. + * Constructs a new BeginExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveExecuteRequest); + constructor(properties?: query.IBeginExecuteRequest); - /** ReserveExecuteRequest effective_caller_id. */ + /** BeginExecuteRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveExecuteRequest immediate_caller_id. */ + /** BeginExecuteRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveExecuteRequest target. */ + /** BeginExecuteRequest target. */ public target?: (query.ITarget|null); - /** ReserveExecuteRequest query. */ + /** BeginExecuteRequest query. */ public query?: (query.IBoundQuery|null); - /** ReserveExecuteRequest transaction_id. */ - public transaction_id: (number|Long); - - /** ReserveExecuteRequest options. */ + /** BeginExecuteRequest options. */ public options?: (query.IExecuteOptions|null); - /** ReserveExecuteRequest pre_queries. */ + /** BeginExecuteRequest reserved_id. */ + public reserved_id: (number|Long); + + /** BeginExecuteRequest pre_queries. */ public pre_queries: string[]; /** - * Creates a new ReserveExecuteRequest instance using the specified properties. + * Creates a new BeginExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveExecuteRequest instance + * @returns BeginExecuteRequest instance */ - public static create(properties?: query.IReserveExecuteRequest): query.ReserveExecuteRequest; + public static create(properties?: query.IBeginExecuteRequest): query.BeginExecuteRequest; /** - * Encodes the specified ReserveExecuteRequest message. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. - * @param message ReserveExecuteRequest message or plain object to encode + * Encodes the specified BeginExecuteRequest message. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. + * @param message BeginExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. - * @param message ReserveExecuteRequest message or plain object to encode + * Encodes the specified BeginExecuteRequest message, length delimited. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. + * @param message BeginExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveExecuteRequest message from the specified reader or buffer. + * Decodes a BeginExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveExecuteRequest + * @returns BeginExecuteRequest * @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): query.ReserveExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginExecuteRequest; /** - * Decodes a ReserveExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveExecuteRequest + * @returns BeginExecuteRequest * @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)): query.ReserveExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginExecuteRequest; /** - * Verifies a ReserveExecuteRequest message. + * Verifies a BeginExecuteRequest 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 ReserveExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveExecuteRequest + * @returns BeginExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.ReserveExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.BeginExecuteRequest; /** - * Creates a plain object from a ReserveExecuteRequest message. Also converts values to other types if specified. - * @param message ReserveExecuteRequest + * Creates a plain object from a BeginExecuteRequest message. Also converts values to other types if specified. + * @param message BeginExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveExecuteRequest to JSON. + * Converts this BeginExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveExecuteRequest + * Gets the default type url for BeginExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveExecuteResponse. */ - interface IReserveExecuteResponse { + /** Properties of a BeginExecuteResponse. */ + interface IBeginExecuteResponse { - /** ReserveExecuteResponse error */ + /** BeginExecuteResponse error */ error?: (vtrpc.IRPCError|null); - /** ReserveExecuteResponse result */ + /** BeginExecuteResponse result */ result?: (query.IQueryResult|null); - /** ReserveExecuteResponse reserved_id */ - reserved_id?: (number|Long|null); + /** BeginExecuteResponse transaction_id */ + transaction_id?: (number|Long|null); - /** ReserveExecuteResponse tablet_alias */ + /** BeginExecuteResponse tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); + + /** BeginExecuteResponse session_state_changes */ + session_state_changes?: (string|null); } - /** Represents a ReserveExecuteResponse. */ - class ReserveExecuteResponse implements IReserveExecuteResponse { + /** Represents a BeginExecuteResponse. */ + class BeginExecuteResponse implements IBeginExecuteResponse { /** - * Constructs a new ReserveExecuteResponse. + * Constructs a new BeginExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveExecuteResponse); + constructor(properties?: query.IBeginExecuteResponse); - /** ReserveExecuteResponse error. */ + /** BeginExecuteResponse error. */ public error?: (vtrpc.IRPCError|null); - /** ReserveExecuteResponse result. */ + /** BeginExecuteResponse result. */ public result?: (query.IQueryResult|null); - /** ReserveExecuteResponse reserved_id. */ - public reserved_id: (number|Long); + /** BeginExecuteResponse transaction_id. */ + public transaction_id: (number|Long); - /** ReserveExecuteResponse tablet_alias. */ + /** BeginExecuteResponse tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); + /** BeginExecuteResponse session_state_changes. */ + public session_state_changes: string; + /** - * Creates a new ReserveExecuteResponse instance using the specified properties. + * Creates a new BeginExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveExecuteResponse instance + * @returns BeginExecuteResponse instance */ - public static create(properties?: query.IReserveExecuteResponse): query.ReserveExecuteResponse; + public static create(properties?: query.IBeginExecuteResponse): query.BeginExecuteResponse; /** - * Encodes the specified ReserveExecuteResponse message. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. - * @param message ReserveExecuteResponse message or plain object to encode + * Encodes the specified BeginExecuteResponse message. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. + * @param message BeginExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. - * @param message ReserveExecuteResponse message or plain object to encode + * Encodes the specified BeginExecuteResponse message, length delimited. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. + * @param message BeginExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveExecuteResponse message from the specified reader or buffer. + * Decodes a BeginExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveExecuteResponse + * @returns BeginExecuteResponse * @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): query.ReserveExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginExecuteResponse; /** - * Decodes a ReserveExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveExecuteResponse + * @returns BeginExecuteResponse * @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)): query.ReserveExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginExecuteResponse; /** - * Verifies a ReserveExecuteResponse message. + * Verifies a BeginExecuteResponse 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 ReserveExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveExecuteResponse + * @returns BeginExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.ReserveExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.BeginExecuteResponse; /** - * Creates a plain object from a ReserveExecuteResponse message. Also converts values to other types if specified. - * @param message ReserveExecuteResponse + * Creates a plain object from a BeginExecuteResponse message. Also converts values to other types if specified. + * @param message BeginExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveExecuteResponse to JSON. + * Converts this BeginExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveExecuteResponse + * Gets the default type url for BeginExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveStreamExecuteRequest. */ - interface IReserveStreamExecuteRequest { + /** Properties of a BeginStreamExecuteRequest. */ + interface IBeginStreamExecuteRequest { - /** ReserveStreamExecuteRequest effective_caller_id */ + /** BeginStreamExecuteRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveStreamExecuteRequest immediate_caller_id */ + /** BeginStreamExecuteRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveStreamExecuteRequest target */ + /** BeginStreamExecuteRequest target */ target?: (query.ITarget|null); - /** ReserveStreamExecuteRequest query */ + /** BeginStreamExecuteRequest query */ query?: (query.IBoundQuery|null); - /** ReserveStreamExecuteRequest options */ + /** BeginStreamExecuteRequest options */ options?: (query.IExecuteOptions|null); - /** ReserveStreamExecuteRequest transaction_id */ - transaction_id?: (number|Long|null); - - /** ReserveStreamExecuteRequest pre_queries */ + /** BeginStreamExecuteRequest pre_queries */ pre_queries?: (string[]|null); + + /** BeginStreamExecuteRequest reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a ReserveStreamExecuteRequest. */ - class ReserveStreamExecuteRequest implements IReserveStreamExecuteRequest { + /** Represents a BeginStreamExecuteRequest. */ + class BeginStreamExecuteRequest implements IBeginStreamExecuteRequest { /** - * Constructs a new ReserveStreamExecuteRequest. + * Constructs a new BeginStreamExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveStreamExecuteRequest); + constructor(properties?: query.IBeginStreamExecuteRequest); - /** ReserveStreamExecuteRequest effective_caller_id. */ + /** BeginStreamExecuteRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveStreamExecuteRequest immediate_caller_id. */ + /** BeginStreamExecuteRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveStreamExecuteRequest target. */ + /** BeginStreamExecuteRequest target. */ public target?: (query.ITarget|null); - /** ReserveStreamExecuteRequest query. */ + /** BeginStreamExecuteRequest query. */ public query?: (query.IBoundQuery|null); - /** ReserveStreamExecuteRequest options. */ + /** BeginStreamExecuteRequest options. */ public options?: (query.IExecuteOptions|null); - /** ReserveStreamExecuteRequest transaction_id. */ - public transaction_id: (number|Long); - - /** ReserveStreamExecuteRequest pre_queries. */ + /** BeginStreamExecuteRequest pre_queries. */ public pre_queries: string[]; + /** BeginStreamExecuteRequest reserved_id. */ + public reserved_id: (number|Long); + /** - * Creates a new ReserveStreamExecuteRequest instance using the specified properties. + * Creates a new BeginStreamExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveStreamExecuteRequest instance + * @returns BeginStreamExecuteRequest instance */ - public static create(properties?: query.IReserveStreamExecuteRequest): query.ReserveStreamExecuteRequest; + public static create(properties?: query.IBeginStreamExecuteRequest): query.BeginStreamExecuteRequest; /** - * Encodes the specified ReserveStreamExecuteRequest message. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. - * @param message ReserveStreamExecuteRequest message or plain object to encode + * Encodes the specified BeginStreamExecuteRequest message. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. + * @param message BeginStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. - * @param message ReserveStreamExecuteRequest message or plain object to encode + * Encodes the specified BeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. + * @param message BeginStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer. + * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveStreamExecuteRequest + * @returns BeginStreamExecuteRequest * @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): query.ReserveStreamExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginStreamExecuteRequest; /** - * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveStreamExecuteRequest + * @returns BeginStreamExecuteRequest * @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)): query.ReserveStreamExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginStreamExecuteRequest; /** - * Verifies a ReserveStreamExecuteRequest message. + * Verifies a BeginStreamExecuteRequest 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 ReserveStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveStreamExecuteRequest + * @returns BeginStreamExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.ReserveStreamExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.BeginStreamExecuteRequest; /** - * Creates a plain object from a ReserveStreamExecuteRequest message. Also converts values to other types if specified. - * @param message ReserveStreamExecuteRequest + * Creates a plain object from a BeginStreamExecuteRequest message. Also converts values to other types if specified. + * @param message BeginStreamExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveStreamExecuteRequest to JSON. + * Converts this BeginStreamExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveStreamExecuteRequest + * Gets the default type url for BeginStreamExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveStreamExecuteResponse. */ - interface IReserveStreamExecuteResponse { + /** Properties of a BeginStreamExecuteResponse. */ + interface IBeginStreamExecuteResponse { - /** ReserveStreamExecuteResponse error */ + /** BeginStreamExecuteResponse error */ error?: (vtrpc.IRPCError|null); - /** ReserveStreamExecuteResponse result */ + /** BeginStreamExecuteResponse result */ result?: (query.IQueryResult|null); - /** ReserveStreamExecuteResponse reserved_id */ - reserved_id?: (number|Long|null); + /** BeginStreamExecuteResponse transaction_id */ + transaction_id?: (number|Long|null); - /** ReserveStreamExecuteResponse tablet_alias */ + /** BeginStreamExecuteResponse tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); + + /** BeginStreamExecuteResponse session_state_changes */ + session_state_changes?: (string|null); } - /** Represents a ReserveStreamExecuteResponse. */ - class ReserveStreamExecuteResponse implements IReserveStreamExecuteResponse { + /** Represents a BeginStreamExecuteResponse. */ + class BeginStreamExecuteResponse implements IBeginStreamExecuteResponse { /** - * Constructs a new ReserveStreamExecuteResponse. + * Constructs a new BeginStreamExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveStreamExecuteResponse); + constructor(properties?: query.IBeginStreamExecuteResponse); - /** ReserveStreamExecuteResponse error. */ + /** BeginStreamExecuteResponse error. */ public error?: (vtrpc.IRPCError|null); - /** ReserveStreamExecuteResponse result. */ + /** BeginStreamExecuteResponse result. */ public result?: (query.IQueryResult|null); - /** ReserveStreamExecuteResponse reserved_id. */ - public reserved_id: (number|Long); + /** BeginStreamExecuteResponse transaction_id. */ + public transaction_id: (number|Long); - /** ReserveStreamExecuteResponse tablet_alias. */ + /** BeginStreamExecuteResponse tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); + /** BeginStreamExecuteResponse session_state_changes. */ + public session_state_changes: string; + /** - * Creates a new ReserveStreamExecuteResponse instance using the specified properties. + * Creates a new BeginStreamExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveStreamExecuteResponse instance + * @returns BeginStreamExecuteResponse instance */ - public static create(properties?: query.IReserveStreamExecuteResponse): query.ReserveStreamExecuteResponse; + public static create(properties?: query.IBeginStreamExecuteResponse): query.BeginStreamExecuteResponse; /** - * Encodes the specified ReserveStreamExecuteResponse message. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. - * @param message ReserveStreamExecuteResponse message or plain object to encode + * Encodes the specified BeginStreamExecuteResponse message. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. + * @param message BeginStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. - * @param message ReserveStreamExecuteResponse message or plain object to encode + * Encodes the specified BeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. + * @param message BeginStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer. + * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveStreamExecuteResponse + * @returns BeginStreamExecuteResponse * @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): query.ReserveStreamExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.BeginStreamExecuteResponse; /** - * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveStreamExecuteResponse + * @returns BeginStreamExecuteResponse * @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)): query.ReserveStreamExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.BeginStreamExecuteResponse; /** - * Verifies a ReserveStreamExecuteResponse message. + * Verifies a BeginStreamExecuteResponse 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 ReserveStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveStreamExecuteResponse + * @returns BeginStreamExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.ReserveStreamExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.BeginStreamExecuteResponse; /** - * Creates a plain object from a ReserveStreamExecuteResponse message. Also converts values to other types if specified. - * @param message ReserveStreamExecuteResponse + * Creates a plain object from a BeginStreamExecuteResponse message. Also converts values to other types if specified. + * @param message BeginStreamExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.BeginStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveStreamExecuteResponse to JSON. + * Converts this BeginStreamExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveStreamExecuteResponse + * Gets the default type url for BeginStreamExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveBeginExecuteRequest. */ - interface IReserveBeginExecuteRequest { + /** Properties of a MessageStreamRequest. */ + interface IMessageStreamRequest { - /** ReserveBeginExecuteRequest effective_caller_id */ + /** MessageStreamRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveBeginExecuteRequest immediate_caller_id */ + /** MessageStreamRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveBeginExecuteRequest target */ + /** MessageStreamRequest target */ target?: (query.ITarget|null); - /** ReserveBeginExecuteRequest query */ - query?: (query.IBoundQuery|null); - - /** ReserveBeginExecuteRequest options */ - options?: (query.IExecuteOptions|null); - - /** ReserveBeginExecuteRequest pre_queries */ - pre_queries?: (string[]|null); - - /** ReserveBeginExecuteRequest post_begin_queries */ - post_begin_queries?: (string[]|null); + /** MessageStreamRequest name */ + name?: (string|null); } - /** Represents a ReserveBeginExecuteRequest. */ - class ReserveBeginExecuteRequest implements IReserveBeginExecuteRequest { + /** Represents a MessageStreamRequest. */ + class MessageStreamRequest implements IMessageStreamRequest { /** - * Constructs a new ReserveBeginExecuteRequest. + * Constructs a new MessageStreamRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveBeginExecuteRequest); + constructor(properties?: query.IMessageStreamRequest); - /** ReserveBeginExecuteRequest effective_caller_id. */ + /** MessageStreamRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveBeginExecuteRequest immediate_caller_id. */ + /** MessageStreamRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveBeginExecuteRequest target. */ + /** MessageStreamRequest target. */ public target?: (query.ITarget|null); - /** ReserveBeginExecuteRequest query. */ - public query?: (query.IBoundQuery|null); - - /** ReserveBeginExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); - - /** ReserveBeginExecuteRequest pre_queries. */ - public pre_queries: string[]; - - /** ReserveBeginExecuteRequest post_begin_queries. */ - public post_begin_queries: string[]; + /** MessageStreamRequest name. */ + public name: string; /** - * Creates a new ReserveBeginExecuteRequest instance using the specified properties. + * Creates a new MessageStreamRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveBeginExecuteRequest instance + * @returns MessageStreamRequest instance */ - public static create(properties?: query.IReserveBeginExecuteRequest): query.ReserveBeginExecuteRequest; + public static create(properties?: query.IMessageStreamRequest): query.MessageStreamRequest; /** - * Encodes the specified ReserveBeginExecuteRequest message. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. - * @param message ReserveBeginExecuteRequest message or plain object to encode + * Encodes the specified MessageStreamRequest message. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. + * @param message MessageStreamRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IMessageStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveBeginExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. - * @param message ReserveBeginExecuteRequest message or plain object to encode + * Encodes the specified MessageStreamRequest message, length delimited. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. + * @param message MessageStreamRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IMessageStreamRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer. + * Decodes a MessageStreamRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveBeginExecuteRequest + * @returns MessageStreamRequest * @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): query.ReserveBeginExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.MessageStreamRequest; /** - * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a MessageStreamRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveBeginExecuteRequest + * @returns MessageStreamRequest * @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)): query.ReserveBeginExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.MessageStreamRequest; /** - * Verifies a ReserveBeginExecuteRequest message. + * Verifies a MessageStreamRequest 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 ReserveBeginExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MessageStreamRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveBeginExecuteRequest + * @returns MessageStreamRequest */ - public static fromObject(object: { [k: string]: any }): query.ReserveBeginExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.MessageStreamRequest; /** - * Creates a plain object from a ReserveBeginExecuteRequest message. Also converts values to other types if specified. - * @param message ReserveBeginExecuteRequest + * Creates a plain object from a MessageStreamRequest message. Also converts values to other types if specified. + * @param message MessageStreamRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveBeginExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.MessageStreamRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveBeginExecuteRequest to JSON. + * Converts this MessageStreamRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveBeginExecuteRequest + * Gets the default type url for MessageStreamRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveBeginExecuteResponse. */ - interface IReserveBeginExecuteResponse { - - /** ReserveBeginExecuteResponse error */ - error?: (vtrpc.IRPCError|null); + /** Properties of a MessageStreamResponse. */ + interface IMessageStreamResponse { - /** ReserveBeginExecuteResponse result */ + /** MessageStreamResponse result */ result?: (query.IQueryResult|null); - - /** ReserveBeginExecuteResponse transaction_id */ - transaction_id?: (number|Long|null); - - /** ReserveBeginExecuteResponse reserved_id */ - reserved_id?: (number|Long|null); - - /** ReserveBeginExecuteResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** ReserveBeginExecuteResponse session_state_changes */ - session_state_changes?: (string|null); } - /** Represents a ReserveBeginExecuteResponse. */ - class ReserveBeginExecuteResponse implements IReserveBeginExecuteResponse { + /** Represents a MessageStreamResponse. */ + class MessageStreamResponse implements IMessageStreamResponse { /** - * Constructs a new ReserveBeginExecuteResponse. + * Constructs a new MessageStreamResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveBeginExecuteResponse); - - /** ReserveBeginExecuteResponse error. */ - public error?: (vtrpc.IRPCError|null); + constructor(properties?: query.IMessageStreamResponse); - /** ReserveBeginExecuteResponse result. */ + /** MessageStreamResponse result. */ public result?: (query.IQueryResult|null); - /** ReserveBeginExecuteResponse transaction_id. */ - public transaction_id: (number|Long); - - /** ReserveBeginExecuteResponse reserved_id. */ - public reserved_id: (number|Long); - - /** ReserveBeginExecuteResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** ReserveBeginExecuteResponse session_state_changes. */ - public session_state_changes: string; - /** - * Creates a new ReserveBeginExecuteResponse instance using the specified properties. + * Creates a new MessageStreamResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveBeginExecuteResponse instance + * @returns MessageStreamResponse instance */ - public static create(properties?: query.IReserveBeginExecuteResponse): query.ReserveBeginExecuteResponse; + public static create(properties?: query.IMessageStreamResponse): query.MessageStreamResponse; /** - * Encodes the specified ReserveBeginExecuteResponse message. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. - * @param message ReserveBeginExecuteResponse message or plain object to encode + * Encodes the specified MessageStreamResponse message. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. + * @param message MessageStreamResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IMessageStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveBeginExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. - * @param message ReserveBeginExecuteResponse message or plain object to encode + * Encodes the specified MessageStreamResponse message, length delimited. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. + * @param message MessageStreamResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IMessageStreamResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer. + * Decodes a MessageStreamResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveBeginExecuteResponse + * @returns MessageStreamResponse * @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): query.ReserveBeginExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.MessageStreamResponse; /** - * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a MessageStreamResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveBeginExecuteResponse + * @returns MessageStreamResponse * @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)): query.ReserveBeginExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.MessageStreamResponse; /** - * Verifies a ReserveBeginExecuteResponse message. + * Verifies a MessageStreamResponse 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 ReserveBeginExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MessageStreamResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveBeginExecuteResponse + * @returns MessageStreamResponse */ - public static fromObject(object: { [k: string]: any }): query.ReserveBeginExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.MessageStreamResponse; /** - * Creates a plain object from a ReserveBeginExecuteResponse message. Also converts values to other types if specified. - * @param message ReserveBeginExecuteResponse + * Creates a plain object from a MessageStreamResponse message. Also converts values to other types if specified. + * @param message MessageStreamResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveBeginExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.MessageStreamResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveBeginExecuteResponse to JSON. + * Converts this MessageStreamResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveBeginExecuteResponse + * Gets the default type url for MessageStreamResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveBeginStreamExecuteRequest. */ - interface IReserveBeginStreamExecuteRequest { + /** Properties of a MessageAckRequest. */ + interface IMessageAckRequest { - /** ReserveBeginStreamExecuteRequest effective_caller_id */ + /** MessageAckRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveBeginStreamExecuteRequest immediate_caller_id */ + /** MessageAckRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveBeginStreamExecuteRequest target */ + /** MessageAckRequest target */ target?: (query.ITarget|null); - /** ReserveBeginStreamExecuteRequest query */ - query?: (query.IBoundQuery|null); - - /** ReserveBeginStreamExecuteRequest options */ - options?: (query.IExecuteOptions|null); - - /** ReserveBeginStreamExecuteRequest pre_queries */ - pre_queries?: (string[]|null); + /** MessageAckRequest name */ + name?: (string|null); - /** ReserveBeginStreamExecuteRequest post_begin_queries */ - post_begin_queries?: (string[]|null); + /** MessageAckRequest ids */ + ids?: (query.IValue[]|null); } - /** Represents a ReserveBeginStreamExecuteRequest. */ - class ReserveBeginStreamExecuteRequest implements IReserveBeginStreamExecuteRequest { + /** Represents a MessageAckRequest. */ + class MessageAckRequest implements IMessageAckRequest { /** - * Constructs a new ReserveBeginStreamExecuteRequest. + * Constructs a new MessageAckRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveBeginStreamExecuteRequest); + constructor(properties?: query.IMessageAckRequest); - /** ReserveBeginStreamExecuteRequest effective_caller_id. */ + /** MessageAckRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReserveBeginStreamExecuteRequest immediate_caller_id. */ + /** MessageAckRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReserveBeginStreamExecuteRequest target. */ + /** MessageAckRequest target. */ public target?: (query.ITarget|null); - /** ReserveBeginStreamExecuteRequest query. */ - public query?: (query.IBoundQuery|null); - - /** ReserveBeginStreamExecuteRequest options. */ - public options?: (query.IExecuteOptions|null); - - /** ReserveBeginStreamExecuteRequest pre_queries. */ - public pre_queries: string[]; + /** MessageAckRequest name. */ + public name: string; - /** ReserveBeginStreamExecuteRequest post_begin_queries. */ - public post_begin_queries: string[]; + /** MessageAckRequest ids. */ + public ids: query.IValue[]; /** - * Creates a new ReserveBeginStreamExecuteRequest instance using the specified properties. + * Creates a new MessageAckRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveBeginStreamExecuteRequest instance + * @returns MessageAckRequest instance */ - public static create(properties?: query.IReserveBeginStreamExecuteRequest): query.ReserveBeginStreamExecuteRequest; + public static create(properties?: query.IMessageAckRequest): query.MessageAckRequest; /** - * Encodes the specified ReserveBeginStreamExecuteRequest message. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. - * @param message ReserveBeginStreamExecuteRequest message or plain object to encode + * Encodes the specified MessageAckRequest message. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. + * @param message MessageAckRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IMessageAckRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveBeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. - * @param message ReserveBeginStreamExecuteRequest message or plain object to encode + * Encodes the specified MessageAckRequest message, length delimited. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. + * @param message MessageAckRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IMessageAckRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer. + * Decodes a MessageAckRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveBeginStreamExecuteRequest + * @returns MessageAckRequest * @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): query.ReserveBeginStreamExecuteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.MessageAckRequest; /** - * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a MessageAckRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveBeginStreamExecuteRequest + * @returns MessageAckRequest * @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)): query.ReserveBeginStreamExecuteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.MessageAckRequest; /** - * Verifies a ReserveBeginStreamExecuteRequest message. + * Verifies a MessageAckRequest 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 ReserveBeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MessageAckRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveBeginStreamExecuteRequest + * @returns MessageAckRequest */ - public static fromObject(object: { [k: string]: any }): query.ReserveBeginStreamExecuteRequest; + public static fromObject(object: { [k: string]: any }): query.MessageAckRequest; /** - * Creates a plain object from a ReserveBeginStreamExecuteRequest message. Also converts values to other types if specified. - * @param message ReserveBeginStreamExecuteRequest + * Creates a plain object from a MessageAckRequest message. Also converts values to other types if specified. + * @param message MessageAckRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveBeginStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.MessageAckRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveBeginStreamExecuteRequest to JSON. + * Converts this MessageAckRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveBeginStreamExecuteRequest + * Gets the default type url for MessageAckRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReserveBeginStreamExecuteResponse. */ - interface IReserveBeginStreamExecuteResponse { - - /** ReserveBeginStreamExecuteResponse error */ - error?: (vtrpc.IRPCError|null); + /** Properties of a MessageAckResponse. */ + interface IMessageAckResponse { - /** ReserveBeginStreamExecuteResponse result */ + /** MessageAckResponse result */ result?: (query.IQueryResult|null); - - /** ReserveBeginStreamExecuteResponse transaction_id */ - transaction_id?: (number|Long|null); - - /** ReserveBeginStreamExecuteResponse reserved_id */ - reserved_id?: (number|Long|null); - - /** ReserveBeginStreamExecuteResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** ReserveBeginStreamExecuteResponse session_state_changes */ - session_state_changes?: (string|null); } - /** Represents a ReserveBeginStreamExecuteResponse. */ - class ReserveBeginStreamExecuteResponse implements IReserveBeginStreamExecuteResponse { + /** Represents a MessageAckResponse. */ + class MessageAckResponse implements IMessageAckResponse { /** - * Constructs a new ReserveBeginStreamExecuteResponse. + * Constructs a new MessageAckResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReserveBeginStreamExecuteResponse); - - /** ReserveBeginStreamExecuteResponse error. */ - public error?: (vtrpc.IRPCError|null); + constructor(properties?: query.IMessageAckResponse); - /** ReserveBeginStreamExecuteResponse result. */ + /** MessageAckResponse result. */ public result?: (query.IQueryResult|null); - /** ReserveBeginStreamExecuteResponse transaction_id. */ - public transaction_id: (number|Long); - - /** ReserveBeginStreamExecuteResponse reserved_id. */ - public reserved_id: (number|Long); - - /** ReserveBeginStreamExecuteResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** ReserveBeginStreamExecuteResponse session_state_changes. */ - public session_state_changes: string; - /** - * Creates a new ReserveBeginStreamExecuteResponse instance using the specified properties. + * Creates a new MessageAckResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReserveBeginStreamExecuteResponse instance + * @returns MessageAckResponse instance */ - public static create(properties?: query.IReserveBeginStreamExecuteResponse): query.ReserveBeginStreamExecuteResponse; + public static create(properties?: query.IMessageAckResponse): query.MessageAckResponse; /** - * Encodes the specified ReserveBeginStreamExecuteResponse message. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. - * @param message ReserveBeginStreamExecuteResponse message or plain object to encode + * Encodes the specified MessageAckResponse message. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. + * @param message MessageAckResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReserveBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IMessageAckResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReserveBeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. - * @param message ReserveBeginStreamExecuteResponse message or plain object to encode + * Encodes the specified MessageAckResponse message, length delimited. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. + * @param message MessageAckResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReserveBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IMessageAckResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer. + * Decodes a MessageAckResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReserveBeginStreamExecuteResponse + * @returns MessageAckResponse * @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): query.ReserveBeginStreamExecuteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.MessageAckResponse; /** - * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a MessageAckResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReserveBeginStreamExecuteResponse + * @returns MessageAckResponse * @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)): query.ReserveBeginStreamExecuteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.MessageAckResponse; /** - * Verifies a ReserveBeginStreamExecuteResponse message. + * Verifies a MessageAckResponse 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 ReserveBeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MessageAckResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReserveBeginStreamExecuteResponse + * @returns MessageAckResponse */ - public static fromObject(object: { [k: string]: any }): query.ReserveBeginStreamExecuteResponse; + public static fromObject(object: { [k: string]: any }): query.MessageAckResponse; /** - * Creates a plain object from a ReserveBeginStreamExecuteResponse message. Also converts values to other types if specified. - * @param message ReserveBeginStreamExecuteResponse + * Creates a plain object from a MessageAckResponse message. Also converts values to other types if specified. + * @param message MessageAckResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReserveBeginStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.MessageAckResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReserveBeginStreamExecuteResponse to JSON. + * Converts this MessageAckResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReserveBeginStreamExecuteResponse + * Gets the default type url for MessageAckResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReleaseRequest. */ - interface IReleaseRequest { + /** Properties of a ReserveExecuteRequest. */ + interface IReserveExecuteRequest { - /** ReleaseRequest effective_caller_id */ + /** ReserveExecuteRequest effective_caller_id */ effective_caller_id?: (vtrpc.ICallerID|null); - /** ReleaseRequest immediate_caller_id */ + /** ReserveExecuteRequest immediate_caller_id */ immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReleaseRequest target */ + /** ReserveExecuteRequest target */ target?: (query.ITarget|null); - /** ReleaseRequest transaction_id */ + /** ReserveExecuteRequest query */ + query?: (query.IBoundQuery|null); + + /** ReserveExecuteRequest transaction_id */ transaction_id?: (number|Long|null); - /** ReleaseRequest reserved_id */ - reserved_id?: (number|Long|null); + /** ReserveExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** ReserveExecuteRequest pre_queries */ + pre_queries?: (string[]|null); } - /** Represents a ReleaseRequest. */ - class ReleaseRequest implements IReleaseRequest { + /** Represents a ReserveExecuteRequest. */ + class ReserveExecuteRequest implements IReserveExecuteRequest { /** - * Constructs a new ReleaseRequest. + * Constructs a new ReserveExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IReleaseRequest); + constructor(properties?: query.IReserveExecuteRequest); - /** ReleaseRequest effective_caller_id. */ + /** ReserveExecuteRequest effective_caller_id. */ public effective_caller_id?: (vtrpc.ICallerID|null); - /** ReleaseRequest immediate_caller_id. */ + /** ReserveExecuteRequest immediate_caller_id. */ public immediate_caller_id?: (query.IVTGateCallerID|null); - /** ReleaseRequest target. */ + /** ReserveExecuteRequest target. */ public target?: (query.ITarget|null); - /** ReleaseRequest transaction_id. */ + /** ReserveExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** ReserveExecuteRequest transaction_id. */ public transaction_id: (number|Long); - /** ReleaseRequest reserved_id. */ - public reserved_id: (number|Long); + /** ReserveExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** ReserveExecuteRequest pre_queries. */ + public pre_queries: string[]; /** - * Creates a new ReleaseRequest instance using the specified properties. + * Creates a new ReserveExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReleaseRequest instance + * @returns ReserveExecuteRequest instance */ - public static create(properties?: query.IReleaseRequest): query.ReleaseRequest; + public static create(properties?: query.IReserveExecuteRequest): query.ReserveExecuteRequest; /** - * Encodes the specified ReleaseRequest message. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. - * @param message ReleaseRequest message or plain object to encode + * Encodes the specified ReserveExecuteRequest message. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. + * @param message ReserveExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReleaseRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReleaseRequest message, length delimited. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. - * @param message ReleaseRequest message or plain object to encode + * Encodes the specified ReserveExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. + * @param message ReserveExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReleaseRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReleaseRequest message from the specified reader or buffer. + * Decodes a ReserveExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReleaseRequest + * @returns ReserveExecuteRequest * @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): query.ReleaseRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveExecuteRequest; /** - * Decodes a ReleaseRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReleaseRequest + * @returns ReserveExecuteRequest * @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)): query.ReleaseRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveExecuteRequest; /** - * Verifies a ReleaseRequest message. + * Verifies a ReserveExecuteRequest 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 ReleaseRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReleaseRequest + * @returns ReserveExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.ReleaseRequest; + public static fromObject(object: { [k: string]: any }): query.ReserveExecuteRequest; /** - * Creates a plain object from a ReleaseRequest message. Also converts values to other types if specified. - * @param message ReleaseRequest + * Creates a plain object from a ReserveExecuteRequest message. Also converts values to other types if specified. + * @param message ReserveExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReleaseRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReleaseRequest to JSON. + * Converts this ReserveExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReleaseRequest + * Gets the default type url for ReserveExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReleaseResponse. */ - interface IReleaseResponse { + /** Properties of a ReserveExecuteResponse. */ + interface IReserveExecuteResponse { + + /** ReserveExecuteResponse error */ + error?: (vtrpc.IRPCError|null); + + /** ReserveExecuteResponse result */ + result?: (query.IQueryResult|null); + + /** ReserveExecuteResponse reserved_id */ + reserved_id?: (number|Long|null); + + /** ReserveExecuteResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a ReleaseResponse. */ - class ReleaseResponse implements IReleaseResponse { + /** Represents a ReserveExecuteResponse. */ + class ReserveExecuteResponse implements IReserveExecuteResponse { /** - * Constructs a new ReleaseResponse. + * Constructs a new ReserveExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IReleaseResponse); + constructor(properties?: query.IReserveExecuteResponse); + + /** ReserveExecuteResponse error. */ + public error?: (vtrpc.IRPCError|null); + + /** ReserveExecuteResponse result. */ + public result?: (query.IQueryResult|null); + + /** ReserveExecuteResponse reserved_id. */ + public reserved_id: (number|Long); + + /** ReserveExecuteResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new ReleaseResponse instance using the specified properties. + * Creates a new ReserveExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReleaseResponse instance + * @returns ReserveExecuteResponse instance */ - public static create(properties?: query.IReleaseResponse): query.ReleaseResponse; + public static create(properties?: query.IReserveExecuteResponse): query.ReserveExecuteResponse; /** - * Encodes the specified ReleaseResponse message. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. - * @param message ReleaseResponse message or plain object to encode + * Encodes the specified ReserveExecuteResponse message. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. + * @param message ReserveExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IReleaseResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReleaseResponse message, length delimited. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. - * @param message ReleaseResponse message or plain object to encode + * Encodes the specified ReserveExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. + * @param message ReserveExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IReleaseResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReleaseResponse message from the specified reader or buffer. + * Decodes a ReserveExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReleaseResponse + * @returns ReserveExecuteResponse * @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): query.ReleaseResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveExecuteResponse; /** - * Decodes a ReleaseResponse message from the specified reader or buffer, length delimited. + * Decodes a ReserveExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReleaseResponse + * @returns ReserveExecuteResponse * @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)): query.ReleaseResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveExecuteResponse; /** - * Verifies a ReleaseResponse message. + * Verifies a ReserveExecuteResponse 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 ReleaseResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReleaseResponse + * @returns ReserveExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.ReleaseResponse; + public static fromObject(object: { [k: string]: any }): query.ReserveExecuteResponse; /** - * Creates a plain object from a ReleaseResponse message. Also converts values to other types if specified. - * @param message ReleaseResponse + * Creates a plain object from a ReserveExecuteResponse message. Also converts values to other types if specified. + * @param message ReserveExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.ReleaseResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReleaseResponse to JSON. + * Converts this ReserveExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReleaseResponse + * Gets the default type url for ReserveExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamHealthRequest. */ - interface IStreamHealthRequest { + /** Properties of a ReserveStreamExecuteRequest. */ + interface IReserveStreamExecuteRequest { + + /** ReserveStreamExecuteRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); + + /** ReserveStreamExecuteRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** ReserveStreamExecuteRequest target */ + target?: (query.ITarget|null); + + /** ReserveStreamExecuteRequest query */ + query?: (query.IBoundQuery|null); + + /** ReserveStreamExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** ReserveStreamExecuteRequest transaction_id */ + transaction_id?: (number|Long|null); + + /** ReserveStreamExecuteRequest pre_queries */ + pre_queries?: (string[]|null); } - /** Represents a StreamHealthRequest. */ - class StreamHealthRequest implements IStreamHealthRequest { + /** Represents a ReserveStreamExecuteRequest. */ + class ReserveStreamExecuteRequest implements IReserveStreamExecuteRequest { /** - * Constructs a new StreamHealthRequest. + * Constructs a new ReserveStreamExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IStreamHealthRequest); + constructor(properties?: query.IReserveStreamExecuteRequest); + + /** ReserveStreamExecuteRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); + + /** ReserveStreamExecuteRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** ReserveStreamExecuteRequest target. */ + public target?: (query.ITarget|null); + + /** ReserveStreamExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** ReserveStreamExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** ReserveStreamExecuteRequest transaction_id. */ + public transaction_id: (number|Long); + + /** ReserveStreamExecuteRequest pre_queries. */ + public pre_queries: string[]; /** - * Creates a new StreamHealthRequest instance using the specified properties. + * Creates a new ReserveStreamExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns StreamHealthRequest instance + * @returns ReserveStreamExecuteRequest instance */ - public static create(properties?: query.IStreamHealthRequest): query.StreamHealthRequest; + public static create(properties?: query.IReserveStreamExecuteRequest): query.ReserveStreamExecuteRequest; /** - * Encodes the specified StreamHealthRequest message. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. - * @param message StreamHealthRequest message or plain object to encode + * Encodes the specified ReserveStreamExecuteRequest message. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. + * @param message ReserveStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStreamHealthRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamHealthRequest message, length delimited. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. - * @param message StreamHealthRequest message or plain object to encode + * Encodes the specified ReserveStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. + * @param message ReserveStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStreamHealthRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamHealthRequest message from the specified reader or buffer. + * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamHealthRequest + * @returns ReserveStreamExecuteRequest * @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): query.StreamHealthRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveStreamExecuteRequest; /** - * Decodes a StreamHealthRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamHealthRequest + * @returns ReserveStreamExecuteRequest * @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)): query.StreamHealthRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveStreamExecuteRequest; /** - * Verifies a StreamHealthRequest message. + * Verifies a ReserveStreamExecuteRequest 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 StreamHealthRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamHealthRequest + * @returns ReserveStreamExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.StreamHealthRequest; + public static fromObject(object: { [k: string]: any }): query.ReserveStreamExecuteRequest; /** - * Creates a plain object from a StreamHealthRequest message. Also converts values to other types if specified. - * @param message StreamHealthRequest + * Creates a plain object from a ReserveStreamExecuteRequest message. Also converts values to other types if specified. + * @param message ReserveStreamExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StreamHealthRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamHealthRequest to JSON. + * Converts this ReserveStreamExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamHealthRequest + * Gets the default type url for ReserveStreamExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RealtimeStats. */ - interface IRealtimeStats { - - /** RealtimeStats health_error */ - health_error?: (string|null); - - /** RealtimeStats replication_lag_seconds */ - replication_lag_seconds?: (number|null); - - /** RealtimeStats binlog_players_count */ - binlog_players_count?: (number|null); - - /** RealtimeStats filtered_replication_lag_seconds */ - filtered_replication_lag_seconds?: (number|Long|null); + /** Properties of a ReserveStreamExecuteResponse. */ + interface IReserveStreamExecuteResponse { - /** RealtimeStats cpu_usage */ - cpu_usage?: (number|null); + /** ReserveStreamExecuteResponse error */ + error?: (vtrpc.IRPCError|null); - /** RealtimeStats qps */ - qps?: (number|null); + /** ReserveStreamExecuteResponse result */ + result?: (query.IQueryResult|null); - /** RealtimeStats table_schema_changed */ - table_schema_changed?: (string[]|null); + /** ReserveStreamExecuteResponse reserved_id */ + reserved_id?: (number|Long|null); - /** RealtimeStats view_schema_changed */ - view_schema_changed?: (string[]|null); + /** ReserveStreamExecuteResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a RealtimeStats. */ - class RealtimeStats implements IRealtimeStats { + /** Represents a ReserveStreamExecuteResponse. */ + class ReserveStreamExecuteResponse implements IReserveStreamExecuteResponse { /** - * Constructs a new RealtimeStats. + * Constructs a new ReserveStreamExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IRealtimeStats); - - /** RealtimeStats health_error. */ - public health_error: string; - - /** RealtimeStats replication_lag_seconds. */ - public replication_lag_seconds: number; - - /** RealtimeStats binlog_players_count. */ - public binlog_players_count: number; - - /** RealtimeStats filtered_replication_lag_seconds. */ - public filtered_replication_lag_seconds: (number|Long); + constructor(properties?: query.IReserveStreamExecuteResponse); - /** RealtimeStats cpu_usage. */ - public cpu_usage: number; + /** ReserveStreamExecuteResponse error. */ + public error?: (vtrpc.IRPCError|null); - /** RealtimeStats qps. */ - public qps: number; + /** ReserveStreamExecuteResponse result. */ + public result?: (query.IQueryResult|null); - /** RealtimeStats table_schema_changed. */ - public table_schema_changed: string[]; + /** ReserveStreamExecuteResponse reserved_id. */ + public reserved_id: (number|Long); - /** RealtimeStats view_schema_changed. */ - public view_schema_changed: string[]; + /** ReserveStreamExecuteResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new RealtimeStats instance using the specified properties. + * Creates a new ReserveStreamExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RealtimeStats instance + * @returns ReserveStreamExecuteResponse instance */ - public static create(properties?: query.IRealtimeStats): query.RealtimeStats; + public static create(properties?: query.IReserveStreamExecuteResponse): query.ReserveStreamExecuteResponse; /** - * Encodes the specified RealtimeStats message. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. - * @param message RealtimeStats message or plain object to encode + * Encodes the specified ReserveStreamExecuteResponse message. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. + * @param message ReserveStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IRealtimeStats, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RealtimeStats message, length delimited. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. - * @param message RealtimeStats message or plain object to encode + * Encodes the specified ReserveStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. + * @param message ReserveStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IRealtimeStats, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RealtimeStats message from the specified reader or buffer. + * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RealtimeStats + * @returns ReserveStreamExecuteResponse * @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): query.RealtimeStats; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveStreamExecuteResponse; /** - * Decodes a RealtimeStats message from the specified reader or buffer, length delimited. + * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RealtimeStats + * @returns ReserveStreamExecuteResponse * @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)): query.RealtimeStats; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveStreamExecuteResponse; /** - * Verifies a RealtimeStats message. + * Verifies a ReserveStreamExecuteResponse 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 RealtimeStats message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RealtimeStats + * @returns ReserveStreamExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.RealtimeStats; + public static fromObject(object: { [k: string]: any }): query.ReserveStreamExecuteResponse; /** - * Creates a plain object from a RealtimeStats message. Also converts values to other types if specified. - * @param message RealtimeStats + * Creates a plain object from a ReserveStreamExecuteResponse message. Also converts values to other types if specified. + * @param message ReserveStreamExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.RealtimeStats, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RealtimeStats to JSON. + * Converts this ReserveStreamExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RealtimeStats + * Gets the default type url for ReserveStreamExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an AggregateStats. */ - interface IAggregateStats { + /** Properties of a ReserveBeginExecuteRequest. */ + interface IReserveBeginExecuteRequest { - /** AggregateStats healthy_tablet_count */ - healthy_tablet_count?: (number|null); + /** ReserveBeginExecuteRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); - /** AggregateStats unhealthy_tablet_count */ - unhealthy_tablet_count?: (number|null); + /** ReserveBeginExecuteRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); - /** AggregateStats replication_lag_seconds_min */ - replication_lag_seconds_min?: (number|null); + /** ReserveBeginExecuteRequest target */ + target?: (query.ITarget|null); - /** AggregateStats replication_lag_seconds_max */ - replication_lag_seconds_max?: (number|null); + /** ReserveBeginExecuteRequest query */ + query?: (query.IBoundQuery|null); + + /** ReserveBeginExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** ReserveBeginExecuteRequest pre_queries */ + pre_queries?: (string[]|null); + + /** ReserveBeginExecuteRequest post_begin_queries */ + post_begin_queries?: (string[]|null); } - /** Represents an AggregateStats. */ - class AggregateStats implements IAggregateStats { + /** Represents a ReserveBeginExecuteRequest. */ + class ReserveBeginExecuteRequest implements IReserveBeginExecuteRequest { /** - * Constructs a new AggregateStats. + * Constructs a new ReserveBeginExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IAggregateStats); + constructor(properties?: query.IReserveBeginExecuteRequest); - /** AggregateStats healthy_tablet_count. */ - public healthy_tablet_count: number; + /** ReserveBeginExecuteRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); - /** AggregateStats unhealthy_tablet_count. */ - public unhealthy_tablet_count: number; + /** ReserveBeginExecuteRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); - /** AggregateStats replication_lag_seconds_min. */ - public replication_lag_seconds_min: number; + /** ReserveBeginExecuteRequest target. */ + public target?: (query.ITarget|null); - /** AggregateStats replication_lag_seconds_max. */ - public replication_lag_seconds_max: number; + /** ReserveBeginExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** ReserveBeginExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** ReserveBeginExecuteRequest pre_queries. */ + public pre_queries: string[]; + + /** ReserveBeginExecuteRequest post_begin_queries. */ + public post_begin_queries: string[]; /** - * Creates a new AggregateStats instance using the specified properties. + * Creates a new ReserveBeginExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns AggregateStats instance + * @returns ReserveBeginExecuteRequest instance */ - public static create(properties?: query.IAggregateStats): query.AggregateStats; + public static create(properties?: query.IReserveBeginExecuteRequest): query.ReserveBeginExecuteRequest; /** - * Encodes the specified AggregateStats message. Does not implicitly {@link query.AggregateStats.verify|verify} messages. - * @param message AggregateStats message or plain object to encode + * Encodes the specified ReserveBeginExecuteRequest message. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. + * @param message ReserveBeginExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IAggregateStats, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AggregateStats message, length delimited. Does not implicitly {@link query.AggregateStats.verify|verify} messages. - * @param message AggregateStats message or plain object to encode + * Encodes the specified ReserveBeginExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. + * @param message ReserveBeginExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IAggregateStats, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveBeginExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AggregateStats message from the specified reader or buffer. + * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AggregateStats + * @returns ReserveBeginExecuteRequest * @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): query.AggregateStats; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveBeginExecuteRequest; /** - * Decodes an AggregateStats message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AggregateStats + * @returns ReserveBeginExecuteRequest * @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)): query.AggregateStats; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveBeginExecuteRequest; /** - * Verifies an AggregateStats message. + * Verifies a ReserveBeginExecuteRequest 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 AggregateStats message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AggregateStats + * @returns ReserveBeginExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.AggregateStats; + public static fromObject(object: { [k: string]: any }): query.ReserveBeginExecuteRequest; /** - * Creates a plain object from an AggregateStats message. Also converts values to other types if specified. - * @param message AggregateStats + * Creates a plain object from a ReserveBeginExecuteRequest message. Also converts values to other types if specified. + * @param message ReserveBeginExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.AggregateStats, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveBeginExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AggregateStats to JSON. + * Converts this ReserveBeginExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AggregateStats + * Gets the default type url for ReserveBeginExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StreamHealthResponse. */ - interface IStreamHealthResponse { + /** Properties of a ReserveBeginExecuteResponse. */ + interface IReserveBeginExecuteResponse { - /** StreamHealthResponse target */ - target?: (query.ITarget|null); + /** ReserveBeginExecuteResponse error */ + error?: (vtrpc.IRPCError|null); - /** StreamHealthResponse serving */ - serving?: (boolean|null); + /** ReserveBeginExecuteResponse result */ + result?: (query.IQueryResult|null); - /** StreamHealthResponse tablet_externally_reparented_timestamp */ - tablet_externally_reparented_timestamp?: (number|Long|null); + /** ReserveBeginExecuteResponse transaction_id */ + transaction_id?: (number|Long|null); - /** StreamHealthResponse realtime_stats */ - realtime_stats?: (query.IRealtimeStats|null); + /** ReserveBeginExecuteResponse reserved_id */ + reserved_id?: (number|Long|null); - /** StreamHealthResponse tablet_alias */ + /** ReserveBeginExecuteResponse tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); + + /** ReserveBeginExecuteResponse session_state_changes */ + session_state_changes?: (string|null); } - /** Represents a StreamHealthResponse. */ - class StreamHealthResponse implements IStreamHealthResponse { + /** Represents a ReserveBeginExecuteResponse. */ + class ReserveBeginExecuteResponse implements IReserveBeginExecuteResponse { /** - * Constructs a new StreamHealthResponse. + * Constructs a new ReserveBeginExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IStreamHealthResponse); + constructor(properties?: query.IReserveBeginExecuteResponse); - /** StreamHealthResponse target. */ - public target?: (query.ITarget|null); + /** ReserveBeginExecuteResponse error. */ + public error?: (vtrpc.IRPCError|null); - /** StreamHealthResponse serving. */ - public serving: boolean; + /** ReserveBeginExecuteResponse result. */ + public result?: (query.IQueryResult|null); - /** StreamHealthResponse tablet_externally_reparented_timestamp. */ - public tablet_externally_reparented_timestamp: (number|Long); + /** ReserveBeginExecuteResponse transaction_id. */ + public transaction_id: (number|Long); - /** StreamHealthResponse realtime_stats. */ - public realtime_stats?: (query.IRealtimeStats|null); + /** ReserveBeginExecuteResponse reserved_id. */ + public reserved_id: (number|Long); - /** StreamHealthResponse tablet_alias. */ + /** ReserveBeginExecuteResponse tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); + /** ReserveBeginExecuteResponse session_state_changes. */ + public session_state_changes: string; + /** - * Creates a new StreamHealthResponse instance using the specified properties. + * Creates a new ReserveBeginExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns StreamHealthResponse instance + * @returns ReserveBeginExecuteResponse instance */ - public static create(properties?: query.IStreamHealthResponse): query.StreamHealthResponse; + public static create(properties?: query.IReserveBeginExecuteResponse): query.ReserveBeginExecuteResponse; /** - * Encodes the specified StreamHealthResponse message. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. - * @param message StreamHealthResponse message or plain object to encode + * Encodes the specified ReserveBeginExecuteResponse message. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. + * @param message ReserveBeginExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IStreamHealthResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StreamHealthResponse message, length delimited. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. - * @param message StreamHealthResponse message or plain object to encode + * Encodes the specified ReserveBeginExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. + * @param message ReserveBeginExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IStreamHealthResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveBeginExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StreamHealthResponse message from the specified reader or buffer. + * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StreamHealthResponse + * @returns ReserveBeginExecuteResponse * @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): query.StreamHealthResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveBeginExecuteResponse; /** - * Decodes a StreamHealthResponse message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StreamHealthResponse + * @returns ReserveBeginExecuteResponse * @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)): query.StreamHealthResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveBeginExecuteResponse; /** - * Verifies a StreamHealthResponse message. + * Verifies a ReserveBeginExecuteResponse 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 StreamHealthResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StreamHealthResponse + * @returns ReserveBeginExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.StreamHealthResponse; + public static fromObject(object: { [k: string]: any }): query.ReserveBeginExecuteResponse; /** - * Creates a plain object from a StreamHealthResponse message. Also converts values to other types if specified. - * @param message StreamHealthResponse + * Creates a plain object from a ReserveBeginExecuteResponse message. Also converts values to other types if specified. + * @param message ReserveBeginExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.StreamHealthResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveBeginExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StreamHealthResponse to JSON. + * Converts this ReserveBeginExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StreamHealthResponse + * Gets the default type url for ReserveBeginExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** TransactionState enum. */ - enum TransactionState { - UNKNOWN = 0, - PREPARE = 1, - COMMIT = 2, - ROLLBACK = 3 - } + /** Properties of a ReserveBeginStreamExecuteRequest. */ + interface IReserveBeginStreamExecuteRequest { - /** Properties of a TransactionMetadata. */ - interface ITransactionMetadata { + /** ReserveBeginStreamExecuteRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); - /** TransactionMetadata dtid */ - dtid?: (string|null); + /** ReserveBeginStreamExecuteRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); - /** TransactionMetadata state */ - state?: (query.TransactionState|null); + /** ReserveBeginStreamExecuteRequest target */ + target?: (query.ITarget|null); - /** TransactionMetadata time_created */ - time_created?: (number|Long|null); + /** ReserveBeginStreamExecuteRequest query */ + query?: (query.IBoundQuery|null); - /** TransactionMetadata participants */ - participants?: (query.ITarget[]|null); + /** ReserveBeginStreamExecuteRequest options */ + options?: (query.IExecuteOptions|null); + + /** ReserveBeginStreamExecuteRequest pre_queries */ + pre_queries?: (string[]|null); + + /** ReserveBeginStreamExecuteRequest post_begin_queries */ + post_begin_queries?: (string[]|null); } - /** Represents a TransactionMetadata. */ - class TransactionMetadata implements ITransactionMetadata { + /** Represents a ReserveBeginStreamExecuteRequest. */ + class ReserveBeginStreamExecuteRequest implements IReserveBeginStreamExecuteRequest { /** - * Constructs a new TransactionMetadata. + * Constructs a new ReserveBeginStreamExecuteRequest. * @param [properties] Properties to set */ - constructor(properties?: query.ITransactionMetadata); + constructor(properties?: query.IReserveBeginStreamExecuteRequest); - /** TransactionMetadata dtid. */ - public dtid: string; + /** ReserveBeginStreamExecuteRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); - /** TransactionMetadata state. */ - public state: query.TransactionState; + /** ReserveBeginStreamExecuteRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); - /** TransactionMetadata time_created. */ - public time_created: (number|Long); + /** ReserveBeginStreamExecuteRequest target. */ + public target?: (query.ITarget|null); - /** TransactionMetadata participants. */ - public participants: query.ITarget[]; + /** ReserveBeginStreamExecuteRequest query. */ + public query?: (query.IBoundQuery|null); + + /** ReserveBeginStreamExecuteRequest options. */ + public options?: (query.IExecuteOptions|null); + + /** ReserveBeginStreamExecuteRequest pre_queries. */ + public pre_queries: string[]; + + /** ReserveBeginStreamExecuteRequest post_begin_queries. */ + public post_begin_queries: string[]; /** - * Creates a new TransactionMetadata instance using the specified properties. + * Creates a new ReserveBeginStreamExecuteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns TransactionMetadata instance + * @returns ReserveBeginStreamExecuteRequest instance */ - public static create(properties?: query.ITransactionMetadata): query.TransactionMetadata; + public static create(properties?: query.IReserveBeginStreamExecuteRequest): query.ReserveBeginStreamExecuteRequest; /** - * Encodes the specified TransactionMetadata message. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. - * @param message TransactionMetadata message or plain object to encode + * Encodes the specified ReserveBeginStreamExecuteRequest message. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. + * @param message ReserveBeginStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.ITransactionMetadata, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TransactionMetadata message, length delimited. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. - * @param message TransactionMetadata message or plain object to encode + * Encodes the specified ReserveBeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. + * @param message ReserveBeginStreamExecuteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.ITransactionMetadata, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveBeginStreamExecuteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TransactionMetadata message from the specified reader or buffer. + * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TransactionMetadata + * @returns ReserveBeginStreamExecuteRequest * @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): query.TransactionMetadata; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveBeginStreamExecuteRequest; /** - * Decodes a TransactionMetadata message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TransactionMetadata + * @returns ReserveBeginStreamExecuteRequest * @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)): query.TransactionMetadata; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveBeginStreamExecuteRequest; /** - * Verifies a TransactionMetadata message. + * Verifies a ReserveBeginStreamExecuteRequest 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 TransactionMetadata message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TransactionMetadata + * @returns ReserveBeginStreamExecuteRequest */ - public static fromObject(object: { [k: string]: any }): query.TransactionMetadata; + public static fromObject(object: { [k: string]: any }): query.ReserveBeginStreamExecuteRequest; /** - * Creates a plain object from a TransactionMetadata message. Also converts values to other types if specified. - * @param message TransactionMetadata + * Creates a plain object from a ReserveBeginStreamExecuteRequest message. Also converts values to other types if specified. + * @param message ReserveBeginStreamExecuteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.TransactionMetadata, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveBeginStreamExecuteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TransactionMetadata to JSON. + * Converts this ReserveBeginStreamExecuteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TransactionMetadata + * Gets the default type url for ReserveBeginStreamExecuteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** SchemaTableType enum. */ - enum SchemaTableType { - VIEWS = 0, - TABLES = 1, - ALL = 2 - } + /** Properties of a ReserveBeginStreamExecuteResponse. */ + interface IReserveBeginStreamExecuteResponse { - /** Properties of a GetSchemaRequest. */ - interface IGetSchemaRequest { + /** ReserveBeginStreamExecuteResponse error */ + error?: (vtrpc.IRPCError|null); - /** GetSchemaRequest target */ - target?: (query.ITarget|null); + /** ReserveBeginStreamExecuteResponse result */ + result?: (query.IQueryResult|null); - /** GetSchemaRequest table_type */ - table_type?: (query.SchemaTableType|null); + /** ReserveBeginStreamExecuteResponse transaction_id */ + transaction_id?: (number|Long|null); - /** GetSchemaRequest table_names */ - table_names?: (string[]|null); + /** ReserveBeginStreamExecuteResponse reserved_id */ + reserved_id?: (number|Long|null); + + /** ReserveBeginStreamExecuteResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** ReserveBeginStreamExecuteResponse session_state_changes */ + session_state_changes?: (string|null); } - /** Represents a GetSchemaRequest. */ - class GetSchemaRequest implements IGetSchemaRequest { + /** Represents a ReserveBeginStreamExecuteResponse. */ + class ReserveBeginStreamExecuteResponse implements IReserveBeginStreamExecuteResponse { /** - * Constructs a new GetSchemaRequest. + * Constructs a new ReserveBeginStreamExecuteResponse. * @param [properties] Properties to set */ - constructor(properties?: query.IGetSchemaRequest); + constructor(properties?: query.IReserveBeginStreamExecuteResponse); - /** GetSchemaRequest target. */ - public target?: (query.ITarget|null); + /** ReserveBeginStreamExecuteResponse error. */ + public error?: (vtrpc.IRPCError|null); - /** GetSchemaRequest table_type. */ - public table_type: query.SchemaTableType; + /** ReserveBeginStreamExecuteResponse result. */ + public result?: (query.IQueryResult|null); - /** GetSchemaRequest table_names. */ - public table_names: string[]; + /** ReserveBeginStreamExecuteResponse transaction_id. */ + public transaction_id: (number|Long); + + /** ReserveBeginStreamExecuteResponse reserved_id. */ + public reserved_id: (number|Long); + + /** ReserveBeginStreamExecuteResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** ReserveBeginStreamExecuteResponse session_state_changes. */ + public session_state_changes: string; /** - * Creates a new GetSchemaRequest instance using the specified properties. + * Creates a new ReserveBeginStreamExecuteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetSchemaRequest instance + * @returns ReserveBeginStreamExecuteResponse instance */ - public static create(properties?: query.IGetSchemaRequest): query.GetSchemaRequest; + public static create(properties?: query.IReserveBeginStreamExecuteResponse): query.ReserveBeginStreamExecuteResponse; /** - * Encodes the specified GetSchemaRequest message. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. - * @param message GetSchemaRequest message or plain object to encode + * Encodes the specified ReserveBeginStreamExecuteResponse message. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. + * @param message ReserveBeginStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReserveBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. - * @param message GetSchemaRequest message or plain object to encode + * Encodes the specified ReserveBeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. + * @param message ReserveBeginStreamExecuteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReserveBeginStreamExecuteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer. + * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSchemaRequest + * @returns ReserveBeginStreamExecuteResponse * @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): query.GetSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReserveBeginStreamExecuteResponse; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSchemaRequest + * @returns ReserveBeginStreamExecuteResponse * @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)): query.GetSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReserveBeginStreamExecuteResponse; /** - * Verifies a GetSchemaRequest message. + * Verifies a ReserveBeginStreamExecuteResponse 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 GetSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSchemaRequest + * @returns ReserveBeginStreamExecuteResponse */ - public static fromObject(object: { [k: string]: any }): query.GetSchemaRequest; + public static fromObject(object: { [k: string]: any }): query.ReserveBeginStreamExecuteResponse; /** - * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. - * @param message GetSchemaRequest + * Creates a plain object from a ReserveBeginStreamExecuteResponse message. Also converts values to other types if specified. + * @param message ReserveBeginStreamExecuteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.GetSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReserveBeginStreamExecuteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSchemaRequest to JSON. + * Converts this ReserveBeginStreamExecuteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSchemaRequest + * Gets the default type url for ReserveBeginStreamExecuteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSchemaResponse. */ - interface IGetSchemaResponse { + /** Properties of a ReleaseRequest. */ + interface IReleaseRequest { - /** GetSchemaResponse table_definition */ - table_definition?: ({ [k: string]: string }|null); + /** ReleaseRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); + + /** ReleaseRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** ReleaseRequest target */ + target?: (query.ITarget|null); + + /** ReleaseRequest transaction_id */ + transaction_id?: (number|Long|null); + + /** ReleaseRequest reserved_id */ + reserved_id?: (number|Long|null); } - /** Represents a GetSchemaResponse. */ - class GetSchemaResponse implements IGetSchemaResponse { + /** Represents a ReleaseRequest. */ + class ReleaseRequest implements IReleaseRequest { /** - * Constructs a new GetSchemaResponse. + * Constructs a new ReleaseRequest. * @param [properties] Properties to set */ - constructor(properties?: query.IGetSchemaResponse); + constructor(properties?: query.IReleaseRequest); - /** GetSchemaResponse table_definition. */ - public table_definition: { [k: string]: string }; + /** ReleaseRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); + + /** ReleaseRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** ReleaseRequest target. */ + public target?: (query.ITarget|null); + + /** ReleaseRequest transaction_id. */ + public transaction_id: (number|Long); + + /** ReleaseRequest reserved_id. */ + public reserved_id: (number|Long); /** - * Creates a new GetSchemaResponse instance using the specified properties. + * Creates a new ReleaseRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSchemaResponse instance + * @returns ReleaseRequest instance */ - public static create(properties?: query.IGetSchemaResponse): query.GetSchemaResponse; + public static create(properties?: query.IReleaseRequest): query.ReleaseRequest; /** - * Encodes the specified GetSchemaResponse message. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. - * @param message GetSchemaResponse message or plain object to encode + * Encodes the specified ReleaseRequest message. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. + * @param message ReleaseRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: query.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IReleaseRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. - * @param message GetSchemaResponse message or plain object to encode + * Encodes the specified ReleaseRequest message, length delimited. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. + * @param message ReleaseRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: query.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IReleaseRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer. + * Decodes a ReleaseRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSchemaResponse + * @returns ReleaseRequest * @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): query.GetSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.ReleaseRequest; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a ReleaseRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSchemaResponse + * @returns ReleaseRequest * @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)): query.GetSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.ReleaseRequest; /** - * Verifies a GetSchemaResponse message. + * Verifies a ReleaseRequest 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 GetSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReleaseRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSchemaResponse + * @returns ReleaseRequest */ - public static fromObject(object: { [k: string]: any }): query.GetSchemaResponse; + public static fromObject(object: { [k: string]: any }): query.ReleaseRequest; /** - * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. - * @param message GetSchemaResponse + * Creates a plain object from a ReleaseRequest message. Also converts values to other types if specified. + * @param message ReleaseRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: query.GetSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.ReleaseRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSchemaResponse to JSON. + * Converts this ReleaseRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSchemaResponse + * Gets the default type url for ReleaseRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} -/** Namespace replicationdata. */ -export namespace replicationdata { - - /** Properties of a Status. */ - interface IStatus { + /** Properties of a ReleaseResponse. */ + interface IReleaseResponse { + } - /** Status position */ - position?: (string|null); + /** Represents a ReleaseResponse. */ + class ReleaseResponse implements IReleaseResponse { - /** Status replication_lag_seconds */ - replication_lag_seconds?: (number|null); + /** + * Constructs a new ReleaseResponse. + * @param [properties] Properties to set + */ + constructor(properties?: query.IReleaseResponse); - /** Status source_host */ - source_host?: (string|null); + /** + * Creates a new ReleaseResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ReleaseResponse instance + */ + public static create(properties?: query.IReleaseResponse): query.ReleaseResponse; - /** Status source_port */ - source_port?: (number|null); + /** + * Encodes the specified ReleaseResponse message. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * @param message ReleaseResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.IReleaseResponse, writer?: $protobuf.Writer): $protobuf.Writer; - /** Status connect_retry */ - connect_retry?: (number|null); + /** + * Encodes the specified ReleaseResponse message, length delimited. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * @param message ReleaseResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: query.IReleaseResponse, writer?: $protobuf.Writer): $protobuf.Writer; - /** Status relay_log_position */ - relay_log_position?: (string|null); + /** + * Decodes a ReleaseResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ReleaseResponse + * @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): query.ReleaseResponse; - /** Status file_position */ - file_position?: (string|null); + /** + * Decodes a ReleaseResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ReleaseResponse + * @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)): query.ReleaseResponse; - /** Status relay_log_source_binlog_equivalent_position */ - relay_log_source_binlog_equivalent_position?: (string|null); + /** + * Verifies a ReleaseResponse 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); - /** Status source_server_id */ - source_server_id?: (number|null); + /** + * Creates a ReleaseResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ReleaseResponse + */ + public static fromObject(object: { [k: string]: any }): query.ReleaseResponse; - /** Status source_uuid */ - source_uuid?: (string|null); + /** + * Creates a plain object from a ReleaseResponse message. Also converts values to other types if specified. + * @param message ReleaseResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: query.ReleaseResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; - /** Status io_state */ - io_state?: (number|null); + /** + * Converts this ReleaseResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; - /** Status last_io_error */ - last_io_error?: (string|null); + /** + * Gets the default type url for ReleaseResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } - /** Status sql_state */ - sql_state?: (number|null); - - /** Status last_sql_error */ - last_sql_error?: (string|null); - - /** Status relay_log_file_position */ - relay_log_file_position?: (string|null); + /** Properties of a StreamHealthRequest. */ + interface IStreamHealthRequest { + } - /** Status source_user */ - source_user?: (string|null); + /** Represents a StreamHealthRequest. */ + class StreamHealthRequest implements IStreamHealthRequest { - /** Status sql_delay */ - sql_delay?: (number|null); + /** + * Constructs a new StreamHealthRequest. + * @param [properties] Properties to set + */ + constructor(properties?: query.IStreamHealthRequest); - /** Status auto_position */ - auto_position?: (boolean|null); + /** + * Creates a new StreamHealthRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns StreamHealthRequest instance + */ + public static create(properties?: query.IStreamHealthRequest): query.StreamHealthRequest; - /** Status using_gtid */ - using_gtid?: (boolean|null); + /** + * Encodes the specified StreamHealthRequest message. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * @param message StreamHealthRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.IStreamHealthRequest, writer?: $protobuf.Writer): $protobuf.Writer; - /** Status has_replication_filters */ - has_replication_filters?: (boolean|null); + /** + * Encodes the specified StreamHealthRequest message, length delimited. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * @param message StreamHealthRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: query.IStreamHealthRequest, writer?: $protobuf.Writer): $protobuf.Writer; - /** Status ssl_allowed */ - ssl_allowed?: (boolean|null); + /** + * Decodes a StreamHealthRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns StreamHealthRequest + * @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): query.StreamHealthRequest; - /** Status replication_lag_unknown */ - replication_lag_unknown?: (boolean|null); - } + /** + * Decodes a StreamHealthRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns StreamHealthRequest + * @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)): query.StreamHealthRequest; - /** Represents a Status. */ - class Status implements IStatus { + /** + * Verifies a StreamHealthRequest 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); /** - * Constructs a new Status. - * @param [properties] Properties to set + * Creates a StreamHealthRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns StreamHealthRequest */ - constructor(properties?: replicationdata.IStatus); + public static fromObject(object: { [k: string]: any }): query.StreamHealthRequest; - /** Status position. */ - public position: string; + /** + * Creates a plain object from a StreamHealthRequest message. Also converts values to other types if specified. + * @param message StreamHealthRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: query.StreamHealthRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; - /** Status replication_lag_seconds. */ - public replication_lag_seconds: number; + /** + * Converts this StreamHealthRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; - /** Status source_host. */ - public source_host: string; + /** + * Gets the default type url for StreamHealthRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } - /** Status source_port. */ - public source_port: number; + /** Properties of a RealtimeStats. */ + interface IRealtimeStats { - /** Status connect_retry. */ - public connect_retry: number; + /** RealtimeStats health_error */ + health_error?: (string|null); - /** Status relay_log_position. */ - public relay_log_position: string; + /** RealtimeStats replication_lag_seconds */ + replication_lag_seconds?: (number|null); - /** Status file_position. */ - public file_position: string; + /** RealtimeStats binlog_players_count */ + binlog_players_count?: (number|null); - /** Status relay_log_source_binlog_equivalent_position. */ - public relay_log_source_binlog_equivalent_position: string; + /** RealtimeStats filtered_replication_lag_seconds */ + filtered_replication_lag_seconds?: (number|Long|null); - /** Status source_server_id. */ - public source_server_id: number; + /** RealtimeStats cpu_usage */ + cpu_usage?: (number|null); - /** Status source_uuid. */ - public source_uuid: string; + /** RealtimeStats qps */ + qps?: (number|null); - /** Status io_state. */ - public io_state: number; + /** RealtimeStats table_schema_changed */ + table_schema_changed?: (string[]|null); - /** Status last_io_error. */ - public last_io_error: string; + /** RealtimeStats view_schema_changed */ + view_schema_changed?: (string[]|null); + } - /** Status sql_state. */ - public sql_state: number; + /** Represents a RealtimeStats. */ + class RealtimeStats implements IRealtimeStats { - /** Status last_sql_error. */ - public last_sql_error: string; + /** + * Constructs a new RealtimeStats. + * @param [properties] Properties to set + */ + constructor(properties?: query.IRealtimeStats); - /** Status relay_log_file_position. */ - public relay_log_file_position: string; + /** RealtimeStats health_error. */ + public health_error: string; - /** Status source_user. */ - public source_user: string; + /** RealtimeStats replication_lag_seconds. */ + public replication_lag_seconds: number; - /** Status sql_delay. */ - public sql_delay: number; + /** RealtimeStats binlog_players_count. */ + public binlog_players_count: number; - /** Status auto_position. */ - public auto_position: boolean; + /** RealtimeStats filtered_replication_lag_seconds. */ + public filtered_replication_lag_seconds: (number|Long); - /** Status using_gtid. */ - public using_gtid: boolean; + /** RealtimeStats cpu_usage. */ + public cpu_usage: number; - /** Status has_replication_filters. */ - public has_replication_filters: boolean; + /** RealtimeStats qps. */ + public qps: number; - /** Status ssl_allowed. */ - public ssl_allowed: boolean; + /** RealtimeStats table_schema_changed. */ + public table_schema_changed: string[]; - /** Status replication_lag_unknown. */ - public replication_lag_unknown: boolean; + /** RealtimeStats view_schema_changed. */ + public view_schema_changed: string[]; /** - * Creates a new Status instance using the specified properties. + * Creates a new RealtimeStats instance using the specified properties. * @param [properties] Properties to set - * @returns Status instance + * @returns RealtimeStats instance */ - public static create(properties?: replicationdata.IStatus): replicationdata.Status; + public static create(properties?: query.IRealtimeStats): query.RealtimeStats; /** - * Encodes the specified Status message. Does not implicitly {@link replicationdata.Status.verify|verify} messages. - * @param message Status message or plain object to encode + * Encodes the specified RealtimeStats message. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. + * @param message RealtimeStats message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: replicationdata.IStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IRealtimeStats, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Status message, length delimited. Does not implicitly {@link replicationdata.Status.verify|verify} messages. - * @param message Status message or plain object to encode + * Encodes the specified RealtimeStats message, length delimited. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. + * @param message RealtimeStats message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: replicationdata.IStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IRealtimeStats, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Status message from the specified reader or buffer. + * Decodes a RealtimeStats message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Status + * @returns RealtimeStats * @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): replicationdata.Status; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.RealtimeStats; /** - * Decodes a Status message from the specified reader or buffer, length delimited. + * Decodes a RealtimeStats message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Status + * @returns RealtimeStats * @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)): replicationdata.Status; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.RealtimeStats; /** - * Verifies a Status message. + * Verifies a RealtimeStats 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 Status message from a plain object. Also converts values to their respective internal types. + * Creates a RealtimeStats message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Status + * @returns RealtimeStats */ - public static fromObject(object: { [k: string]: any }): replicationdata.Status; + public static fromObject(object: { [k: string]: any }): query.RealtimeStats; /** - * Creates a plain object from a Status message. Also converts values to other types if specified. - * @param message Status + * Creates a plain object from a RealtimeStats message. Also converts values to other types if specified. + * @param message RealtimeStats * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: replicationdata.Status, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.RealtimeStats, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Status to JSON. + * Converts this RealtimeStats to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Status + * Gets the default type url for RealtimeStats * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StopReplicationStatus. */ - interface IStopReplicationStatus { + /** Properties of an AggregateStats. */ + interface IAggregateStats { - /** StopReplicationStatus before */ - before?: (replicationdata.IStatus|null); + /** AggregateStats healthy_tablet_count */ + healthy_tablet_count?: (number|null); - /** StopReplicationStatus after */ - after?: (replicationdata.IStatus|null); + /** AggregateStats unhealthy_tablet_count */ + unhealthy_tablet_count?: (number|null); + + /** AggregateStats replication_lag_seconds_min */ + replication_lag_seconds_min?: (number|null); + + /** AggregateStats replication_lag_seconds_max */ + replication_lag_seconds_max?: (number|null); } - /** Represents a StopReplicationStatus. */ - class StopReplicationStatus implements IStopReplicationStatus { + /** Represents an AggregateStats. */ + class AggregateStats implements IAggregateStats { /** - * Constructs a new StopReplicationStatus. + * Constructs a new AggregateStats. * @param [properties] Properties to set */ - constructor(properties?: replicationdata.IStopReplicationStatus); + constructor(properties?: query.IAggregateStats); - /** StopReplicationStatus before. */ - public before?: (replicationdata.IStatus|null); + /** AggregateStats healthy_tablet_count. */ + public healthy_tablet_count: number; - /** StopReplicationStatus after. */ - public after?: (replicationdata.IStatus|null); + /** AggregateStats unhealthy_tablet_count. */ + public unhealthy_tablet_count: number; + + /** AggregateStats replication_lag_seconds_min. */ + public replication_lag_seconds_min: number; + + /** AggregateStats replication_lag_seconds_max. */ + public replication_lag_seconds_max: number; /** - * Creates a new StopReplicationStatus instance using the specified properties. + * Creates a new AggregateStats instance using the specified properties. * @param [properties] Properties to set - * @returns StopReplicationStatus instance + * @returns AggregateStats instance */ - public static create(properties?: replicationdata.IStopReplicationStatus): replicationdata.StopReplicationStatus; + public static create(properties?: query.IAggregateStats): query.AggregateStats; /** - * Encodes the specified StopReplicationStatus message. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. - * @param message StopReplicationStatus message or plain object to encode + * Encodes the specified AggregateStats message. Does not implicitly {@link query.AggregateStats.verify|verify} messages. + * @param message AggregateStats message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: replicationdata.IStopReplicationStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IAggregateStats, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StopReplicationStatus message, length delimited. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. - * @param message StopReplicationStatus message or plain object to encode + * Encodes the specified AggregateStats message, length delimited. Does not implicitly {@link query.AggregateStats.verify|verify} messages. + * @param message AggregateStats message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: replicationdata.IStopReplicationStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IAggregateStats, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StopReplicationStatus message from the specified reader or buffer. + * Decodes an AggregateStats message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StopReplicationStatus + * @returns AggregateStats * @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): replicationdata.StopReplicationStatus; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.AggregateStats; /** - * Decodes a StopReplicationStatus message from the specified reader or buffer, length delimited. + * Decodes an AggregateStats message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StopReplicationStatus + * @returns AggregateStats * @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)): replicationdata.StopReplicationStatus; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.AggregateStats; /** - * Verifies a StopReplicationStatus message. + * Verifies an AggregateStats 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 StopReplicationStatus message from a plain object. Also converts values to their respective internal types. + * Creates an AggregateStats message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StopReplicationStatus + * @returns AggregateStats */ - public static fromObject(object: { [k: string]: any }): replicationdata.StopReplicationStatus; + public static fromObject(object: { [k: string]: any }): query.AggregateStats; /** - * Creates a plain object from a StopReplicationStatus message. Also converts values to other types if specified. - * @param message StopReplicationStatus + * Creates a plain object from an AggregateStats message. Also converts values to other types if specified. + * @param message AggregateStats * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: replicationdata.StopReplicationStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.AggregateStats, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StopReplicationStatus to JSON. + * Converts this AggregateStats to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StopReplicationStatus + * Gets the default type url for AggregateStats * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** StopReplicationMode enum. */ - enum StopReplicationMode { - IOANDSQLTHREAD = 0, - IOTHREADONLY = 1 - } + /** Properties of a StreamHealthResponse. */ + interface IStreamHealthResponse { - /** Properties of a PrimaryStatus. */ - interface IPrimaryStatus { + /** StreamHealthResponse target */ + target?: (query.ITarget|null); - /** PrimaryStatus position */ - position?: (string|null); + /** StreamHealthResponse serving */ + serving?: (boolean|null); - /** PrimaryStatus file_position */ - file_position?: (string|null); + /** StreamHealthResponse tablet_externally_reparented_timestamp */ + tablet_externally_reparented_timestamp?: (number|Long|null); + + /** StreamHealthResponse realtime_stats */ + realtime_stats?: (query.IRealtimeStats|null); + + /** StreamHealthResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a PrimaryStatus. */ - class PrimaryStatus implements IPrimaryStatus { + /** Represents a StreamHealthResponse. */ + class StreamHealthResponse implements IStreamHealthResponse { /** - * Constructs a new PrimaryStatus. + * Constructs a new StreamHealthResponse. * @param [properties] Properties to set */ - constructor(properties?: replicationdata.IPrimaryStatus); + constructor(properties?: query.IStreamHealthResponse); - /** PrimaryStatus position. */ - public position: string; + /** StreamHealthResponse target. */ + public target?: (query.ITarget|null); - /** PrimaryStatus file_position. */ - public file_position: string; + /** StreamHealthResponse serving. */ + public serving: boolean; + + /** StreamHealthResponse tablet_externally_reparented_timestamp. */ + public tablet_externally_reparented_timestamp: (number|Long); + + /** StreamHealthResponse realtime_stats. */ + public realtime_stats?: (query.IRealtimeStats|null); + + /** StreamHealthResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new PrimaryStatus instance using the specified properties. + * Creates a new StreamHealthResponse instance using the specified properties. * @param [properties] Properties to set - * @returns PrimaryStatus instance + * @returns StreamHealthResponse instance */ - public static create(properties?: replicationdata.IPrimaryStatus): replicationdata.PrimaryStatus; + public static create(properties?: query.IStreamHealthResponse): query.StreamHealthResponse; /** - * Encodes the specified PrimaryStatus message. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. - * @param message PrimaryStatus message or plain object to encode + * Encodes the specified StreamHealthResponse message. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. + * @param message StreamHealthResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: replicationdata.IPrimaryStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IStreamHealthResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PrimaryStatus message, length delimited. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. - * @param message PrimaryStatus message or plain object to encode + * Encodes the specified StreamHealthResponse message, length delimited. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. + * @param message StreamHealthResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: replicationdata.IPrimaryStatus, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IStreamHealthResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PrimaryStatus message from the specified reader or buffer. + * Decodes a StreamHealthResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PrimaryStatus + * @returns StreamHealthResponse * @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): replicationdata.PrimaryStatus; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.StreamHealthResponse; /** - * Decodes a PrimaryStatus message from the specified reader or buffer, length delimited. + * Decodes a StreamHealthResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PrimaryStatus + * @returns StreamHealthResponse * @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)): replicationdata.PrimaryStatus; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.StreamHealthResponse; /** - * Verifies a PrimaryStatus message. + * Verifies a StreamHealthResponse 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 PrimaryStatus message from a plain object. Also converts values to their respective internal types. + * Creates a StreamHealthResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PrimaryStatus + * @returns StreamHealthResponse */ - public static fromObject(object: { [k: string]: any }): replicationdata.PrimaryStatus; + public static fromObject(object: { [k: string]: any }): query.StreamHealthResponse; /** - * Creates a plain object from a PrimaryStatus message. Also converts values to other types if specified. - * @param message PrimaryStatus + * Creates a plain object from a StreamHealthResponse message. Also converts values to other types if specified. + * @param message StreamHealthResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: replicationdata.PrimaryStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.StreamHealthResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PrimaryStatus to JSON. + * Converts this StreamHealthResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PrimaryStatus + * Gets the default type url for StreamHealthResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a FullStatus. */ - interface IFullStatus { - - /** FullStatus server_id */ - server_id?: (number|null); - - /** FullStatus server_uuid */ - server_uuid?: (string|null); + /** TransactionState enum. */ + enum TransactionState { + UNKNOWN = 0, + PREPARE = 1, + COMMIT = 2, + ROLLBACK = 3 + } - /** FullStatus replication_status */ - replication_status?: (replicationdata.IStatus|null); + /** Properties of a TransactionMetadata. */ + interface ITransactionMetadata { - /** FullStatus primary_status */ - primary_status?: (replicationdata.IPrimaryStatus|null); + /** TransactionMetadata dtid */ + dtid?: (string|null); - /** FullStatus gtid_purged */ - gtid_purged?: (string|null); + /** TransactionMetadata state */ + state?: (query.TransactionState|null); - /** FullStatus version */ - version?: (string|null); + /** TransactionMetadata time_created */ + time_created?: (number|Long|null); - /** FullStatus version_comment */ - version_comment?: (string|null); + /** TransactionMetadata participants */ + participants?: (query.ITarget[]|null); + } - /** FullStatus read_only */ - read_only?: (boolean|null); + /** Represents a TransactionMetadata. */ + class TransactionMetadata implements ITransactionMetadata { - /** FullStatus gtid_mode */ - gtid_mode?: (string|null); + /** + * Constructs a new TransactionMetadata. + * @param [properties] Properties to set + */ + constructor(properties?: query.ITransactionMetadata); - /** FullStatus binlog_format */ - binlog_format?: (string|null); + /** TransactionMetadata dtid. */ + public dtid: string; - /** FullStatus binlog_row_image */ - binlog_row_image?: (string|null); + /** TransactionMetadata state. */ + public state: query.TransactionState; - /** FullStatus log_bin_enabled */ - log_bin_enabled?: (boolean|null); + /** TransactionMetadata time_created. */ + public time_created: (number|Long); - /** FullStatus log_replica_updates */ - log_replica_updates?: (boolean|null); + /** TransactionMetadata participants. */ + public participants: query.ITarget[]; - /** FullStatus semi_sync_primary_enabled */ - semi_sync_primary_enabled?: (boolean|null); + /** + * Creates a new TransactionMetadata instance using the specified properties. + * @param [properties] Properties to set + * @returns TransactionMetadata instance + */ + public static create(properties?: query.ITransactionMetadata): query.TransactionMetadata; - /** FullStatus semi_sync_replica_enabled */ - semi_sync_replica_enabled?: (boolean|null); + /** + * Encodes the specified TransactionMetadata message. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. + * @param message TransactionMetadata message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: query.ITransactionMetadata, writer?: $protobuf.Writer): $protobuf.Writer; - /** FullStatus semi_sync_primary_status */ - semi_sync_primary_status?: (boolean|null); - - /** FullStatus semi_sync_replica_status */ - semi_sync_replica_status?: (boolean|null); - - /** FullStatus semi_sync_primary_clients */ - semi_sync_primary_clients?: (number|null); - - /** FullStatus semi_sync_primary_timeout */ - semi_sync_primary_timeout?: (number|Long|null); - - /** FullStatus semi_sync_wait_for_replica_count */ - semi_sync_wait_for_replica_count?: (number|null); - - /** FullStatus super_read_only */ - super_read_only?: (boolean|null); - } - - /** Represents a FullStatus. */ - class FullStatus implements IFullStatus { - - /** - * Constructs a new FullStatus. - * @param [properties] Properties to set - */ - constructor(properties?: replicationdata.IFullStatus); - - /** FullStatus server_id. */ - public server_id: number; - - /** FullStatus server_uuid. */ - public server_uuid: string; - - /** FullStatus replication_status. */ - public replication_status?: (replicationdata.IStatus|null); - - /** FullStatus primary_status. */ - public primary_status?: (replicationdata.IPrimaryStatus|null); - - /** FullStatus gtid_purged. */ - public gtid_purged: string; - - /** FullStatus version. */ - public version: string; - - /** FullStatus version_comment. */ - public version_comment: string; - - /** FullStatus read_only. */ - public read_only: boolean; - - /** FullStatus gtid_mode. */ - public gtid_mode: string; - - /** FullStatus binlog_format. */ - public binlog_format: string; - - /** FullStatus binlog_row_image. */ - public binlog_row_image: string; - - /** FullStatus log_bin_enabled. */ - public log_bin_enabled: boolean; - - /** FullStatus log_replica_updates. */ - public log_replica_updates: boolean; - - /** FullStatus semi_sync_primary_enabled. */ - public semi_sync_primary_enabled: boolean; - - /** FullStatus semi_sync_replica_enabled. */ - public semi_sync_replica_enabled: boolean; - - /** FullStatus semi_sync_primary_status. */ - public semi_sync_primary_status: boolean; - - /** FullStatus semi_sync_replica_status. */ - public semi_sync_replica_status: boolean; - - /** FullStatus semi_sync_primary_clients. */ - public semi_sync_primary_clients: number; - - /** FullStatus semi_sync_primary_timeout. */ - public semi_sync_primary_timeout: (number|Long); - - /** FullStatus semi_sync_wait_for_replica_count. */ - public semi_sync_wait_for_replica_count: number; - - /** FullStatus super_read_only. */ - public super_read_only: boolean; - - /** - * Creates a new FullStatus instance using the specified properties. - * @param [properties] Properties to set - * @returns FullStatus instance - */ - public static create(properties?: replicationdata.IFullStatus): replicationdata.FullStatus; - - /** - * Encodes the specified FullStatus message. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. - * @param message FullStatus message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: replicationdata.IFullStatus, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified FullStatus message, length delimited. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. - * @param message FullStatus message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: replicationdata.IFullStatus, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Encodes the specified TransactionMetadata message, length delimited. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. + * @param message TransactionMetadata message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: query.ITransactionMetadata, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a FullStatus message from the specified reader or buffer. + * Decodes a TransactionMetadata message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns FullStatus + * @returns TransactionMetadata * @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): replicationdata.FullStatus; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.TransactionMetadata; /** - * Decodes a FullStatus message from the specified reader or buffer, length delimited. + * Decodes a TransactionMetadata message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns FullStatus + * @returns TransactionMetadata * @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)): replicationdata.FullStatus; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.TransactionMetadata; /** - * Verifies a FullStatus message. + * Verifies a TransactionMetadata 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 FullStatus message from a plain object. Also converts values to their respective internal types. + * Creates a TransactionMetadata message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns FullStatus + * @returns TransactionMetadata */ - public static fromObject(object: { [k: string]: any }): replicationdata.FullStatus; + public static fromObject(object: { [k: string]: any }): query.TransactionMetadata; /** - * Creates a plain object from a FullStatus message. Also converts values to other types if specified. - * @param message FullStatus + * Creates a plain object from a TransactionMetadata message. Also converts values to other types if specified. + * @param message TransactionMetadata * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: replicationdata.FullStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.TransactionMetadata, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this FullStatus to JSON. + * Converts this TransactionMetadata to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for FullStatus + * Gets the default type url for TransactionMetadata * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} -/** Namespace vschema. */ -export namespace vschema { + /** SchemaTableType enum. */ + enum SchemaTableType { + VIEWS = 0, + TABLES = 1, + ALL = 2 + } - /** Properties of a RoutingRules. */ - interface IRoutingRules { + /** Properties of a GetSchemaRequest. */ + interface IGetSchemaRequest { - /** RoutingRules rules */ - rules?: (vschema.IRoutingRule[]|null); + /** GetSchemaRequest target */ + target?: (query.ITarget|null); + + /** GetSchemaRequest table_type */ + table_type?: (query.SchemaTableType|null); + + /** GetSchemaRequest table_names */ + table_names?: (string[]|null); } - /** Represents a RoutingRules. */ - class RoutingRules implements IRoutingRules { + /** Represents a GetSchemaRequest. */ + class GetSchemaRequest implements IGetSchemaRequest { /** - * Constructs a new RoutingRules. + * Constructs a new GetSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vschema.IRoutingRules); + constructor(properties?: query.IGetSchemaRequest); - /** RoutingRules rules. */ - public rules: vschema.IRoutingRule[]; + /** GetSchemaRequest target. */ + public target?: (query.ITarget|null); + + /** GetSchemaRequest table_type. */ + public table_type: query.SchemaTableType; + + /** GetSchemaRequest table_names. */ + public table_names: string[]; /** - * Creates a new RoutingRules instance using the specified properties. + * Creates a new GetSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RoutingRules instance + * @returns GetSchemaRequest instance */ - public static create(properties?: vschema.IRoutingRules): vschema.RoutingRules; + public static create(properties?: query.IGetSchemaRequest): query.GetSchemaRequest; /** - * Encodes the specified RoutingRules message. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. - * @param message RoutingRules message or plain object to encode + * Encodes the specified GetSchemaRequest message. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. + * @param message GetSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RoutingRules message, length delimited. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. - * @param message RoutingRules message or plain object to encode + * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. + * @param message GetSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RoutingRules message from the specified reader or buffer. + * Decodes a GetSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RoutingRules + * @returns GetSchemaRequest * @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.RoutingRules; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.GetSchemaRequest; /** - * Decodes a RoutingRules message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RoutingRules + * @returns GetSchemaRequest * @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.RoutingRules; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.GetSchemaRequest; /** - * Verifies a RoutingRules message. + * Verifies a GetSchemaRequest 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 RoutingRules message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RoutingRules + * @returns GetSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vschema.RoutingRules; + public static fromObject(object: { [k: string]: any }): query.GetSchemaRequest; /** - * Creates a plain object from a RoutingRules message. Also converts values to other types if specified. - * @param message RoutingRules + * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. + * @param message GetSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.RoutingRules, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.GetSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RoutingRules to JSON. + * Converts this GetSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RoutingRules + * Gets the default type url for GetSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RoutingRule. */ - interface IRoutingRule { - - /** RoutingRule from_table */ - from_table?: (string|null); + /** Properties of a GetSchemaResponse. */ + interface IGetSchemaResponse { - /** RoutingRule to_tables */ - to_tables?: (string[]|null); + /** GetSchemaResponse table_definition */ + table_definition?: ({ [k: string]: string }|null); } - /** Represents a RoutingRule. */ - class RoutingRule implements IRoutingRule { + /** Represents a GetSchemaResponse. */ + class GetSchemaResponse implements IGetSchemaResponse { /** - * Constructs a new RoutingRule. + * Constructs a new GetSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vschema.IRoutingRule); - - /** RoutingRule from_table. */ - public from_table: string; + constructor(properties?: query.IGetSchemaResponse); - /** RoutingRule to_tables. */ - public to_tables: string[]; + /** GetSchemaResponse table_definition. */ + public table_definition: { [k: string]: string }; /** - * Creates a new RoutingRule instance using the specified properties. + * Creates a new GetSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RoutingRule instance + * @returns GetSchemaResponse instance */ - public static create(properties?: vschema.IRoutingRule): vschema.RoutingRule; + public static create(properties?: query.IGetSchemaResponse): query.GetSchemaResponse; /** - * Encodes the specified RoutingRule message. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. - * @param message RoutingRule message or plain object to encode + * Encodes the specified GetSchemaResponse message. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. + * @param message GetSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: query.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RoutingRule message, length delimited. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. - * @param message RoutingRule message or plain object to encode + * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. + * @param message GetSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: query.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RoutingRule message from the specified reader or buffer. + * Decodes a GetSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RoutingRule + * @returns GetSchemaResponse * @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.RoutingRule; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): query.GetSchemaResponse; /** - * Decodes a RoutingRule message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RoutingRule + * @returns GetSchemaResponse * @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.RoutingRule; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): query.GetSchemaResponse; /** - * Verifies a RoutingRule message. + * Verifies a GetSchemaResponse 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 RoutingRule message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RoutingRule + * @returns GetSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vschema.RoutingRule; + public static fromObject(object: { [k: string]: any }): query.GetSchemaResponse; /** - * Creates a plain object from a RoutingRule message. Also converts values to other types if specified. - * @param message RoutingRule + * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. + * @param message GetSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.RoutingRule, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: query.GetSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RoutingRule to JSON. + * Converts this GetSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RoutingRule + * Gets the default type url for GetSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } +} - /** Properties of a Keyspace. */ - interface IKeyspace { +/** Namespace replicationdata. */ +export namespace replicationdata { - /** Keyspace sharded */ - sharded?: (boolean|null); + /** Properties of a Status. */ + interface IStatus { - /** Keyspace vindexes */ - vindexes?: ({ [k: string]: vschema.IVindex }|null); + /** Status position */ + position?: (string|null); - /** Keyspace tables */ - tables?: ({ [k: string]: vschema.ITable }|null); + /** Status replication_lag_seconds */ + replication_lag_seconds?: (number|null); - /** Keyspace require_explicit_routing */ - require_explicit_routing?: (boolean|null); + /** Status source_host */ + source_host?: (string|null); - /** Keyspace foreign_key_mode */ - foreign_key_mode?: (vschema.Keyspace.ForeignKeyMode|null); + /** Status source_port */ + source_port?: (number|null); + + /** Status connect_retry */ + connect_retry?: (number|null); + + /** Status relay_log_position */ + relay_log_position?: (string|null); + + /** Status file_position */ + file_position?: (string|null); + + /** Status relay_log_source_binlog_equivalent_position */ + relay_log_source_binlog_equivalent_position?: (string|null); + + /** Status source_server_id */ + source_server_id?: (number|null); + + /** Status source_uuid */ + source_uuid?: (string|null); + + /** Status io_state */ + io_state?: (number|null); + + /** Status last_io_error */ + last_io_error?: (string|null); + + /** Status sql_state */ + sql_state?: (number|null); + + /** Status last_sql_error */ + last_sql_error?: (string|null); + + /** Status relay_log_file_position */ + relay_log_file_position?: (string|null); + + /** Status source_user */ + source_user?: (string|null); + + /** Status sql_delay */ + sql_delay?: (number|null); + + /** Status auto_position */ + auto_position?: (boolean|null); + + /** Status using_gtid */ + using_gtid?: (boolean|null); + + /** Status has_replication_filters */ + has_replication_filters?: (boolean|null); + + /** Status ssl_allowed */ + ssl_allowed?: (boolean|null); + + /** Status replication_lag_unknown */ + replication_lag_unknown?: (boolean|null); } - /** Represents a Keyspace. */ - class Keyspace implements IKeyspace { + /** Represents a Status. */ + class Status implements IStatus { /** - * Constructs a new Keyspace. + * Constructs a new Status. * @param [properties] Properties to set */ - constructor(properties?: vschema.IKeyspace); + constructor(properties?: replicationdata.IStatus); - /** Keyspace sharded. */ - public sharded: boolean; + /** Status position. */ + public position: string; - /** Keyspace vindexes. */ - public vindexes: { [k: string]: vschema.IVindex }; + /** Status replication_lag_seconds. */ + public replication_lag_seconds: number; - /** Keyspace tables. */ - public tables: { [k: string]: vschema.ITable }; + /** Status source_host. */ + public source_host: string; - /** Keyspace require_explicit_routing. */ - public require_explicit_routing: boolean; + /** Status source_port. */ + public source_port: number; - /** Keyspace foreign_key_mode. */ - public foreign_key_mode: vschema.Keyspace.ForeignKeyMode; + /** Status connect_retry. */ + public connect_retry: number; + + /** Status relay_log_position. */ + public relay_log_position: string; + + /** Status file_position. */ + public file_position: string; + + /** Status relay_log_source_binlog_equivalent_position. */ + public relay_log_source_binlog_equivalent_position: string; + + /** Status source_server_id. */ + public source_server_id: number; + + /** Status source_uuid. */ + public source_uuid: string; + + /** Status io_state. */ + public io_state: number; + + /** Status last_io_error. */ + public last_io_error: string; + + /** Status sql_state. */ + public sql_state: number; + + /** Status last_sql_error. */ + public last_sql_error: string; + + /** Status relay_log_file_position. */ + public relay_log_file_position: string; + + /** Status source_user. */ + public source_user: string; + + /** Status sql_delay. */ + public sql_delay: number; + + /** Status auto_position. */ + public auto_position: boolean; + + /** Status using_gtid. */ + public using_gtid: boolean; + + /** Status has_replication_filters. */ + public has_replication_filters: boolean; + + /** Status ssl_allowed. */ + public ssl_allowed: boolean; + + /** Status replication_lag_unknown. */ + public replication_lag_unknown: boolean; /** - * Creates a new Keyspace instance using the specified properties. + * Creates a new Status instance using the specified properties. * @param [properties] Properties to set - * @returns Keyspace instance + * @returns Status instance */ - public static create(properties?: vschema.IKeyspace): vschema.Keyspace; + public static create(properties?: replicationdata.IStatus): replicationdata.Status; /** - * Encodes the specified Keyspace message. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. - * @param message Keyspace message or plain object to encode + * Encodes the specified Status message. Does not implicitly {@link replicationdata.Status.verify|verify} messages. + * @param message Status message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: replicationdata.IStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. - * @param message Keyspace message or plain object to encode + * Encodes the specified Status message, length delimited. Does not implicitly {@link replicationdata.Status.verify|verify} messages. + * @param message Status message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: replicationdata.IStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Keyspace message from the specified reader or buffer. + * Decodes a Status message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Keyspace + * @returns Status * @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.Keyspace; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): replicationdata.Status; /** - * Decodes a Keyspace message from the specified reader or buffer, length delimited. + * Decodes a Status message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Keyspace + * @returns Status * @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.Keyspace; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): replicationdata.Status; /** - * Verifies a Keyspace message. + * Verifies a Status 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 Keyspace message from a plain object. Also converts values to their respective internal types. + * Creates a Status message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Keyspace + * @returns Status */ - public static fromObject(object: { [k: string]: any }): vschema.Keyspace; + public static fromObject(object: { [k: string]: any }): replicationdata.Status; /** - * Creates a plain object from a Keyspace message. Also converts values to other types if specified. - * @param message Keyspace + * Creates a plain object from a Status message. Also converts values to other types if specified. + * @param message Status * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.Keyspace, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: replicationdata.Status, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Keyspace to JSON. + * Converts this Status to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Keyspace + * Gets the default type url for Status * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace Keyspace { - - /** ForeignKeyMode enum. */ - enum ForeignKeyMode { - FK_DEFAULT = 0, - FK_DISALLOW = 1, - FK_UNMANAGED = 2, - FK_MANAGED = 3 - } - } - - /** Properties of a Vindex. */ - interface IVindex { - - /** Vindex type */ - type?: (string|null); + /** Properties of a StopReplicationStatus. */ + interface IStopReplicationStatus { - /** Vindex params */ - params?: ({ [k: string]: string }|null); + /** StopReplicationStatus before */ + before?: (replicationdata.IStatus|null); - /** Vindex owner */ - owner?: (string|null); + /** StopReplicationStatus after */ + after?: (replicationdata.IStatus|null); } - /** Represents a Vindex. */ - class Vindex implements IVindex { + /** Represents a StopReplicationStatus. */ + class StopReplicationStatus implements IStopReplicationStatus { /** - * Constructs a new Vindex. + * Constructs a new StopReplicationStatus. * @param [properties] Properties to set */ - constructor(properties?: vschema.IVindex); - - /** Vindex type. */ - public type: string; + constructor(properties?: replicationdata.IStopReplicationStatus); - /** Vindex params. */ - public params: { [k: string]: string }; + /** StopReplicationStatus before. */ + public before?: (replicationdata.IStatus|null); - /** Vindex owner. */ - public owner: string; + /** StopReplicationStatus after. */ + public after?: (replicationdata.IStatus|null); /** - * Creates a new Vindex instance using the specified properties. + * Creates a new StopReplicationStatus instance using the specified properties. * @param [properties] Properties to set - * @returns Vindex instance + * @returns StopReplicationStatus instance */ - public static create(properties?: vschema.IVindex): vschema.Vindex; + public static create(properties?: replicationdata.IStopReplicationStatus): replicationdata.StopReplicationStatus; /** - * Encodes the specified Vindex message. Does not implicitly {@link vschema.Vindex.verify|verify} messages. - * @param message Vindex message or plain object to encode + * Encodes the specified StopReplicationStatus message. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. + * @param message StopReplicationStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IVindex, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: replicationdata.IStopReplicationStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Vindex message, length delimited. Does not implicitly {@link vschema.Vindex.verify|verify} messages. - * @param message Vindex message or plain object to encode + * Encodes the specified StopReplicationStatus message, length delimited. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. + * @param message StopReplicationStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IVindex, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: replicationdata.IStopReplicationStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Vindex message from the specified reader or buffer. + * Decodes a StopReplicationStatus message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Vindex + * @returns StopReplicationStatus * @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.Vindex; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): replicationdata.StopReplicationStatus; /** - * Decodes a Vindex message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationStatus message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Vindex + * @returns StopReplicationStatus * @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.Vindex; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): replicationdata.StopReplicationStatus; /** - * Verifies a Vindex message. + * Verifies a StopReplicationStatus 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 Vindex message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationStatus message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Vindex + * @returns StopReplicationStatus */ - public static fromObject(object: { [k: string]: any }): vschema.Vindex; + public static fromObject(object: { [k: string]: any }): replicationdata.StopReplicationStatus; /** - * Creates a plain object from a Vindex message. Also converts values to other types if specified. - * @param message Vindex + * Creates a plain object from a StopReplicationStatus message. Also converts values to other types if specified. + * @param message StopReplicationStatus * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.Vindex, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: replicationdata.StopReplicationStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Vindex to JSON. + * Converts this StopReplicationStatus to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Vindex + * Gets the default type url for StopReplicationStatus * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Table. */ - interface ITable { - - /** Table type */ - type?: (string|null); - - /** Table column_vindexes */ - column_vindexes?: (vschema.IColumnVindex[]|null); - - /** Table auto_increment */ - auto_increment?: (vschema.IAutoIncrement|null); - - /** Table columns */ - columns?: (vschema.IColumn[]|null); + /** StopReplicationMode enum. */ + enum StopReplicationMode { + IOANDSQLTHREAD = 0, + IOTHREADONLY = 1 + } - /** Table pinned */ - pinned?: (string|null); + /** Properties of a PrimaryStatus. */ + interface IPrimaryStatus { - /** Table column_list_authoritative */ - column_list_authoritative?: (boolean|null); + /** PrimaryStatus position */ + position?: (string|null); - /** Table source */ - source?: (string|null); + /** PrimaryStatus file_position */ + file_position?: (string|null); } - /** Represents a Table. */ - class Table implements ITable { + /** Represents a PrimaryStatus. */ + class PrimaryStatus implements IPrimaryStatus { /** - * Constructs a new Table. + * Constructs a new PrimaryStatus. * @param [properties] Properties to set */ - constructor(properties?: vschema.ITable); - - /** Table type. */ - public type: string; - - /** Table column_vindexes. */ - public column_vindexes: vschema.IColumnVindex[]; - - /** Table auto_increment. */ - public auto_increment?: (vschema.IAutoIncrement|null); - - /** Table columns. */ - public columns: vschema.IColumn[]; - - /** Table pinned. */ - public pinned: string; + constructor(properties?: replicationdata.IPrimaryStatus); - /** Table column_list_authoritative. */ - public column_list_authoritative: boolean; + /** PrimaryStatus position. */ + public position: string; - /** Table source. */ - public source: string; + /** PrimaryStatus file_position. */ + public file_position: string; /** - * Creates a new Table instance using the specified properties. + * Creates a new PrimaryStatus instance using the specified properties. * @param [properties] Properties to set - * @returns Table instance + * @returns PrimaryStatus instance */ - public static create(properties?: vschema.ITable): vschema.Table; + public static create(properties?: replicationdata.IPrimaryStatus): replicationdata.PrimaryStatus; /** - * Encodes the specified Table message. Does not implicitly {@link vschema.Table.verify|verify} messages. - * @param message Table message or plain object to encode + * Encodes the specified PrimaryStatus message. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. + * @param message PrimaryStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.ITable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: replicationdata.IPrimaryStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Table message, length delimited. Does not implicitly {@link vschema.Table.verify|verify} messages. - * @param message Table message or plain object to encode + * Encodes the specified PrimaryStatus message, length delimited. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. + * @param message PrimaryStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.ITable, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: replicationdata.IPrimaryStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Table message from the specified reader or buffer. + * Decodes a PrimaryStatus message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Table + * @returns PrimaryStatus * @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.Table; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): replicationdata.PrimaryStatus; /** - * Decodes a Table message from the specified reader or buffer, length delimited. + * Decodes a PrimaryStatus message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Table + * @returns PrimaryStatus * @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.Table; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): replicationdata.PrimaryStatus; /** - * Verifies a Table message. + * Verifies a PrimaryStatus 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 Table message from a plain object. Also converts values to their respective internal types. + * Creates a PrimaryStatus message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Table + * @returns PrimaryStatus */ - public static fromObject(object: { [k: string]: any }): vschema.Table; + public static fromObject(object: { [k: string]: any }): replicationdata.PrimaryStatus; /** - * Creates a plain object from a Table message. Also converts values to other types if specified. - * @param message Table + * Creates a plain object from a PrimaryStatus message. Also converts values to other types if specified. + * @param message PrimaryStatus * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.Table, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: replicationdata.PrimaryStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Table to JSON. + * Converts this PrimaryStatus to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Table + * Gets the default type url for PrimaryStatus * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ColumnVindex. */ - interface IColumnVindex { + /** Properties of a FullStatus. */ + interface IFullStatus { - /** ColumnVindex column */ - column?: (string|null); + /** FullStatus server_id */ + server_id?: (number|null); - /** ColumnVindex name */ - name?: (string|null); + /** FullStatus server_uuid */ + server_uuid?: (string|null); - /** ColumnVindex columns */ - columns?: (string[]|null); + /** FullStatus replication_status */ + replication_status?: (replicationdata.IStatus|null); + + /** FullStatus primary_status */ + primary_status?: (replicationdata.IPrimaryStatus|null); + + /** FullStatus gtid_purged */ + gtid_purged?: (string|null); + + /** FullStatus version */ + version?: (string|null); + + /** FullStatus version_comment */ + version_comment?: (string|null); + + /** FullStatus read_only */ + read_only?: (boolean|null); + + /** FullStatus gtid_mode */ + gtid_mode?: (string|null); + + /** FullStatus binlog_format */ + binlog_format?: (string|null); + + /** FullStatus binlog_row_image */ + binlog_row_image?: (string|null); + + /** FullStatus log_bin_enabled */ + log_bin_enabled?: (boolean|null); + + /** FullStatus log_replica_updates */ + log_replica_updates?: (boolean|null); + + /** FullStatus semi_sync_primary_enabled */ + semi_sync_primary_enabled?: (boolean|null); + + /** FullStatus semi_sync_replica_enabled */ + semi_sync_replica_enabled?: (boolean|null); + + /** FullStatus semi_sync_primary_status */ + semi_sync_primary_status?: (boolean|null); + + /** FullStatus semi_sync_replica_status */ + semi_sync_replica_status?: (boolean|null); + + /** FullStatus semi_sync_primary_clients */ + semi_sync_primary_clients?: (number|null); + + /** FullStatus semi_sync_primary_timeout */ + semi_sync_primary_timeout?: (number|Long|null); + + /** FullStatus semi_sync_wait_for_replica_count */ + semi_sync_wait_for_replica_count?: (number|null); + + /** FullStatus super_read_only */ + super_read_only?: (boolean|null); } - /** Represents a ColumnVindex. */ - class ColumnVindex implements IColumnVindex { + /** Represents a FullStatus. */ + class FullStatus implements IFullStatus { /** - * Constructs a new ColumnVindex. + * Constructs a new FullStatus. * @param [properties] Properties to set */ - constructor(properties?: vschema.IColumnVindex); + constructor(properties?: replicationdata.IFullStatus); - /** ColumnVindex column. */ - public column: string; + /** FullStatus server_id. */ + public server_id: number; - /** ColumnVindex name. */ - public name: string; + /** FullStatus server_uuid. */ + public server_uuid: string; - /** ColumnVindex columns. */ - public columns: string[]; + /** FullStatus replication_status. */ + public replication_status?: (replicationdata.IStatus|null); + + /** FullStatus primary_status. */ + public primary_status?: (replicationdata.IPrimaryStatus|null); + + /** FullStatus gtid_purged. */ + public gtid_purged: string; + + /** FullStatus version. */ + public version: string; + + /** FullStatus version_comment. */ + public version_comment: string; + + /** FullStatus read_only. */ + public read_only: boolean; + + /** FullStatus gtid_mode. */ + public gtid_mode: string; + + /** FullStatus binlog_format. */ + public binlog_format: string; + + /** FullStatus binlog_row_image. */ + public binlog_row_image: string; + + /** FullStatus log_bin_enabled. */ + public log_bin_enabled: boolean; + + /** FullStatus log_replica_updates. */ + public log_replica_updates: boolean; + + /** FullStatus semi_sync_primary_enabled. */ + public semi_sync_primary_enabled: boolean; + + /** FullStatus semi_sync_replica_enabled. */ + public semi_sync_replica_enabled: boolean; + + /** FullStatus semi_sync_primary_status. */ + public semi_sync_primary_status: boolean; + + /** FullStatus semi_sync_replica_status. */ + public semi_sync_replica_status: boolean; + + /** FullStatus semi_sync_primary_clients. */ + public semi_sync_primary_clients: number; + + /** FullStatus semi_sync_primary_timeout. */ + public semi_sync_primary_timeout: (number|Long); + + /** FullStatus semi_sync_wait_for_replica_count. */ + public semi_sync_wait_for_replica_count: number; + + /** FullStatus super_read_only. */ + public super_read_only: boolean; /** - * Creates a new ColumnVindex instance using the specified properties. + * Creates a new FullStatus instance using the specified properties. * @param [properties] Properties to set - * @returns ColumnVindex instance + * @returns FullStatus instance */ - public static create(properties?: vschema.IColumnVindex): vschema.ColumnVindex; + public static create(properties?: replicationdata.IFullStatus): replicationdata.FullStatus; /** - * Encodes the specified ColumnVindex message. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. - * @param message ColumnVindex message or plain object to encode + * Encodes the specified FullStatus message. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. + * @param message FullStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IColumnVindex, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: replicationdata.IFullStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ColumnVindex message, length delimited. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. - * @param message ColumnVindex message or plain object to encode + * Encodes the specified FullStatus message, length delimited. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. + * @param message FullStatus message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IColumnVindex, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: replicationdata.IFullStatus, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ColumnVindex message from the specified reader or buffer. + * Decodes a FullStatus message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ColumnVindex + * @returns FullStatus * @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.ColumnVindex; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): replicationdata.FullStatus; /** - * Decodes a ColumnVindex message from the specified reader or buffer, length delimited. + * Decodes a FullStatus message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ColumnVindex + * @returns FullStatus * @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.ColumnVindex; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): replicationdata.FullStatus; /** - * Verifies a ColumnVindex message. + * Verifies a FullStatus 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 ColumnVindex message from a plain object. Also converts values to their respective internal types. + * Creates a FullStatus message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ColumnVindex + * @returns FullStatus */ - public static fromObject(object: { [k: string]: any }): vschema.ColumnVindex; + public static fromObject(object: { [k: string]: any }): replicationdata.FullStatus; /** - * Creates a plain object from a ColumnVindex message. Also converts values to other types if specified. - * @param message ColumnVindex + * Creates a plain object from a FullStatus message. Also converts values to other types if specified. + * @param message FullStatus * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.ColumnVindex, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: replicationdata.FullStatus, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ColumnVindex to JSON. + * Converts this FullStatus to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ColumnVindex + * Gets the default type url for FullStatus * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } +} - /** Properties of an AutoIncrement. */ - interface IAutoIncrement { +/** Namespace vschema. */ +export namespace vschema { - /** AutoIncrement column */ - column?: (string|null); + /** Properties of a RoutingRules. */ + interface IRoutingRules { - /** AutoIncrement sequence */ - sequence?: (string|null); + /** RoutingRules rules */ + rules?: (vschema.IRoutingRule[]|null); } - /** Represents an AutoIncrement. */ - class AutoIncrement implements IAutoIncrement { + /** Represents a RoutingRules. */ + class RoutingRules implements IRoutingRules { /** - * Constructs a new AutoIncrement. + * Constructs a new RoutingRules. * @param [properties] Properties to set */ - constructor(properties?: vschema.IAutoIncrement); - - /** AutoIncrement column. */ - public column: string; + constructor(properties?: vschema.IRoutingRules); - /** AutoIncrement sequence. */ - public sequence: string; + /** RoutingRules rules. */ + public rules: vschema.IRoutingRule[]; /** - * Creates a new AutoIncrement instance using the specified properties. + * Creates a new RoutingRules instance using the specified properties. * @param [properties] Properties to set - * @returns AutoIncrement instance + * @returns RoutingRules instance */ - public static create(properties?: vschema.IAutoIncrement): vschema.AutoIncrement; + public static create(properties?: vschema.IRoutingRules): vschema.RoutingRules; /** - * Encodes the specified AutoIncrement message. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. - * @param message AutoIncrement message or plain object to encode + * Encodes the specified RoutingRules message. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. + * @param message RoutingRules message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IAutoIncrement, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AutoIncrement message, length delimited. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. - * @param message AutoIncrement message or plain object to encode + * Encodes the specified RoutingRules message, length delimited. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. + * @param message RoutingRules message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IAutoIncrement, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AutoIncrement message from the specified reader or buffer. + * Decodes a RoutingRules message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AutoIncrement + * @returns RoutingRules * @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.AutoIncrement; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.RoutingRules; /** - * Decodes an AutoIncrement message from the specified reader or buffer, length delimited. + * Decodes a RoutingRules message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AutoIncrement + * @returns RoutingRules * @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.AutoIncrement; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.RoutingRules; /** - * Verifies an AutoIncrement message. + * Verifies a RoutingRules 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 AutoIncrement message from a plain object. Also converts values to their respective internal types. + * Creates a RoutingRules message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AutoIncrement + * @returns RoutingRules */ - public static fromObject(object: { [k: string]: any }): vschema.AutoIncrement; + public static fromObject(object: { [k: string]: any }): vschema.RoutingRules; /** - * Creates a plain object from an AutoIncrement message. Also converts values to other types if specified. - * @param message AutoIncrement + * Creates a plain object from a RoutingRules message. Also converts values to other types if specified. + * @param message RoutingRules * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.AutoIncrement, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.RoutingRules, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AutoIncrement to JSON. + * Converts this RoutingRules to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AutoIncrement + * Gets the default type url for RoutingRules * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Column. */ - interface IColumn { + /** Properties of a RoutingRule. */ + interface IRoutingRule { - /** Column name */ - name?: (string|null); + /** RoutingRule from_table */ + from_table?: (string|null); - /** Column type */ - type?: (query.Type|null); + /** RoutingRule to_tables */ + to_tables?: (string[]|null); } - /** Represents a Column. */ - class Column implements IColumn { + /** Represents a RoutingRule. */ + class RoutingRule implements IRoutingRule { /** - * Constructs a new Column. + * Constructs a new RoutingRule. * @param [properties] Properties to set */ - constructor(properties?: vschema.IColumn); + constructor(properties?: vschema.IRoutingRule); - /** Column name. */ - public name: string; + /** RoutingRule from_table. */ + public from_table: string; - /** Column type. */ - public type: query.Type; + /** RoutingRule to_tables. */ + public to_tables: string[]; /** - * Creates a new Column instance using the specified properties. + * Creates a new RoutingRule instance using the specified properties. * @param [properties] Properties to set - * @returns Column instance + * @returns RoutingRule instance */ - public static create(properties?: vschema.IColumn): vschema.Column; + public static create(properties?: vschema.IRoutingRule): vschema.RoutingRule; /** - * Encodes the specified Column message. Does not implicitly {@link vschema.Column.verify|verify} messages. - * @param message Column message or plain object to encode + * Encodes the specified RoutingRule message. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. + * @param message RoutingRule message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IColumn, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Column message, length delimited. Does not implicitly {@link vschema.Column.verify|verify} messages. - * @param message Column message or plain object to encode - * @param [writer] Writer to encode to + * Encodes the specified RoutingRule message, length delimited. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. + * @param message RoutingRule message or plain object to encode + * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IColumn, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Column message from the specified reader or buffer. + * Decodes a RoutingRule message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Column + * @returns RoutingRule * @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.Column; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.RoutingRule; /** - * Decodes a Column message from the specified reader or buffer, length delimited. + * Decodes a RoutingRule message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Column + * @returns RoutingRule * @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.Column; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.RoutingRule; /** - * Verifies a Column message. + * Verifies a RoutingRule 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 Column message from a plain object. Also converts values to their respective internal types. + * Creates a RoutingRule message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Column + * @returns RoutingRule */ - public static fromObject(object: { [k: string]: any }): vschema.Column; + public static fromObject(object: { [k: string]: any }): vschema.RoutingRule; /** - * Creates a plain object from a Column message. Also converts values to other types if specified. - * @param message Column + * Creates a plain object from a RoutingRule message. Also converts values to other types if specified. + * @param message RoutingRule * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.Column, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.RoutingRule, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Column to JSON. + * Converts this RoutingRule to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Column + * Gets the default type url for RoutingRule * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SrvVSchema. */ - interface ISrvVSchema { + /** Properties of a Keyspace. */ + interface IKeyspace { - /** SrvVSchema keyspaces */ - keyspaces?: ({ [k: string]: vschema.IKeyspace }|null); + /** Keyspace sharded */ + sharded?: (boolean|null); - /** SrvVSchema routing_rules */ - routing_rules?: (vschema.IRoutingRules|null); + /** Keyspace vindexes */ + vindexes?: ({ [k: string]: vschema.IVindex }|null); - /** SrvVSchema shard_routing_rules */ - shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** Keyspace tables */ + tables?: ({ [k: string]: vschema.ITable }|null); + + /** Keyspace require_explicit_routing */ + require_explicit_routing?: (boolean|null); + + /** Keyspace foreign_key_mode */ + foreign_key_mode?: (vschema.Keyspace.ForeignKeyMode|null); } - /** Represents a SrvVSchema. */ - class SrvVSchema implements ISrvVSchema { + /** Represents a Keyspace. */ + class Keyspace implements IKeyspace { /** - * Constructs a new SrvVSchema. + * Constructs a new Keyspace. * @param [properties] Properties to set */ - constructor(properties?: vschema.ISrvVSchema); + constructor(properties?: vschema.IKeyspace); - /** SrvVSchema keyspaces. */ - public keyspaces: { [k: string]: vschema.IKeyspace }; + /** Keyspace sharded. */ + public sharded: boolean; - /** SrvVSchema routing_rules. */ - public routing_rules?: (vschema.IRoutingRules|null); + /** Keyspace vindexes. */ + public vindexes: { [k: string]: vschema.IVindex }; - /** SrvVSchema shard_routing_rules. */ - public shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** Keyspace tables. */ + public tables: { [k: string]: vschema.ITable }; + + /** Keyspace require_explicit_routing. */ + public require_explicit_routing: boolean; + + /** Keyspace foreign_key_mode. */ + public foreign_key_mode: vschema.Keyspace.ForeignKeyMode; /** - * Creates a new SrvVSchema instance using the specified properties. + * Creates a new Keyspace instance using the specified properties. * @param [properties] Properties to set - * @returns SrvVSchema instance + * @returns Keyspace instance */ - public static create(properties?: vschema.ISrvVSchema): vschema.SrvVSchema; + public static create(properties?: vschema.IKeyspace): vschema.Keyspace; /** - * Encodes the specified SrvVSchema message. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. - * @param message SrvVSchema message or plain object to encode + * Encodes the specified Keyspace message. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. + * @param message Keyspace message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.ISrvVSchema, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SrvVSchema message, length delimited. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. - * @param message SrvVSchema message or plain object to encode + * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. + * @param message Keyspace message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.ISrvVSchema, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SrvVSchema message from the specified reader or buffer. + * Decodes a Keyspace message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SrvVSchema + * @returns Keyspace * @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.SrvVSchema; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.Keyspace; /** - * Decodes a SrvVSchema message from the specified reader or buffer, length delimited. + * Decodes a Keyspace message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SrvVSchema + * @returns Keyspace * @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.SrvVSchema; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.Keyspace; /** - * Verifies a SrvVSchema message. + * Verifies a Keyspace 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 SrvVSchema message from a plain object. Also converts values to their respective internal types. + * Creates a Keyspace message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SrvVSchema + * @returns Keyspace */ - public static fromObject(object: { [k: string]: any }): vschema.SrvVSchema; + public static fromObject(object: { [k: string]: any }): vschema.Keyspace; /** - * Creates a plain object from a SrvVSchema message. Also converts values to other types if specified. - * @param message SrvVSchema + * Creates a plain object from a Keyspace message. Also converts values to other types if specified. + * @param message Keyspace * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.SrvVSchema, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.Keyspace, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SrvVSchema to JSON. + * Converts this Keyspace to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SrvVSchema + * Gets the default type url for Keyspace * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardRoutingRules. */ - interface IShardRoutingRules { + namespace Keyspace { - /** ShardRoutingRules rules */ - rules?: (vschema.IShardRoutingRule[]|null); + /** ForeignKeyMode enum. */ + enum ForeignKeyMode { + FK_DEFAULT = 0, + FK_DISALLOW = 1, + FK_UNMANAGED = 2, + FK_MANAGED = 3 + } } - /** Represents a ShardRoutingRules. */ - class ShardRoutingRules implements IShardRoutingRules { + /** Properties of a Vindex. */ + interface IVindex { + + /** Vindex type */ + type?: (string|null); + + /** Vindex params */ + params?: ({ [k: string]: string }|null); + + /** Vindex owner */ + owner?: (string|null); + } + + /** Represents a Vindex. */ + class Vindex implements IVindex { /** - * Constructs a new ShardRoutingRules. + * Constructs a new Vindex. * @param [properties] Properties to set */ - constructor(properties?: vschema.IShardRoutingRules); + constructor(properties?: vschema.IVindex); - /** ShardRoutingRules rules. */ - public rules: vschema.IShardRoutingRule[]; + /** Vindex type. */ + public type: string; + + /** Vindex params. */ + public params: { [k: string]: string }; + + /** Vindex owner. */ + public owner: string; /** - * Creates a new ShardRoutingRules instance using the specified properties. + * Creates a new Vindex instance using the specified properties. * @param [properties] Properties to set - * @returns ShardRoutingRules instance + * @returns Vindex instance */ - public static create(properties?: vschema.IShardRoutingRules): vschema.ShardRoutingRules; + public static create(properties?: vschema.IVindex): vschema.Vindex; /** - * Encodes the specified ShardRoutingRules message. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. - * @param message ShardRoutingRules message or plain object to encode + * Encodes the specified Vindex message. Does not implicitly {@link vschema.Vindex.verify|verify} messages. + * @param message Vindex message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IShardRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IVindex, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardRoutingRules message, length delimited. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. - * @param message ShardRoutingRules message or plain object to encode + * Encodes the specified Vindex message, length delimited. Does not implicitly {@link vschema.Vindex.verify|verify} messages. + * @param message Vindex message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IShardRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IVindex, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardRoutingRules message from the specified reader or buffer. + * Decodes a Vindex message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardRoutingRules + * @returns Vindex * @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.ShardRoutingRules; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.Vindex; /** - * Decodes a ShardRoutingRules message from the specified reader or buffer, length delimited. + * Decodes a Vindex message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardRoutingRules + * @returns Vindex * @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.ShardRoutingRules; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.Vindex; /** - * Verifies a ShardRoutingRules message. + * Verifies a Vindex 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 ShardRoutingRules message from a plain object. Also converts values to their respective internal types. + * Creates a Vindex message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardRoutingRules + * @returns Vindex */ - public static fromObject(object: { [k: string]: any }): vschema.ShardRoutingRules; + public static fromObject(object: { [k: string]: any }): vschema.Vindex; /** - * Creates a plain object from a ShardRoutingRules message. Also converts values to other types if specified. - * @param message ShardRoutingRules + * Creates a plain object from a Vindex message. Also converts values to other types if specified. + * @param message Vindex * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.ShardRoutingRules, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.Vindex, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardRoutingRules to JSON. + * Converts this Vindex to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardRoutingRules + * Gets the default type url for Vindex * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardRoutingRule. */ - interface IShardRoutingRule { + /** Properties of a Table. */ + interface ITable { - /** ShardRoutingRule from_keyspace */ - from_keyspace?: (string|null); + /** Table type */ + type?: (string|null); - /** ShardRoutingRule to_keyspace */ - to_keyspace?: (string|null); + /** Table column_vindexes */ + column_vindexes?: (vschema.IColumnVindex[]|null); - /** ShardRoutingRule shard */ - shard?: (string|null); + /** Table auto_increment */ + auto_increment?: (vschema.IAutoIncrement|null); + + /** Table columns */ + columns?: (vschema.IColumn[]|null); + + /** Table pinned */ + pinned?: (string|null); + + /** Table column_list_authoritative */ + column_list_authoritative?: (boolean|null); + + /** Table source */ + source?: (string|null); } - /** Represents a ShardRoutingRule. */ - class ShardRoutingRule implements IShardRoutingRule { + /** Represents a Table. */ + class Table implements ITable { /** - * Constructs a new ShardRoutingRule. + * Constructs a new Table. * @param [properties] Properties to set */ - constructor(properties?: vschema.IShardRoutingRule); + constructor(properties?: vschema.ITable); - /** ShardRoutingRule from_keyspace. */ - public from_keyspace: string; + /** Table type. */ + public type: string; - /** ShardRoutingRule to_keyspace. */ - public to_keyspace: string; + /** Table column_vindexes. */ + public column_vindexes: vschema.IColumnVindex[]; - /** ShardRoutingRule shard. */ - public shard: string; + /** Table auto_increment. */ + public auto_increment?: (vschema.IAutoIncrement|null); + + /** Table columns. */ + public columns: vschema.IColumn[]; + + /** Table pinned. */ + public pinned: string; + + /** Table column_list_authoritative. */ + public column_list_authoritative: boolean; + + /** Table source. */ + public source: string; /** - * Creates a new ShardRoutingRule instance using the specified properties. + * Creates a new Table instance using the specified properties. * @param [properties] Properties to set - * @returns ShardRoutingRule instance + * @returns Table instance */ - public static create(properties?: vschema.IShardRoutingRule): vschema.ShardRoutingRule; + public static create(properties?: vschema.ITable): vschema.Table; /** - * Encodes the specified ShardRoutingRule message. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. - * @param message ShardRoutingRule message or plain object to encode + * Encodes the specified Table message. Does not implicitly {@link vschema.Table.verify|verify} messages. + * @param message Table message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vschema.IShardRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.ITable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardRoutingRule message, length delimited. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. - * @param message ShardRoutingRule message or plain object to encode + * Encodes the specified Table message, length delimited. Does not implicitly {@link vschema.Table.verify|verify} messages. + * @param message Table message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vschema.IShardRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.ITable, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardRoutingRule message from the specified reader or buffer. + * Decodes a Table message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardRoutingRule + * @returns Table * @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.ShardRoutingRule; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.Table; /** - * Decodes a ShardRoutingRule message from the specified reader or buffer, length delimited. + * Decodes a Table message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardRoutingRule + * @returns Table * @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.ShardRoutingRule; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.Table; /** - * Verifies a ShardRoutingRule message. + * Verifies a Table 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 ShardRoutingRule message from a plain object. Also converts values to their respective internal types. + * Creates a Table message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardRoutingRule + * @returns Table */ - public static fromObject(object: { [k: string]: any }): vschema.ShardRoutingRule; + public static fromObject(object: { [k: string]: any }): vschema.Table; /** - * Creates a plain object from a ShardRoutingRule message. Also converts values to other types if specified. - * @param message ShardRoutingRule + * Creates a plain object from a Table message. Also converts values to other types if specified. + * @param message Table * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vschema.ShardRoutingRule, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.Table, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardRoutingRule to JSON. + * Converts this Table to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardRoutingRule + * Gets the default type url for Table * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } -} -/** Namespace vtctldata. */ -export namespace vtctldata { + /** Properties of a ColumnVindex. */ + interface IColumnVindex { - /** Properties of an ExecuteVtctlCommandRequest. */ - interface IExecuteVtctlCommandRequest { + /** ColumnVindex column */ + column?: (string|null); - /** ExecuteVtctlCommandRequest args */ - args?: (string[]|null); + /** ColumnVindex name */ + name?: (string|null); - /** ExecuteVtctlCommandRequest action_timeout */ - action_timeout?: (number|Long|null); + /** ColumnVindex columns */ + columns?: (string[]|null); } - /** Represents an ExecuteVtctlCommandRequest. */ - class ExecuteVtctlCommandRequest implements IExecuteVtctlCommandRequest { + /** Represents a ColumnVindex. */ + class ColumnVindex implements IColumnVindex { /** - * Constructs a new ExecuteVtctlCommandRequest. + * Constructs a new ColumnVindex. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteVtctlCommandRequest); + constructor(properties?: vschema.IColumnVindex); - /** ExecuteVtctlCommandRequest args. */ - public args: string[]; + /** ColumnVindex column. */ + public column: string; - /** ExecuteVtctlCommandRequest action_timeout. */ - public action_timeout: (number|Long); + /** ColumnVindex name. */ + public name: string; + + /** ColumnVindex columns. */ + public columns: string[]; /** - * Creates a new ExecuteVtctlCommandRequest instance using the specified properties. + * Creates a new ColumnVindex instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteVtctlCommandRequest instance + * @returns ColumnVindex instance */ - public static create(properties?: vtctldata.IExecuteVtctlCommandRequest): vtctldata.ExecuteVtctlCommandRequest; + public static create(properties?: vschema.IColumnVindex): vschema.ColumnVindex; /** - * Encodes the specified ExecuteVtctlCommandRequest message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. - * @param message ExecuteVtctlCommandRequest message or plain object to encode + * Encodes the specified ColumnVindex message. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. + * @param message ColumnVindex message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteVtctlCommandRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IColumnVindex, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteVtctlCommandRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. - * @param message ExecuteVtctlCommandRequest message or plain object to encode + * Encodes the specified ColumnVindex message, length delimited. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. + * @param message ColumnVindex message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteVtctlCommandRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IColumnVindex, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer. + * Decodes a ColumnVindex message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteVtctlCommandRequest + * @returns ColumnVindex * @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.ExecuteVtctlCommandRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.ColumnVindex; /** - * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer, length delimited. + * Decodes a ColumnVindex message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteVtctlCommandRequest + * @returns ColumnVindex * @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.ExecuteVtctlCommandRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.ColumnVindex; /** - * Verifies an ExecuteVtctlCommandRequest message. + * Verifies a ColumnVindex 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 ExecuteVtctlCommandRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ColumnVindex message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteVtctlCommandRequest + * @returns ColumnVindex */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteVtctlCommandRequest; + public static fromObject(object: { [k: string]: any }): vschema.ColumnVindex; /** - * Creates a plain object from an ExecuteVtctlCommandRequest message. Also converts values to other types if specified. - * @param message ExecuteVtctlCommandRequest + * Creates a plain object from a ColumnVindex message. Also converts values to other types if specified. + * @param message ColumnVindex * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteVtctlCommandRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.ColumnVindex, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteVtctlCommandRequest to JSON. + * Converts this ColumnVindex to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteVtctlCommandRequest + * Gets the default type url for ColumnVindex * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteVtctlCommandResponse. */ - interface IExecuteVtctlCommandResponse { + /** Properties of an AutoIncrement. */ + interface IAutoIncrement { - /** ExecuteVtctlCommandResponse event */ - event?: (logutil.IEvent|null); + /** AutoIncrement column */ + column?: (string|null); + + /** AutoIncrement sequence */ + sequence?: (string|null); } - /** Represents an ExecuteVtctlCommandResponse. */ - class ExecuteVtctlCommandResponse implements IExecuteVtctlCommandResponse { + /** Represents an AutoIncrement. */ + class AutoIncrement implements IAutoIncrement { /** - * Constructs a new ExecuteVtctlCommandResponse. + * Constructs a new AutoIncrement. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteVtctlCommandResponse); + constructor(properties?: vschema.IAutoIncrement); - /** ExecuteVtctlCommandResponse event. */ - public event?: (logutil.IEvent|null); + /** AutoIncrement column. */ + public column: string; + + /** AutoIncrement sequence. */ + public sequence: string; /** - * Creates a new ExecuteVtctlCommandResponse instance using the specified properties. + * Creates a new AutoIncrement instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteVtctlCommandResponse instance + * @returns AutoIncrement instance */ - public static create(properties?: vtctldata.IExecuteVtctlCommandResponse): vtctldata.ExecuteVtctlCommandResponse; + public static create(properties?: vschema.IAutoIncrement): vschema.AutoIncrement; /** - * Encodes the specified ExecuteVtctlCommandResponse message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. - * @param message ExecuteVtctlCommandResponse message or plain object to encode + * Encodes the specified AutoIncrement message. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. + * @param message AutoIncrement message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteVtctlCommandResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IAutoIncrement, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteVtctlCommandResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. - * @param message ExecuteVtctlCommandResponse message or plain object to encode + * Encodes the specified AutoIncrement message, length delimited. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. + * @param message AutoIncrement message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteVtctlCommandResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IAutoIncrement, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer. + * Decodes an AutoIncrement message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteVtctlCommandResponse + * @returns AutoIncrement * @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.ExecuteVtctlCommandResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.AutoIncrement; /** - * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer, length delimited. + * Decodes an AutoIncrement message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteVtctlCommandResponse + * @returns AutoIncrement * @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.ExecuteVtctlCommandResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.AutoIncrement; /** - * Verifies an ExecuteVtctlCommandResponse message. + * Verifies an AutoIncrement 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 ExecuteVtctlCommandResponse message from a plain object. Also converts values to their respective internal types. + * Creates an AutoIncrement message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteVtctlCommandResponse + * @returns AutoIncrement */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteVtctlCommandResponse; + public static fromObject(object: { [k: string]: any }): vschema.AutoIncrement; /** - * Creates a plain object from an ExecuteVtctlCommandResponse message. Also converts values to other types if specified. - * @param message ExecuteVtctlCommandResponse + * Creates a plain object from an AutoIncrement message. Also converts values to other types if specified. + * @param message AutoIncrement * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteVtctlCommandResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.AutoIncrement, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteVtctlCommandResponse to JSON. + * Converts this AutoIncrement to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteVtctlCommandResponse + * Gets the default type url for AutoIncrement * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** MaterializationIntent enum. */ - enum MaterializationIntent { - CUSTOM = 0, - MOVETABLES = 1, - CREATELOOKUPINDEX = 2 - } - - /** Properties of a TableMaterializeSettings. */ - interface ITableMaterializeSettings { - - /** TableMaterializeSettings target_table */ - target_table?: (string|null); + /** Properties of a Column. */ + interface IColumn { - /** TableMaterializeSettings source_expression */ - source_expression?: (string|null); + /** Column name */ + name?: (string|null); - /** TableMaterializeSettings create_ddl */ - create_ddl?: (string|null); + /** Column type */ + type?: (query.Type|null); } - /** Represents a TableMaterializeSettings. */ - class TableMaterializeSettings implements ITableMaterializeSettings { + /** Represents a Column. */ + class Column implements IColumn { /** - * Constructs a new TableMaterializeSettings. + * Constructs a new Column. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ITableMaterializeSettings); - - /** TableMaterializeSettings target_table. */ - public target_table: string; + constructor(properties?: vschema.IColumn); - /** TableMaterializeSettings source_expression. */ - public source_expression: string; + /** Column name. */ + public name: string; - /** TableMaterializeSettings create_ddl. */ - public create_ddl: string; + /** Column type. */ + public type: query.Type; /** - * Creates a new TableMaterializeSettings instance using the specified properties. + * Creates a new Column instance using the specified properties. * @param [properties] Properties to set - * @returns TableMaterializeSettings instance + * @returns Column instance */ - public static create(properties?: vtctldata.ITableMaterializeSettings): vtctldata.TableMaterializeSettings; + public static create(properties?: vschema.IColumn): vschema.Column; /** - * Encodes the specified TableMaterializeSettings message. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. - * @param message TableMaterializeSettings message or plain object to encode + * Encodes the specified Column message. Does not implicitly {@link vschema.Column.verify|verify} messages. + * @param message Column message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ITableMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IColumn, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TableMaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. - * @param message TableMaterializeSettings message or plain object to encode + * Encodes the specified Column message, length delimited. Does not implicitly {@link vschema.Column.verify|verify} messages. + * @param message Column message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ITableMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IColumn, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TableMaterializeSettings message from the specified reader or buffer. + * Decodes a Column message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TableMaterializeSettings + * @returns Column * @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.TableMaterializeSettings; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.Column; /** - * Decodes a TableMaterializeSettings message from the specified reader or buffer, length delimited. + * Decodes a Column message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TableMaterializeSettings + * @returns Column * @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.TableMaterializeSettings; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.Column; /** - * Verifies a TableMaterializeSettings message. + * Verifies a Column 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 TableMaterializeSettings message from a plain object. Also converts values to their respective internal types. + * Creates a Column message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TableMaterializeSettings + * @returns Column */ - public static fromObject(object: { [k: string]: any }): vtctldata.TableMaterializeSettings; + public static fromObject(object: { [k: string]: any }): vschema.Column; /** - * Creates a plain object from a TableMaterializeSettings message. Also converts values to other types if specified. - * @param message TableMaterializeSettings + * Creates a plain object from a Column message. Also converts values to other types if specified. + * @param message Column * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.TableMaterializeSettings, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.Column, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TableMaterializeSettings to JSON. + * Converts this Column to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TableMaterializeSettings + * Gets the default type url for Column * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a MaterializeSettings. */ - interface IMaterializeSettings { - - /** MaterializeSettings workflow */ - workflow?: (string|null); - - /** MaterializeSettings source_keyspace */ - source_keyspace?: (string|null); - - /** MaterializeSettings target_keyspace */ - target_keyspace?: (string|null); - - /** MaterializeSettings stop_after_copy */ - stop_after_copy?: (boolean|null); - - /** MaterializeSettings table_settings */ - table_settings?: (vtctldata.ITableMaterializeSettings[]|null); - - /** MaterializeSettings cell */ - cell?: (string|null); - - /** MaterializeSettings tablet_types */ - tablet_types?: (string|null); - - /** MaterializeSettings external_cluster */ - external_cluster?: (string|null); - - /** MaterializeSettings materialization_intent */ - materialization_intent?: (vtctldata.MaterializationIntent|null); - - /** MaterializeSettings source_time_zone */ - source_time_zone?: (string|null); - - /** MaterializeSettings target_time_zone */ - target_time_zone?: (string|null); + /** Properties of a SrvVSchema. */ + interface ISrvVSchema { - /** MaterializeSettings source_shards */ - source_shards?: (string[]|null); + /** SrvVSchema keyspaces */ + keyspaces?: ({ [k: string]: vschema.IKeyspace }|null); - /** MaterializeSettings on_ddl */ - on_ddl?: (string|null); + /** SrvVSchema routing_rules */ + routing_rules?: (vschema.IRoutingRules|null); - /** MaterializeSettings defer_secondary_keys */ - defer_secondary_keys?: (boolean|null); + /** SrvVSchema shard_routing_rules */ + shard_routing_rules?: (vschema.IShardRoutingRules|null); } - /** Represents a MaterializeSettings. */ - class MaterializeSettings implements IMaterializeSettings { + /** Represents a SrvVSchema. */ + class SrvVSchema implements ISrvVSchema { /** - * Constructs a new MaterializeSettings. + * Constructs a new SrvVSchema. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IMaterializeSettings); - - /** MaterializeSettings workflow. */ - public workflow: string; - - /** MaterializeSettings source_keyspace. */ - public source_keyspace: string; - - /** MaterializeSettings target_keyspace. */ - public target_keyspace: string; - - /** MaterializeSettings stop_after_copy. */ - public stop_after_copy: boolean; - - /** MaterializeSettings table_settings. */ - public table_settings: vtctldata.ITableMaterializeSettings[]; - - /** MaterializeSettings cell. */ - public cell: string; - - /** MaterializeSettings tablet_types. */ - public tablet_types: string; - - /** MaterializeSettings external_cluster. */ - public external_cluster: string; - - /** MaterializeSettings materialization_intent. */ - public materialization_intent: vtctldata.MaterializationIntent; - - /** MaterializeSettings source_time_zone. */ - public source_time_zone: string; - - /** MaterializeSettings target_time_zone. */ - public target_time_zone: string; + constructor(properties?: vschema.ISrvVSchema); - /** MaterializeSettings source_shards. */ - public source_shards: string[]; + /** SrvVSchema keyspaces. */ + public keyspaces: { [k: string]: vschema.IKeyspace }; - /** MaterializeSettings on_ddl. */ - public on_ddl: string; + /** SrvVSchema routing_rules. */ + public routing_rules?: (vschema.IRoutingRules|null); - /** MaterializeSettings defer_secondary_keys. */ - public defer_secondary_keys: boolean; + /** SrvVSchema shard_routing_rules. */ + public shard_routing_rules?: (vschema.IShardRoutingRules|null); /** - * Creates a new MaterializeSettings instance using the specified properties. + * Creates a new SrvVSchema instance using the specified properties. * @param [properties] Properties to set - * @returns MaterializeSettings instance + * @returns SrvVSchema instance */ - public static create(properties?: vtctldata.IMaterializeSettings): vtctldata.MaterializeSettings; + public static create(properties?: vschema.ISrvVSchema): vschema.SrvVSchema; /** - * Encodes the specified MaterializeSettings message. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. - * @param message MaterializeSettings message or plain object to encode + * Encodes the specified SrvVSchema message. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. + * @param message SrvVSchema message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.ISrvVSchema, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified MaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. - * @param message MaterializeSettings message or plain object to encode + * Encodes the specified SrvVSchema message, length delimited. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. + * @param message SrvVSchema message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.ISrvVSchema, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a MaterializeSettings message from the specified reader or buffer. + * Decodes a SrvVSchema message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns MaterializeSettings + * @returns SrvVSchema * @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.MaterializeSettings; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.SrvVSchema; /** - * Decodes a MaterializeSettings message from the specified reader or buffer, length delimited. + * Decodes a SrvVSchema message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns MaterializeSettings + * @returns SrvVSchema * @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.MaterializeSettings; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.SrvVSchema; /** - * Verifies a MaterializeSettings message. + * Verifies a SrvVSchema 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 MaterializeSettings message from a plain object. Also converts values to their respective internal types. + * Creates a SrvVSchema message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns MaterializeSettings + * @returns SrvVSchema */ - public static fromObject(object: { [k: string]: any }): vtctldata.MaterializeSettings; + public static fromObject(object: { [k: string]: any }): vschema.SrvVSchema; /** - * Creates a plain object from a MaterializeSettings message. Also converts values to other types if specified. - * @param message MaterializeSettings + * Creates a plain object from a SrvVSchema message. Also converts values to other types if specified. + * @param message SrvVSchema * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.MaterializeSettings, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.SrvVSchema, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this MaterializeSettings to JSON. + * Converts this SrvVSchema to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for MaterializeSettings + * Gets the default type url for SrvVSchema * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Keyspace. */ - interface IKeyspace { - - /** Keyspace name */ - name?: (string|null); + /** Properties of a ShardRoutingRules. */ + interface IShardRoutingRules { - /** Keyspace keyspace */ - keyspace?: (topodata.IKeyspace|null); + /** ShardRoutingRules rules */ + rules?: (vschema.IShardRoutingRule[]|null); } - /** Represents a Keyspace. */ - class Keyspace implements IKeyspace { + /** Represents a ShardRoutingRules. */ + class ShardRoutingRules implements IShardRoutingRules { /** - * Constructs a new Keyspace. + * Constructs a new ShardRoutingRules. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IKeyspace); - - /** Keyspace name. */ - public name: string; + constructor(properties?: vschema.IShardRoutingRules); - /** Keyspace keyspace. */ - public keyspace?: (topodata.IKeyspace|null); + /** ShardRoutingRules rules. */ + public rules: vschema.IShardRoutingRule[]; /** - * Creates a new Keyspace instance using the specified properties. + * Creates a new ShardRoutingRules instance using the specified properties. * @param [properties] Properties to set - * @returns Keyspace instance + * @returns ShardRoutingRules instance */ - public static create(properties?: vtctldata.IKeyspace): vtctldata.Keyspace; + public static create(properties?: vschema.IShardRoutingRules): vschema.ShardRoutingRules; /** - * Encodes the specified Keyspace message. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. - * @param message Keyspace message or plain object to encode + * Encodes the specified ShardRoutingRules message. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. + * @param message ShardRoutingRules message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IShardRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. - * @param message Keyspace message or plain object to encode + * Encodes the specified ShardRoutingRules message, length delimited. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. + * @param message ShardRoutingRules message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IShardRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Keyspace message from the specified reader or buffer. + * Decodes a ShardRoutingRules message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Keyspace + * @returns ShardRoutingRules * @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.Keyspace; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.ShardRoutingRules; /** - * Decodes a Keyspace message from the specified reader or buffer, length delimited. + * Decodes a ShardRoutingRules message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Keyspace + * @returns ShardRoutingRules * @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.Keyspace; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.ShardRoutingRules; /** - * Verifies a Keyspace message. + * Verifies a ShardRoutingRules 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 Keyspace message from a plain object. Also converts values to their respective internal types. + * Creates a ShardRoutingRules message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Keyspace + * @returns ShardRoutingRules */ - public static fromObject(object: { [k: string]: any }): vtctldata.Keyspace; + public static fromObject(object: { [k: string]: any }): vschema.ShardRoutingRules; /** - * Creates a plain object from a Keyspace message. Also converts values to other types if specified. - * @param message Keyspace + * Creates a plain object from a ShardRoutingRules message. Also converts values to other types if specified. + * @param message ShardRoutingRules * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.Keyspace, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.ShardRoutingRules, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Keyspace to JSON. + * Converts this ShardRoutingRules to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Keyspace + * Gets the default type url for ShardRoutingRules * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Shard. */ - interface IShard { + /** Properties of a ShardRoutingRule. */ + interface IShardRoutingRule { - /** Shard keyspace */ - keyspace?: (string|null); + /** ShardRoutingRule from_keyspace */ + from_keyspace?: (string|null); - /** Shard name */ - name?: (string|null); + /** ShardRoutingRule to_keyspace */ + to_keyspace?: (string|null); - /** Shard shard */ - shard?: (topodata.IShard|null); + /** ShardRoutingRule shard */ + shard?: (string|null); } - /** Represents a Shard. */ - class Shard implements IShard { + /** Represents a ShardRoutingRule. */ + class ShardRoutingRule implements IShardRoutingRule { /** - * Constructs a new Shard. + * Constructs a new ShardRoutingRule. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShard); + constructor(properties?: vschema.IShardRoutingRule); - /** Shard keyspace. */ - public keyspace: string; + /** ShardRoutingRule from_keyspace. */ + public from_keyspace: string; - /** Shard name. */ - public name: string; + /** ShardRoutingRule to_keyspace. */ + public to_keyspace: string; - /** Shard shard. */ - public shard?: (topodata.IShard|null); + /** ShardRoutingRule shard. */ + public shard: string; /** - * Creates a new Shard instance using the specified properties. + * Creates a new ShardRoutingRule instance using the specified properties. * @param [properties] Properties to set - * @returns Shard instance + * @returns ShardRoutingRule instance */ - public static create(properties?: vtctldata.IShard): vtctldata.Shard; + public static create(properties?: vschema.IShardRoutingRule): vschema.ShardRoutingRule; /** - * Encodes the specified Shard message. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. - * @param message Shard message or plain object to encode + * Encodes the specified ShardRoutingRule message. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * @param message ShardRoutingRule message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShard, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vschema.IShardRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Shard message, length delimited. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. - * @param message Shard message or plain object to encode + * Encodes the specified ShardRoutingRule message, length delimited. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * @param message ShardRoutingRule message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShard, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vschema.IShardRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Shard message from the specified reader or buffer. + * Decodes a ShardRoutingRule message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Shard + * @returns ShardRoutingRule * @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.Shard; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vschema.ShardRoutingRule; /** - * Decodes a Shard message from the specified reader or buffer, length delimited. + * Decodes a ShardRoutingRule message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Shard + * @returns ShardRoutingRule * @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.Shard; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vschema.ShardRoutingRule; /** - * Verifies a Shard message. + * Verifies a ShardRoutingRule 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 Shard message from a plain object. Also converts values to their respective internal types. + * Creates a ShardRoutingRule message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Shard + * @returns ShardRoutingRule */ - public static fromObject(object: { [k: string]: any }): vtctldata.Shard; + public static fromObject(object: { [k: string]: any }): vschema.ShardRoutingRule; /** - * Creates a plain object from a Shard message. Also converts values to other types if specified. - * @param message Shard + * Creates a plain object from a ShardRoutingRule message. Also converts values to other types if specified. + * @param message ShardRoutingRule * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.Shard, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vschema.ShardRoutingRule, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Shard to JSON. + * Converts this ShardRoutingRule to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Shard + * Gets the default type url for ShardRoutingRule * @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); +/** Namespace vtctldata. */ +export namespace vtctldata { - /** Workflow shard_streams */ - shard_streams?: ({ [k: string]: vtctldata.Workflow.IShardStream }|null); + /** Properties of an ExecuteVtctlCommandRequest. */ + interface IExecuteVtctlCommandRequest { - /** Workflow workflow_type */ - workflow_type?: (string|null); + /** ExecuteVtctlCommandRequest args */ + args?: (string[]|null); - /** Workflow workflow_sub_type */ - workflow_sub_type?: (string|null); + /** ExecuteVtctlCommandRequest action_timeout */ + action_timeout?: (number|Long|null); } - /** Represents a Workflow. */ - class Workflow implements IWorkflow { + /** Represents an ExecuteVtctlCommandRequest. */ + class ExecuteVtctlCommandRequest implements IExecuteVtctlCommandRequest { /** - * Constructs a new Workflow. + * Constructs a new ExecuteVtctlCommandRequest. * @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); - - /** Workflow max_v_replication_lag. */ - public max_v_replication_lag: (number|Long); - - /** Workflow shard_streams. */ - public shard_streams: { [k: string]: vtctldata.Workflow.IShardStream }; + constructor(properties?: vtctldata.IExecuteVtctlCommandRequest); - /** Workflow workflow_type. */ - public workflow_type: string; + /** ExecuteVtctlCommandRequest args. */ + public args: string[]; - /** Workflow workflow_sub_type. */ - public workflow_sub_type: string; + /** ExecuteVtctlCommandRequest action_timeout. */ + public action_timeout: (number|Long); /** - * Creates a new Workflow instance using the specified properties. + * Creates a new ExecuteVtctlCommandRequest instance using the specified properties. * @param [properties] Properties to set - * @returns Workflow instance + * @returns ExecuteVtctlCommandRequest instance */ - public static create(properties?: vtctldata.IWorkflow): vtctldata.Workflow; + public static create(properties?: vtctldata.IExecuteVtctlCommandRequest): vtctldata.ExecuteVtctlCommandRequest; /** - * Encodes the specified Workflow message. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. - * @param message Workflow message or plain object to encode + * Encodes the specified ExecuteVtctlCommandRequest message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * @param message ExecuteVtctlCommandRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IWorkflow, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteVtctlCommandRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified Workflow message, length delimited. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. - * @param message Workflow message or plain object to encode + * Encodes the specified ExecuteVtctlCommandRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * @param message ExecuteVtctlCommandRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IWorkflow, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteVtctlCommandRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a Workflow message from the specified reader or buffer. + * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Workflow + * @returns ExecuteVtctlCommandRequest * @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.Workflow; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteVtctlCommandRequest; /** - * Decodes a Workflow message from the specified reader or buffer, length delimited. + * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Workflow + * @returns ExecuteVtctlCommandRequest * @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.Workflow; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteVtctlCommandRequest; /** - * Verifies a Workflow message. + * Verifies an ExecuteVtctlCommandRequest 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 Workflow message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteVtctlCommandRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Workflow + * @returns ExecuteVtctlCommandRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.Workflow; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteVtctlCommandRequest; /** - * Creates a plain object from a Workflow message. Also converts values to other types if specified. - * @param message Workflow + * Creates a plain object from an ExecuteVtctlCommandRequest message. Also converts values to other types if specified. + * @param message ExecuteVtctlCommandRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.Workflow, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteVtctlCommandRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Workflow to JSON. + * Converts this ExecuteVtctlCommandRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Workflow + * Gets the default type url for ExecuteVtctlCommandRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace Workflow { - - /** Properties of a ReplicationLocation. */ - interface IReplicationLocation { + /** Properties of an ExecuteVtctlCommandResponse. */ + interface IExecuteVtctlCommandResponse { - /** ReplicationLocation keyspace */ - keyspace?: (string|null); + /** ExecuteVtctlCommandResponse event */ + event?: (logutil.IEvent|null); + } - /** ReplicationLocation shards */ - shards?: (string[]|null); - } + /** Represents an ExecuteVtctlCommandResponse. */ + class ExecuteVtctlCommandResponse implements IExecuteVtctlCommandResponse { - /** Represents a ReplicationLocation. */ - class ReplicationLocation implements IReplicationLocation { + /** + * Constructs a new ExecuteVtctlCommandResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IExecuteVtctlCommandResponse); - /** - * Constructs a new ReplicationLocation. - * @param [properties] Properties to set - */ - constructor(properties?: vtctldata.Workflow.IReplicationLocation); + /** ExecuteVtctlCommandResponse event. */ + public event?: (logutil.IEvent|null); - /** ReplicationLocation keyspace. */ - public keyspace: string; + /** + * Creates a new ExecuteVtctlCommandResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ExecuteVtctlCommandResponse instance + */ + public static create(properties?: vtctldata.IExecuteVtctlCommandResponse): vtctldata.ExecuteVtctlCommandResponse; - /** ReplicationLocation shards. */ - public shards: string[]; + /** + * Encodes the specified ExecuteVtctlCommandResponse message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. + * @param message ExecuteVtctlCommandResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IExecuteVtctlCommandResponse, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Creates a new ReplicationLocation instance using the specified properties. - * @param [properties] Properties to set - * @returns ReplicationLocation instance - */ - public static create(properties?: vtctldata.Workflow.IReplicationLocation): vtctldata.Workflow.ReplicationLocation; + /** + * Encodes the specified ExecuteVtctlCommandResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. + * @param message ExecuteVtctlCommandResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IExecuteVtctlCommandResponse, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Encodes the specified ReplicationLocation message. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. - * @param message ReplicationLocation message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: vtctldata.Workflow.IReplicationLocation, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ExecuteVtctlCommandResponse + * @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.ExecuteVtctlCommandResponse; - /** - * Encodes the specified ReplicationLocation message, length delimited. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. - * @param message ReplicationLocation message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: vtctldata.Workflow.IReplicationLocation, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ExecuteVtctlCommandResponse + * @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.ExecuteVtctlCommandResponse; - /** - * Decodes a ReplicationLocation message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns ReplicationLocation - * @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.Workflow.ReplicationLocation; + /** + * Verifies an ExecuteVtctlCommandResponse 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); - /** - * Decodes a ReplicationLocation message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns ReplicationLocation - * @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.Workflow.ReplicationLocation; + /** + * Creates an ExecuteVtctlCommandResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ExecuteVtctlCommandResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteVtctlCommandResponse; - /** - * Verifies a ReplicationLocation 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 plain object from an ExecuteVtctlCommandResponse message. Also converts values to other types if specified. + * @param message ExecuteVtctlCommandResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ExecuteVtctlCommandResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; - /** - * Creates a ReplicationLocation message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns ReplicationLocation - */ - public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.ReplicationLocation; + /** + * Converts this ExecuteVtctlCommandResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; - /** - * Creates a plain object from a ReplicationLocation message. Also converts values to other types if specified. - * @param message ReplicationLocation - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: vtctldata.Workflow.ReplicationLocation, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** + * Gets the default type url for ExecuteVtctlCommandResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } - /** - * Converts this ReplicationLocation to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** MaterializationIntent enum. */ + enum MaterializationIntent { + CUSTOM = 0, + MOVETABLES = 1, + CREATELOOKUPINDEX = 2 + } - /** - * Gets the default type url for ReplicationLocation - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } + /** Properties of a TableMaterializeSettings. */ + interface ITableMaterializeSettings { - /** Properties of a ShardStream. */ - interface IShardStream { + /** TableMaterializeSettings target_table */ + target_table?: (string|null); - /** ShardStream streams */ - streams?: (vtctldata.Workflow.IStream[]|null); + /** TableMaterializeSettings source_expression */ + source_expression?: (string|null); - /** ShardStream tablet_controls */ - tablet_controls?: (topodata.Shard.ITabletControl[]|null); + /** TableMaterializeSettings create_ddl */ + create_ddl?: (string|null); + } - /** ShardStream is_primary_serving */ - is_primary_serving?: (boolean|null); - } + /** Represents a TableMaterializeSettings. */ + class TableMaterializeSettings implements ITableMaterializeSettings { - /** Represents a ShardStream. */ - class ShardStream implements IShardStream { + /** + * Constructs a new TableMaterializeSettings. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.ITableMaterializeSettings); - /** - * Constructs a new ShardStream. - * @param [properties] Properties to set - */ - constructor(properties?: vtctldata.Workflow.IShardStream); + /** TableMaterializeSettings target_table. */ + public target_table: string; - /** ShardStream streams. */ - public streams: vtctldata.Workflow.IStream[]; + /** TableMaterializeSettings source_expression. */ + public source_expression: string; - /** ShardStream tablet_controls. */ - public tablet_controls: topodata.Shard.ITabletControl[]; + /** TableMaterializeSettings create_ddl. */ + public create_ddl: string; - /** ShardStream is_primary_serving. */ - public is_primary_serving: boolean; + /** + * Creates a new TableMaterializeSettings instance using the specified properties. + * @param [properties] Properties to set + * @returns TableMaterializeSettings instance + */ + public static create(properties?: vtctldata.ITableMaterializeSettings): vtctldata.TableMaterializeSettings; - /** - * Creates a new ShardStream instance using the specified properties. - * @param [properties] Properties to set - * @returns ShardStream instance - */ - public static create(properties?: vtctldata.Workflow.IShardStream): vtctldata.Workflow.ShardStream; + /** + * Encodes the specified TableMaterializeSettings message. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * @param message TableMaterializeSettings message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.ITableMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Encodes the specified ShardStream message. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. - * @param message ShardStream message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: vtctldata.Workflow.IShardStream, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Encodes the specified TableMaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * @param message TableMaterializeSettings message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.ITableMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Encodes the specified ShardStream message, length delimited. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. - * @param message ShardStream message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: vtctldata.Workflow.IShardStream, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Decodes a TableMaterializeSettings message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns TableMaterializeSettings + * @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.TableMaterializeSettings; - /** - * Decodes a ShardStream message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns ShardStream - * @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.Workflow.ShardStream; + /** + * Decodes a TableMaterializeSettings message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns TableMaterializeSettings + * @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.TableMaterializeSettings; - /** - * Decodes a ShardStream message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns ShardStream - * @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.Workflow.ShardStream; + /** + * Verifies a TableMaterializeSettings 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); - /** - * Verifies a ShardStream 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 TableMaterializeSettings message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns TableMaterializeSettings + */ + public static fromObject(object: { [k: string]: any }): vtctldata.TableMaterializeSettings; - /** - * Creates a ShardStream message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns ShardStream - */ - public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.ShardStream; + /** + * Creates a plain object from a TableMaterializeSettings message. Also converts values to other types if specified. + * @param message TableMaterializeSettings + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.TableMaterializeSettings, options?: $protobuf.IConversionOptions): { [k: string]: any }; - /** - * Creates a plain object from a ShardStream message. Also converts values to other types if specified. - * @param message ShardStream - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: vtctldata.Workflow.ShardStream, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** + * Converts this TableMaterializeSettings to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; - /** - * Converts this ShardStream to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** + * Gets the default type url for TableMaterializeSettings + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } - /** - * Gets the default type url for ShardStream - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } + /** Properties of a MaterializeSettings. */ + interface IMaterializeSettings { - /** Properties of a Stream. */ - interface IStream { + /** MaterializeSettings workflow */ + workflow?: (string|null); - /** Stream id */ - id?: (number|Long|null); + /** MaterializeSettings source_keyspace */ + source_keyspace?: (string|null); - /** Stream shard */ - shard?: (string|null); + /** MaterializeSettings target_keyspace */ + target_keyspace?: (string|null); - /** Stream tablet */ - tablet?: (topodata.ITabletAlias|null); + /** MaterializeSettings stop_after_copy */ + stop_after_copy?: (boolean|null); - /** Stream binlog_source */ - binlog_source?: (binlogdata.IBinlogSource|null); + /** MaterializeSettings table_settings */ + table_settings?: (vtctldata.ITableMaterializeSettings[]|null); - /** Stream position */ - position?: (string|null); + /** MaterializeSettings cell */ + cell?: (string|null); - /** Stream stop_position */ - stop_position?: (string|null); + /** MaterializeSettings tablet_types */ + tablet_types?: (string|null); - /** Stream state */ - state?: (string|null); + /** MaterializeSettings external_cluster */ + external_cluster?: (string|null); - /** Stream db_name */ - db_name?: (string|null); + /** MaterializeSettings materialization_intent */ + materialization_intent?: (vtctldata.MaterializationIntent|null); - /** Stream transaction_timestamp */ - transaction_timestamp?: (vttime.ITime|null); + /** MaterializeSettings source_time_zone */ + source_time_zone?: (string|null); - /** Stream time_updated */ - time_updated?: (vttime.ITime|null); + /** MaterializeSettings target_time_zone */ + target_time_zone?: (string|null); - /** Stream message */ - message?: (string|null); + /** MaterializeSettings source_shards */ + source_shards?: (string[]|null); - /** Stream copy_states */ - copy_states?: (vtctldata.Workflow.Stream.ICopyState[]|null); + /** MaterializeSettings on_ddl */ + on_ddl?: (string|null); - /** Stream logs */ - logs?: (vtctldata.Workflow.Stream.ILog[]|null); + /** MaterializeSettings defer_secondary_keys */ + defer_secondary_keys?: (boolean|null); - /** Stream log_fetch_error */ - log_fetch_error?: (string|null); + /** MaterializeSettings tablet_selection_preference */ + tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); + } - /** Stream tags */ - tags?: (string[]|null); - } + /** Represents a MaterializeSettings. */ + class MaterializeSettings implements IMaterializeSettings { - /** Represents a Stream. */ - class Stream implements IStream { + /** + * Constructs a new MaterializeSettings. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IMaterializeSettings); - /** - * Constructs a new Stream. - * @param [properties] Properties to set - */ - constructor(properties?: vtctldata.Workflow.IStream); + /** MaterializeSettings workflow. */ + public workflow: string; - /** Stream id. */ - public id: (number|Long); + /** MaterializeSettings source_keyspace. */ + public source_keyspace: string; - /** Stream shard. */ - public shard: string; + /** MaterializeSettings target_keyspace. */ + public target_keyspace: string; - /** Stream tablet. */ - public tablet?: (topodata.ITabletAlias|null); + /** MaterializeSettings stop_after_copy. */ + public stop_after_copy: boolean; - /** Stream binlog_source. */ - public binlog_source?: (binlogdata.IBinlogSource|null); + /** MaterializeSettings table_settings. */ + public table_settings: vtctldata.ITableMaterializeSettings[]; - /** Stream position. */ - public position: string; + /** MaterializeSettings cell. */ + public cell: string; - /** Stream stop_position. */ - public stop_position: string; + /** MaterializeSettings tablet_types. */ + public tablet_types: string; - /** Stream state. */ - public state: string; + /** MaterializeSettings external_cluster. */ + public external_cluster: string; - /** Stream db_name. */ - public db_name: string; + /** MaterializeSettings materialization_intent. */ + public materialization_intent: vtctldata.MaterializationIntent; - /** Stream transaction_timestamp. */ - public transaction_timestamp?: (vttime.ITime|null); + /** MaterializeSettings source_time_zone. */ + public source_time_zone: string; - /** Stream time_updated. */ - public time_updated?: (vttime.ITime|null); + /** MaterializeSettings target_time_zone. */ + public target_time_zone: string; - /** Stream message. */ - public message: string; + /** MaterializeSettings source_shards. */ + public source_shards: string[]; - /** Stream copy_states. */ - public copy_states: vtctldata.Workflow.Stream.ICopyState[]; + /** MaterializeSettings on_ddl. */ + public on_ddl: string; - /** Stream logs. */ - public logs: vtctldata.Workflow.Stream.ILog[]; + /** MaterializeSettings defer_secondary_keys. */ + public defer_secondary_keys: boolean; - /** Stream log_fetch_error. */ - public log_fetch_error: string; + /** MaterializeSettings tablet_selection_preference. */ + public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; - /** Stream tags. */ - public tags: string[]; + /** + * Creates a new MaterializeSettings instance using the specified properties. + * @param [properties] Properties to set + * @returns MaterializeSettings instance + */ + public static create(properties?: vtctldata.IMaterializeSettings): vtctldata.MaterializeSettings; - /** - * Creates a new Stream instance using the specified properties. - * @param [properties] Properties to set - * @returns Stream instance - */ - public static create(properties?: vtctldata.Workflow.IStream): vtctldata.Workflow.Stream; + /** + * Encodes the specified MaterializeSettings message. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * @param message MaterializeSettings message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Encodes the specified Stream message. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. - * @param message Stream message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: vtctldata.Workflow.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Encodes the specified MaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * @param message MaterializeSettings message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IMaterializeSettings, writer?: $protobuf.Writer): $protobuf.Writer; - /** - * Encodes the specified Stream message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. - * @param message Stream message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: vtctldata.Workflow.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Decodes a MaterializeSettings message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns MaterializeSettings + * @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.MaterializeSettings; - /** - * Decodes a Stream message from the specified reader or buffer. + /** + * Decodes a MaterializeSettings message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns MaterializeSettings + * @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.MaterializeSettings; + + /** + * Verifies a MaterializeSettings 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 MaterializeSettings message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns MaterializeSettings + */ + public static fromObject(object: { [k: string]: any }): vtctldata.MaterializeSettings; + + /** + * Creates a plain object from a MaterializeSettings message. Also converts values to other types if specified. + * @param message MaterializeSettings + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.MaterializeSettings, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this MaterializeSettings to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for MaterializeSettings + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a Keyspace. */ + interface IKeyspace { + + /** Keyspace name */ + name?: (string|null); + + /** Keyspace keyspace */ + keyspace?: (topodata.IKeyspace|null); + } + + /** Represents a Keyspace. */ + class Keyspace implements IKeyspace { + + /** + * Constructs a new Keyspace. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IKeyspace); + + /** Keyspace name. */ + public name: string; + + /** Keyspace keyspace. */ + public keyspace?: (topodata.IKeyspace|null); + + /** + * Creates a new Keyspace instance using the specified properties. + * @param [properties] Properties to set + * @returns Keyspace instance + */ + public static create(properties?: vtctldata.IKeyspace): vtctldata.Keyspace; + + /** + * Encodes the specified Keyspace message. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * @param message Keyspace message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * @param message Keyspace message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IKeyspace, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Keyspace message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Keyspace + * @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.Keyspace; + + /** + * Decodes a Keyspace message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Keyspace + * @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.Keyspace; + + /** + * Verifies a Keyspace 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 Keyspace message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Keyspace + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Keyspace; + + /** + * Creates a plain object from a Keyspace message. Also converts values to other types if specified. + * @param message Keyspace + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Keyspace, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Keyspace to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Keyspace + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a Shard. */ + interface IShard { + + /** Shard keyspace */ + keyspace?: (string|null); + + /** Shard name */ + name?: (string|null); + + /** Shard shard */ + shard?: (topodata.IShard|null); + } + + /** Represents a Shard. */ + class Shard implements IShard { + + /** + * Constructs a new Shard. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IShard); + + /** Shard keyspace. */ + public keyspace: string; + + /** Shard name. */ + public name: string; + + /** Shard shard. */ + public shard?: (topodata.IShard|null); + + /** + * Creates a new Shard instance using the specified properties. + * @param [properties] Properties to set + * @returns Shard instance + */ + public static create(properties?: vtctldata.IShard): vtctldata.Shard; + + /** + * Encodes the specified Shard message. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * @param message Shard message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IShard, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Shard message, length delimited. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * @param message Shard message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IShard, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Shard message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Shard + * @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.Shard; + + /** + * Decodes a Shard message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Shard + * @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.Shard; + + /** + * Verifies a Shard 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 Shard message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Shard + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Shard; + + /** + * Creates a plain object from a Shard message. Also converts values to other types if specified. + * @param message Shard + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Shard, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Shard to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Shard + * @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); + } + + /** 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); + + /** Workflow max_v_replication_lag. */ + public max_v_replication_lag: (number|Long); + + /** Workflow shard_streams. */ + public shard_streams: { [k: string]: vtctldata.Workflow.IShardStream }; + + /** Workflow workflow_type. */ + public workflow_type: string; + + /** Workflow workflow_sub_type. */ + public workflow_sub_type: string; + + /** + * Creates a new Workflow instance using the specified properties. + * @param [properties] Properties to set + * @returns Workflow instance + */ + public static create(properties?: vtctldata.IWorkflow): vtctldata.Workflow; + + /** + * Encodes the specified Workflow message. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * @param message Workflow message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IWorkflow, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Workflow message, length delimited. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * @param message Workflow message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IWorkflow, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Workflow message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Workflow + * @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.Workflow; + + /** + * Decodes a Workflow message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Workflow + * @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.Workflow; + + /** + * Verifies a Workflow 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 Workflow message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Workflow + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Workflow; + + /** + * Creates a plain object from a Workflow message. Also converts values to other types if specified. + * @param message Workflow + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Workflow, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Workflow to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Workflow + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + namespace Workflow { + + /** Properties of a ReplicationLocation. */ + interface IReplicationLocation { + + /** ReplicationLocation keyspace */ + keyspace?: (string|null); + + /** ReplicationLocation shards */ + shards?: (string[]|null); + } + + /** Represents a ReplicationLocation. */ + class ReplicationLocation implements IReplicationLocation { + + /** + * Constructs a new ReplicationLocation. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.Workflow.IReplicationLocation); + + /** ReplicationLocation keyspace. */ + public keyspace: string; + + /** ReplicationLocation shards. */ + public shards: string[]; + + /** + * Creates a new ReplicationLocation instance using the specified properties. + * @param [properties] Properties to set + * @returns ReplicationLocation instance + */ + public static create(properties?: vtctldata.Workflow.IReplicationLocation): vtctldata.Workflow.ReplicationLocation; + + /** + * Encodes the specified ReplicationLocation message. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. + * @param message ReplicationLocation message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.Workflow.IReplicationLocation, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ReplicationLocation message, length delimited. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. + * @param message ReplicationLocation message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.Workflow.IReplicationLocation, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ReplicationLocation message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns Stream + * @returns ReplicationLocation * @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.Workflow.Stream; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.Workflow.ReplicationLocation; /** - * Decodes a Stream message from the specified reader or buffer, length delimited. + * Decodes a ReplicationLocation message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns Stream + * @returns ReplicationLocation * @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.Workflow.Stream; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.Workflow.ReplicationLocation; /** - * Verifies a Stream message. + * Verifies a ReplicationLocation 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 Stream message from a plain object. Also converts values to their respective internal types. + * Creates a ReplicationLocation message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns Stream + * @returns ReplicationLocation */ - public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.Stream; + public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.ReplicationLocation; /** - * Creates a plain object from a Stream message. Also converts values to other types if specified. - * @param message Stream + * Creates a plain object from a ReplicationLocation message. Also converts values to other types if specified. + * @param message ReplicationLocation * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.Workflow.Stream, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.Workflow.ReplicationLocation, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this Stream to JSON. + * Converts this ReplicationLocation to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for Stream + * Gets the default type url for ReplicationLocation * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace Stream { + /** Properties of a ShardStream. */ + interface IShardStream { - /** Properties of a CopyState. */ - interface ICopyState { + /** ShardStream streams */ + streams?: (vtctldata.Workflow.IStream[]|null); - /** CopyState table */ - table?: (string|null); + /** ShardStream tablet_controls */ + tablet_controls?: (topodata.Shard.ITabletControl[]|null); - /** CopyState last_pk */ - last_pk?: (string|null); - } + /** ShardStream is_primary_serving */ + is_primary_serving?: (boolean|null); + } - /** Represents a CopyState. */ - class CopyState implements ICopyState { + /** Represents a ShardStream. */ + class ShardStream implements IShardStream { - /** - * Constructs a new CopyState. - * @param [properties] Properties to set - */ - constructor(properties?: vtctldata.Workflow.Stream.ICopyState); + /** + * Constructs a new ShardStream. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.Workflow.IShardStream); - /** CopyState table. */ - public table: string; + /** ShardStream streams. */ + public streams: vtctldata.Workflow.IStream[]; - /** CopyState last_pk. */ - public last_pk: string; + /** ShardStream tablet_controls. */ + public tablet_controls: topodata.Shard.ITabletControl[]; - /** - * Creates a new CopyState instance using the specified properties. - * @param [properties] Properties to set + /** ShardStream is_primary_serving. */ + public is_primary_serving: boolean; + + /** + * Creates a new ShardStream instance using the specified properties. + * @param [properties] Properties to set + * @returns ShardStream instance + */ + public static create(properties?: vtctldata.Workflow.IShardStream): vtctldata.Workflow.ShardStream; + + /** + * Encodes the specified ShardStream message. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. + * @param message ShardStream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.Workflow.IShardStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ShardStream message, length delimited. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. + * @param message ShardStream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.Workflow.IShardStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ShardStream message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ShardStream + * @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.Workflow.ShardStream; + + /** + * Decodes a ShardStream message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ShardStream + * @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.Workflow.ShardStream; + + /** + * Verifies a ShardStream 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 ShardStream message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ShardStream + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.ShardStream; + + /** + * Creates a plain object from a ShardStream message. Also converts values to other types if specified. + * @param message ShardStream + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Workflow.ShardStream, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ShardStream to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ShardStream + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a Stream. */ + interface IStream { + + /** Stream id */ + id?: (number|Long|null); + + /** Stream shard */ + shard?: (string|null); + + /** Stream tablet */ + tablet?: (topodata.ITabletAlias|null); + + /** Stream binlog_source */ + binlog_source?: (binlogdata.IBinlogSource|null); + + /** Stream position */ + position?: (string|null); + + /** Stream stop_position */ + stop_position?: (string|null); + + /** Stream state */ + state?: (string|null); + + /** Stream db_name */ + db_name?: (string|null); + + /** Stream transaction_timestamp */ + transaction_timestamp?: (vttime.ITime|null); + + /** Stream time_updated */ + time_updated?: (vttime.ITime|null); + + /** Stream message */ + message?: (string|null); + + /** Stream copy_states */ + copy_states?: (vtctldata.Workflow.Stream.ICopyState[]|null); + + /** Stream logs */ + logs?: (vtctldata.Workflow.Stream.ILog[]|null); + + /** Stream log_fetch_error */ + log_fetch_error?: (string|null); + + /** Stream tags */ + tags?: (string[]|null); + } + + /** Represents a Stream. */ + class Stream implements IStream { + + /** + * Constructs a new Stream. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.Workflow.IStream); + + /** Stream id. */ + public id: (number|Long); + + /** Stream shard. */ + public shard: string; + + /** Stream tablet. */ + public tablet?: (topodata.ITabletAlias|null); + + /** Stream binlog_source. */ + public binlog_source?: (binlogdata.IBinlogSource|null); + + /** Stream position. */ + public position: string; + + /** Stream stop_position. */ + public stop_position: string; + + /** Stream state. */ + public state: string; + + /** Stream db_name. */ + public db_name: string; + + /** Stream transaction_timestamp. */ + public transaction_timestamp?: (vttime.ITime|null); + + /** Stream time_updated. */ + public time_updated?: (vttime.ITime|null); + + /** Stream message. */ + public message: string; + + /** Stream copy_states. */ + public copy_states: vtctldata.Workflow.Stream.ICopyState[]; + + /** Stream logs. */ + public logs: vtctldata.Workflow.Stream.ILog[]; + + /** Stream log_fetch_error. */ + public log_fetch_error: string; + + /** Stream tags. */ + public tags: string[]; + + /** + * Creates a new Stream instance using the specified properties. + * @param [properties] Properties to set + * @returns Stream instance + */ + public static create(properties?: vtctldata.Workflow.IStream): vtctldata.Workflow.Stream; + + /** + * Encodes the specified Stream message. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. + * @param message Stream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.Workflow.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Stream message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. + * @param message Stream message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.Workflow.IStream, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Stream message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Stream + * @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.Workflow.Stream; + + /** + * Decodes a Stream message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Stream + * @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.Workflow.Stream; + + /** + * Verifies a Stream 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 Stream message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Stream + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.Stream; + + /** + * Creates a plain object from a Stream message. Also converts values to other types if specified. + * @param message Stream + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Workflow.Stream, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Stream to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Stream + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + namespace Stream { + + /** Properties of a CopyState. */ + interface ICopyState { + + /** CopyState table */ + table?: (string|null); + + /** CopyState last_pk */ + last_pk?: (string|null); + } + + /** Represents a CopyState. */ + class CopyState implements ICopyState { + + /** + * Constructs a new CopyState. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.Workflow.Stream.ICopyState); + + /** CopyState table. */ + public table: string; + + /** CopyState last_pk. */ + public last_pk: string; + + /** + * Creates a new CopyState instance using the specified properties. + * @param [properties] Properties to set * @returns CopyState instance */ public static create(properties?: vtctldata.Workflow.Stream.ICopyState): vtctldata.Workflow.Stream.CopyState; @@ -42011,17854 +42915,19636 @@ export namespace vtctldata { /** Log updated_at. */ public updated_at?: (vttime.ITime|null); - /** Log message. */ - public message: string; + /** Log message. */ + public message: string; + + /** Log count. */ + public count: (number|Long); + + /** + * Creates a new Log instance using the specified properties. + * @param [properties] Properties to set + * @returns Log instance + */ + public static create(properties?: vtctldata.Workflow.Stream.ILog): vtctldata.Workflow.Stream.Log; + + /** + * Encodes the specified Log message. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. + * @param message Log message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.Workflow.Stream.ILog, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified Log message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. + * @param message Log message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.Workflow.Stream.ILog, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a Log message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns Log + * @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.Workflow.Stream.Log; + + /** + * Decodes a Log message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns Log + * @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.Workflow.Stream.Log; + + /** + * Verifies a Log 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 Log message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns Log + */ + public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.Stream.Log; + + /** + * Creates a plain object from a Log message. Also converts values to other types if specified. + * @param message Log + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.Workflow.Stream.Log, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this Log to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for Log + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + } + } + + /** Properties of an AddCellInfoRequest. */ + interface IAddCellInfoRequest { + + /** AddCellInfoRequest name */ + name?: (string|null); + + /** AddCellInfoRequest cell_info */ + cell_info?: (topodata.ICellInfo|null); + } + + /** Represents an AddCellInfoRequest. */ + class AddCellInfoRequest implements IAddCellInfoRequest { + + /** + * Constructs a new AddCellInfoRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IAddCellInfoRequest); + + /** AddCellInfoRequest name. */ + public name: string; + + /** AddCellInfoRequest cell_info. */ + public cell_info?: (topodata.ICellInfo|null); + + /** + * Creates a new AddCellInfoRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns AddCellInfoRequest instance + */ + public static create(properties?: vtctldata.IAddCellInfoRequest): vtctldata.AddCellInfoRequest; + + /** + * Encodes the specified AddCellInfoRequest message. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * @param message AddCellInfoRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IAddCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified AddCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * @param message AddCellInfoRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IAddCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an AddCellInfoRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns AddCellInfoRequest + * @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.AddCellInfoRequest; + + /** + * Decodes an AddCellInfoRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns AddCellInfoRequest + * @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.AddCellInfoRequest; + + /** + * Verifies an AddCellInfoRequest 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 AddCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns AddCellInfoRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.AddCellInfoRequest; + + /** + * Creates a plain object from an AddCellInfoRequest message. Also converts values to other types if specified. + * @param message AddCellInfoRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.AddCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this AddCellInfoRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for AddCellInfoRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an AddCellInfoResponse. */ + interface IAddCellInfoResponse { + } + + /** Represents an AddCellInfoResponse. */ + class AddCellInfoResponse implements IAddCellInfoResponse { + + /** + * Constructs a new AddCellInfoResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IAddCellInfoResponse); + + /** + * Creates a new AddCellInfoResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns AddCellInfoResponse instance + */ + public static create(properties?: vtctldata.IAddCellInfoResponse): vtctldata.AddCellInfoResponse; + + /** + * Encodes the specified AddCellInfoResponse message. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * @param message AddCellInfoResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IAddCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified AddCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * @param message AddCellInfoResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IAddCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an AddCellInfoResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns AddCellInfoResponse + * @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.AddCellInfoResponse; + + /** + * Decodes an AddCellInfoResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns AddCellInfoResponse + * @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.AddCellInfoResponse; + + /** + * Verifies an AddCellInfoResponse 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 AddCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns AddCellInfoResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.AddCellInfoResponse; + + /** + * Creates a plain object from an AddCellInfoResponse message. Also converts values to other types if specified. + * @param message AddCellInfoResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.AddCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this AddCellInfoResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for AddCellInfoResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an AddCellsAliasRequest. */ + interface IAddCellsAliasRequest { + + /** AddCellsAliasRequest name */ + name?: (string|null); + + /** AddCellsAliasRequest cells */ + cells?: (string[]|null); + } + + /** Represents an AddCellsAliasRequest. */ + class AddCellsAliasRequest implements IAddCellsAliasRequest { + + /** + * Constructs a new AddCellsAliasRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IAddCellsAliasRequest); + + /** AddCellsAliasRequest name. */ + public name: string; + + /** AddCellsAliasRequest cells. */ + public cells: string[]; + + /** + * Creates a new AddCellsAliasRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns AddCellsAliasRequest instance + */ + public static create(properties?: vtctldata.IAddCellsAliasRequest): vtctldata.AddCellsAliasRequest; + + /** + * Encodes the specified AddCellsAliasRequest message. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * @param message AddCellsAliasRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IAddCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified AddCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * @param message AddCellsAliasRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IAddCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an AddCellsAliasRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns AddCellsAliasRequest + * @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.AddCellsAliasRequest; + + /** + * Decodes an AddCellsAliasRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns AddCellsAliasRequest + * @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.AddCellsAliasRequest; + + /** + * Verifies an AddCellsAliasRequest 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 AddCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns AddCellsAliasRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.AddCellsAliasRequest; + + /** + * Creates a plain object from an AddCellsAliasRequest message. Also converts values to other types if specified. + * @param message AddCellsAliasRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.AddCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this AddCellsAliasRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for AddCellsAliasRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an AddCellsAliasResponse. */ + interface IAddCellsAliasResponse { + } + + /** Represents an AddCellsAliasResponse. */ + class AddCellsAliasResponse implements IAddCellsAliasResponse { + + /** + * Constructs a new AddCellsAliasResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IAddCellsAliasResponse); + + /** + * Creates a new AddCellsAliasResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns AddCellsAliasResponse instance + */ + public static create(properties?: vtctldata.IAddCellsAliasResponse): vtctldata.AddCellsAliasResponse; + + /** + * Encodes the specified AddCellsAliasResponse message. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * @param message AddCellsAliasResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IAddCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified AddCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * @param message AddCellsAliasResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IAddCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an AddCellsAliasResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns AddCellsAliasResponse + * @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.AddCellsAliasResponse; + + /** + * Decodes an AddCellsAliasResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns AddCellsAliasResponse + * @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.AddCellsAliasResponse; + + /** + * Verifies an AddCellsAliasResponse 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 AddCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns AddCellsAliasResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.AddCellsAliasResponse; + + /** + * Creates a plain object from an AddCellsAliasResponse message. Also converts values to other types if specified. + * @param message AddCellsAliasResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.AddCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this AddCellsAliasResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for AddCellsAliasResponse + * @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 { + + /** ApplyRoutingRulesRequest routing_rules */ + routing_rules?: (vschema.IRoutingRules|null); + + /** ApplyRoutingRulesRequest skip_rebuild */ + skip_rebuild?: (boolean|null); + + /** ApplyRoutingRulesRequest rebuild_cells */ + rebuild_cells?: (string[]|null); + } + + /** Represents an ApplyRoutingRulesRequest. */ + class ApplyRoutingRulesRequest implements IApplyRoutingRulesRequest { + + /** + * Constructs a new ApplyRoutingRulesRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyRoutingRulesRequest); + + /** ApplyRoutingRulesRequest routing_rules. */ + public routing_rules?: (vschema.IRoutingRules|null); + + /** ApplyRoutingRulesRequest skip_rebuild. */ + public skip_rebuild: boolean; + + /** ApplyRoutingRulesRequest rebuild_cells. */ + public rebuild_cells: string[]; + + /** + * Creates a new ApplyRoutingRulesRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyRoutingRulesRequest instance + */ + public static create(properties?: vtctldata.IApplyRoutingRulesRequest): vtctldata.ApplyRoutingRulesRequest; + + /** + * Encodes the specified ApplyRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * @param message ApplyRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * @param message ApplyRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyRoutingRulesRequest + * @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.ApplyRoutingRulesRequest; + + /** + * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyRoutingRulesRequest + * @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.ApplyRoutingRulesRequest; + + /** + * Verifies an ApplyRoutingRulesRequest 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 ApplyRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyRoutingRulesRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyRoutingRulesRequest; + + /** + * Creates a plain object from an ApplyRoutingRulesRequest message. Also converts values to other types if specified. + * @param message ApplyRoutingRulesRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyRoutingRulesRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyRoutingRulesRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplyRoutingRulesResponse. */ + interface IApplyRoutingRulesResponse { + } + + /** Represents an ApplyRoutingRulesResponse. */ + class ApplyRoutingRulesResponse implements IApplyRoutingRulesResponse { + + /** + * Constructs a new ApplyRoutingRulesResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyRoutingRulesResponse); + + /** + * Creates a new ApplyRoutingRulesResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyRoutingRulesResponse instance + */ + public static create(properties?: vtctldata.IApplyRoutingRulesResponse): vtctldata.ApplyRoutingRulesResponse; + + /** + * Encodes the specified ApplyRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * @param message ApplyRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * @param message ApplyRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyRoutingRulesResponse + * @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.ApplyRoutingRulesResponse; + + /** + * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyRoutingRulesResponse + * @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.ApplyRoutingRulesResponse; + + /** + * Verifies an ApplyRoutingRulesResponse 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 ApplyRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyRoutingRulesResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyRoutingRulesResponse; + + /** + * Creates a plain object from an ApplyRoutingRulesResponse message. Also converts values to other types if specified. + * @param message ApplyRoutingRulesResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyRoutingRulesResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyRoutingRulesResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplyShardRoutingRulesRequest. */ + interface IApplyShardRoutingRulesRequest { + + /** ApplyShardRoutingRulesRequest shard_routing_rules */ + shard_routing_rules?: (vschema.IShardRoutingRules|null); + + /** ApplyShardRoutingRulesRequest skip_rebuild */ + skip_rebuild?: (boolean|null); + + /** ApplyShardRoutingRulesRequest rebuild_cells */ + rebuild_cells?: (string[]|null); + } + + /** Represents an ApplyShardRoutingRulesRequest. */ + class ApplyShardRoutingRulesRequest implements IApplyShardRoutingRulesRequest { + + /** + * Constructs a new ApplyShardRoutingRulesRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyShardRoutingRulesRequest); + + /** ApplyShardRoutingRulesRequest shard_routing_rules. */ + public shard_routing_rules?: (vschema.IShardRoutingRules|null); + + /** ApplyShardRoutingRulesRequest skip_rebuild. */ + public skip_rebuild: boolean; + + /** ApplyShardRoutingRulesRequest rebuild_cells. */ + public rebuild_cells: string[]; + + /** + * Creates a new ApplyShardRoutingRulesRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyShardRoutingRulesRequest instance + */ + public static create(properties?: vtctldata.IApplyShardRoutingRulesRequest): vtctldata.ApplyShardRoutingRulesRequest; + + /** + * Encodes the specified ApplyShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * @param message ApplyShardRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * @param message ApplyShardRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyShardRoutingRulesRequest + * @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.ApplyShardRoutingRulesRequest; + + /** + * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyShardRoutingRulesRequest + * @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.ApplyShardRoutingRulesRequest; + + /** + * Verifies an ApplyShardRoutingRulesRequest 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 ApplyShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyShardRoutingRulesRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyShardRoutingRulesRequest; + + /** + * Creates a plain object from an ApplyShardRoutingRulesRequest message. Also converts values to other types if specified. + * @param message ApplyShardRoutingRulesRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyShardRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyShardRoutingRulesRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyShardRoutingRulesRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplyShardRoutingRulesResponse. */ + interface IApplyShardRoutingRulesResponse { + } + + /** Represents an ApplyShardRoutingRulesResponse. */ + class ApplyShardRoutingRulesResponse implements IApplyShardRoutingRulesResponse { + + /** + * Constructs a new ApplyShardRoutingRulesResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyShardRoutingRulesResponse); + + /** + * Creates a new ApplyShardRoutingRulesResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyShardRoutingRulesResponse instance + */ + public static create(properties?: vtctldata.IApplyShardRoutingRulesResponse): vtctldata.ApplyShardRoutingRulesResponse; + + /** + * Encodes the specified ApplyShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * @param message ApplyShardRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * @param message ApplyShardRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyShardRoutingRulesResponse + * @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.ApplyShardRoutingRulesResponse; + + /** + * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyShardRoutingRulesResponse + * @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.ApplyShardRoutingRulesResponse; + + /** + * Verifies an ApplyShardRoutingRulesResponse 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 ApplyShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyShardRoutingRulesResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyShardRoutingRulesResponse; + + /** + * Creates a plain object from an ApplyShardRoutingRulesResponse message. Also converts values to other types if specified. + * @param message ApplyShardRoutingRulesResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyShardRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyShardRoutingRulesResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyShardRoutingRulesResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplySchemaRequest. */ + interface IApplySchemaRequest { + + /** ApplySchemaRequest keyspace */ + keyspace?: (string|null); + + /** ApplySchemaRequest sql */ + sql?: (string[]|null); + + /** ApplySchemaRequest ddl_strategy */ + ddl_strategy?: (string|null); + + /** ApplySchemaRequest uuid_list */ + uuid_list?: (string[]|null); + + /** ApplySchemaRequest migration_context */ + migration_context?: (string|null); + + /** ApplySchemaRequest wait_replicas_timeout */ + wait_replicas_timeout?: (vttime.IDuration|null); + + /** ApplySchemaRequest skip_preflight */ + skip_preflight?: (boolean|null); + + /** ApplySchemaRequest caller_id */ + caller_id?: (vtrpc.ICallerID|null); + } + + /** Represents an ApplySchemaRequest. */ + class ApplySchemaRequest implements IApplySchemaRequest { + + /** + * Constructs a new ApplySchemaRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplySchemaRequest); + + /** ApplySchemaRequest keyspace. */ + public keyspace: string; + + /** ApplySchemaRequest sql. */ + public sql: string[]; + + /** ApplySchemaRequest ddl_strategy. */ + public ddl_strategy: string; + + /** ApplySchemaRequest uuid_list. */ + public uuid_list: string[]; + + /** ApplySchemaRequest migration_context. */ + public migration_context: string; + + /** ApplySchemaRequest wait_replicas_timeout. */ + public wait_replicas_timeout?: (vttime.IDuration|null); + + /** ApplySchemaRequest skip_preflight. */ + public skip_preflight: boolean; + + /** ApplySchemaRequest caller_id. */ + public caller_id?: (vtrpc.ICallerID|null); + + /** + * Creates a new ApplySchemaRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplySchemaRequest instance + */ + public static create(properties?: vtctldata.IApplySchemaRequest): vtctldata.ApplySchemaRequest; + + /** + * Encodes the specified ApplySchemaRequest message. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * @param message ApplySchemaRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplySchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplySchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * @param message ApplySchemaRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplySchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplySchemaRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplySchemaRequest + * @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.ApplySchemaRequest; + + /** + * Decodes an ApplySchemaRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplySchemaRequest + * @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.ApplySchemaRequest; + + /** + * Verifies an ApplySchemaRequest 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 ApplySchemaRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplySchemaRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplySchemaRequest; + + /** + * Creates a plain object from an ApplySchemaRequest message. Also converts values to other types if specified. + * @param message ApplySchemaRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplySchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplySchemaRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplySchemaRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplySchemaResponse. */ + interface IApplySchemaResponse { + + /** ApplySchemaResponse uuid_list */ + uuid_list?: (string[]|null); + } + + /** Represents an ApplySchemaResponse. */ + class ApplySchemaResponse implements IApplySchemaResponse { + + /** + * Constructs a new ApplySchemaResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplySchemaResponse); + + /** ApplySchemaResponse uuid_list. */ + public uuid_list: string[]; + + /** + * Creates a new ApplySchemaResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplySchemaResponse instance + */ + public static create(properties?: vtctldata.IApplySchemaResponse): vtctldata.ApplySchemaResponse; + + /** + * Encodes the specified ApplySchemaResponse message. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * @param message ApplySchemaResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplySchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplySchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * @param message ApplySchemaResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplySchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; - /** Log count. */ - public count: (number|Long); + /** + * Decodes an ApplySchemaResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplySchemaResponse + * @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.ApplySchemaResponse; - /** - * Creates a new Log instance using the specified properties. - * @param [properties] Properties to set - * @returns Log instance - */ - public static create(properties?: vtctldata.Workflow.Stream.ILog): vtctldata.Workflow.Stream.Log; + /** + * Decodes an ApplySchemaResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplySchemaResponse + * @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.ApplySchemaResponse; - /** - * Encodes the specified Log message. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. - * @param message Log message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: vtctldata.Workflow.Stream.ILog, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Verifies an ApplySchemaResponse 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); - /** - * Encodes the specified Log message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. - * @param message Log message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: vtctldata.Workflow.Stream.ILog, writer?: $protobuf.Writer): $protobuf.Writer; + /** + * Creates an ApplySchemaResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplySchemaResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplySchemaResponse; - /** - * Decodes a Log message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns Log - * @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.Workflow.Stream.Log; + /** + * Creates a plain object from an ApplySchemaResponse message. Also converts values to other types if specified. + * @param message ApplySchemaResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplySchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; - /** - * Decodes a Log message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns Log - * @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.Workflow.Stream.Log; + /** + * Converts this ApplySchemaResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; - /** - * Verifies a Log 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); + /** + * Gets the default type url for ApplySchemaResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } - /** - * Creates a Log message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns Log - */ - public static fromObject(object: { [k: string]: any }): vtctldata.Workflow.Stream.Log; + /** Properties of an ApplyVSchemaRequest. */ + interface IApplyVSchemaRequest { - /** - * Creates a plain object from a Log message. Also converts values to other types if specified. - * @param message Log - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: vtctldata.Workflow.Stream.Log, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** ApplyVSchemaRequest keyspace */ + keyspace?: (string|null); - /** - * Converts this Log to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** ApplyVSchemaRequest skip_rebuild */ + skip_rebuild?: (boolean|null); - /** - * Gets the default type url for Log - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } - } - } + /** ApplyVSchemaRequest dry_run */ + dry_run?: (boolean|null); - /** Properties of an AddCellInfoRequest. */ - interface IAddCellInfoRequest { + /** ApplyVSchemaRequest cells */ + cells?: (string[]|null); - /** AddCellInfoRequest name */ - name?: (string|null); + /** ApplyVSchemaRequest v_schema */ + v_schema?: (vschema.IKeyspace|null); - /** AddCellInfoRequest cell_info */ - cell_info?: (topodata.ICellInfo|null); + /** ApplyVSchemaRequest sql */ + sql?: (string|null); } - /** Represents an AddCellInfoRequest. */ - class AddCellInfoRequest implements IAddCellInfoRequest { + /** Represents an ApplyVSchemaRequest. */ + class ApplyVSchemaRequest implements IApplyVSchemaRequest { /** - * Constructs a new AddCellInfoRequest. + * Constructs a new ApplyVSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IAddCellInfoRequest); + constructor(properties?: vtctldata.IApplyVSchemaRequest); - /** AddCellInfoRequest name. */ - public name: string; + /** ApplyVSchemaRequest keyspace. */ + public keyspace: string; - /** AddCellInfoRequest cell_info. */ - public cell_info?: (topodata.ICellInfo|null); + /** ApplyVSchemaRequest skip_rebuild. */ + public skip_rebuild: boolean; + + /** ApplyVSchemaRequest dry_run. */ + public dry_run: boolean; + + /** ApplyVSchemaRequest cells. */ + public cells: string[]; + + /** ApplyVSchemaRequest v_schema. */ + public v_schema?: (vschema.IKeyspace|null); + + /** ApplyVSchemaRequest sql. */ + public sql: string; /** - * Creates a new AddCellInfoRequest instance using the specified properties. + * Creates a new ApplyVSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns AddCellInfoRequest instance + * @returns ApplyVSchemaRequest instance */ - public static create(properties?: vtctldata.IAddCellInfoRequest): vtctldata.AddCellInfoRequest; + public static create(properties?: vtctldata.IApplyVSchemaRequest): vtctldata.ApplyVSchemaRequest; /** - * Encodes the specified AddCellInfoRequest message. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. - * @param message AddCellInfoRequest message or plain object to encode + * Encodes the specified ApplyVSchemaRequest message. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * @param message ApplyVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IAddCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IApplyVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AddCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. - * @param message AddCellInfoRequest message or plain object to encode + * Encodes the specified ApplyVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * @param message ApplyVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IAddCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IApplyVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AddCellInfoRequest message from the specified reader or buffer. + * Decodes an ApplyVSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AddCellInfoRequest + * @returns ApplyVSchemaRequest * @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.AddCellInfoRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ApplyVSchemaRequest; /** - * Decodes an AddCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes an ApplyVSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AddCellInfoRequest + * @returns ApplyVSchemaRequest * @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.AddCellInfoRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ApplyVSchemaRequest; /** - * Verifies an AddCellInfoRequest message. + * Verifies an ApplyVSchemaRequest 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 AddCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ApplyVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AddCellInfoRequest + * @returns ApplyVSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.AddCellInfoRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyVSchemaRequest; /** - * Creates a plain object from an AddCellInfoRequest message. Also converts values to other types if specified. - * @param message AddCellInfoRequest + * Creates a plain object from an ApplyVSchemaRequest message. Also converts values to other types if specified. + * @param message ApplyVSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.AddCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ApplyVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AddCellInfoRequest to JSON. + * Converts this ApplyVSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AddCellInfoRequest + * Gets the default type url for ApplyVSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an AddCellInfoResponse. */ - interface IAddCellInfoResponse { + /** Properties of an ApplyVSchemaResponse. */ + interface IApplyVSchemaResponse { + + /** ApplyVSchemaResponse v_schema */ + v_schema?: (vschema.IKeyspace|null); } - /** Represents an AddCellInfoResponse. */ - class AddCellInfoResponse implements IAddCellInfoResponse { + /** Represents an ApplyVSchemaResponse. */ + class ApplyVSchemaResponse implements IApplyVSchemaResponse { /** - * Constructs a new AddCellInfoResponse. + * Constructs a new ApplyVSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IAddCellInfoResponse); + constructor(properties?: vtctldata.IApplyVSchemaResponse); + + /** ApplyVSchemaResponse v_schema. */ + public v_schema?: (vschema.IKeyspace|null); /** - * Creates a new AddCellInfoResponse instance using the specified properties. + * Creates a new ApplyVSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns AddCellInfoResponse instance + * @returns ApplyVSchemaResponse instance */ - public static create(properties?: vtctldata.IAddCellInfoResponse): vtctldata.AddCellInfoResponse; + public static create(properties?: vtctldata.IApplyVSchemaResponse): vtctldata.ApplyVSchemaResponse; /** - * Encodes the specified AddCellInfoResponse message. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. - * @param message AddCellInfoResponse message or plain object to encode + * Encodes the specified ApplyVSchemaResponse message. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * @param message ApplyVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IAddCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IApplyVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AddCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. - * @param message AddCellInfoResponse message or plain object to encode + * Encodes the specified ApplyVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * @param message ApplyVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IAddCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IApplyVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AddCellInfoResponse message from the specified reader or buffer. + * Decodes an ApplyVSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AddCellInfoResponse + * @returns ApplyVSchemaResponse * @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.AddCellInfoResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ApplyVSchemaResponse; /** - * Decodes an AddCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes an ApplyVSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AddCellInfoResponse + * @returns ApplyVSchemaResponse * @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.AddCellInfoResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ApplyVSchemaResponse; /** - * Verifies an AddCellInfoResponse message. + * Verifies an ApplyVSchemaResponse 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 AddCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ApplyVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AddCellInfoResponse + * @returns ApplyVSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.AddCellInfoResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyVSchemaResponse; /** - * Creates a plain object from an AddCellInfoResponse message. Also converts values to other types if specified. - * @param message AddCellInfoResponse + * Creates a plain object from an ApplyVSchemaResponse message. Also converts values to other types if specified. + * @param message ApplyVSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.AddCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ApplyVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AddCellInfoResponse to JSON. + * Converts this ApplyVSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AddCellInfoResponse + * Gets the default type url for ApplyVSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an AddCellsAliasRequest. */ - interface IAddCellsAliasRequest { + /** Properties of a BackupRequest. */ + interface IBackupRequest { - /** AddCellsAliasRequest name */ - name?: (string|null); + /** BackupRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** AddCellsAliasRequest cells */ - cells?: (string[]|null); + /** BackupRequest allow_primary */ + allow_primary?: (boolean|null); + + /** BackupRequest concurrency */ + concurrency?: (number|Long|null); + + /** BackupRequest incremental_from_pos */ + incremental_from_pos?: (string|null); + + /** BackupRequest upgrade_safe */ + upgrade_safe?: (boolean|null); } - /** Represents an AddCellsAliasRequest. */ - class AddCellsAliasRequest implements IAddCellsAliasRequest { + /** Represents a BackupRequest. */ + class BackupRequest implements IBackupRequest { /** - * Constructs a new AddCellsAliasRequest. + * Constructs a new BackupRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IAddCellsAliasRequest); + constructor(properties?: vtctldata.IBackupRequest); - /** AddCellsAliasRequest name. */ - public name: string; + /** BackupRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** AddCellsAliasRequest cells. */ - public cells: string[]; + /** BackupRequest allow_primary. */ + public allow_primary: boolean; + + /** BackupRequest concurrency. */ + public concurrency: (number|Long); + + /** BackupRequest incremental_from_pos. */ + public incremental_from_pos: string; + + /** BackupRequest upgrade_safe. */ + public upgrade_safe: boolean; /** - * Creates a new AddCellsAliasRequest instance using the specified properties. + * Creates a new BackupRequest instance using the specified properties. * @param [properties] Properties to set - * @returns AddCellsAliasRequest instance + * @returns BackupRequest instance */ - public static create(properties?: vtctldata.IAddCellsAliasRequest): vtctldata.AddCellsAliasRequest; + public static create(properties?: vtctldata.IBackupRequest): vtctldata.BackupRequest; /** - * Encodes the specified AddCellsAliasRequest message. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. - * @param message AddCellsAliasRequest message or plain object to encode + * Encodes the specified BackupRequest message. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. + * @param message BackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IAddCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AddCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. - * @param message AddCellsAliasRequest message or plain object to encode + * Encodes the specified BackupRequest message, length delimited. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. + * @param message BackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IAddCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AddCellsAliasRequest message from the specified reader or buffer. + * Decodes a BackupRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AddCellsAliasRequest + * @returns BackupRequest * @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.AddCellsAliasRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.BackupRequest; /** - * Decodes an AddCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes a BackupRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AddCellsAliasRequest + * @returns BackupRequest * @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.AddCellsAliasRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.BackupRequest; /** - * Verifies an AddCellsAliasRequest message. + * Verifies a BackupRequest 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 AddCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BackupRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AddCellsAliasRequest + * @returns BackupRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.AddCellsAliasRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.BackupRequest; /** - * Creates a plain object from an AddCellsAliasRequest message. Also converts values to other types if specified. - * @param message AddCellsAliasRequest + * Creates a plain object from a BackupRequest message. Also converts values to other types if specified. + * @param message BackupRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.AddCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.BackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AddCellsAliasRequest to JSON. + * Converts this BackupRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AddCellsAliasRequest + * Gets the default type url for BackupRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an AddCellsAliasResponse. */ - interface IAddCellsAliasResponse { + /** Properties of a BackupResponse. */ + interface IBackupResponse { + + /** BackupResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** BackupResponse keyspace */ + keyspace?: (string|null); + + /** BackupResponse shard */ + shard?: (string|null); + + /** BackupResponse event */ + event?: (logutil.IEvent|null); } - /** Represents an AddCellsAliasResponse. */ - class AddCellsAliasResponse implements IAddCellsAliasResponse { + /** Represents a BackupResponse. */ + class BackupResponse implements IBackupResponse { /** - * Constructs a new AddCellsAliasResponse. + * Constructs a new BackupResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IAddCellsAliasResponse); + constructor(properties?: vtctldata.IBackupResponse); + + /** BackupResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** BackupResponse keyspace. */ + public keyspace: string; + + /** BackupResponse shard. */ + public shard: string; + + /** BackupResponse event. */ + public event?: (logutil.IEvent|null); /** - * Creates a new AddCellsAliasResponse instance using the specified properties. + * Creates a new BackupResponse instance using the specified properties. * @param [properties] Properties to set - * @returns AddCellsAliasResponse instance + * @returns BackupResponse instance */ - public static create(properties?: vtctldata.IAddCellsAliasResponse): vtctldata.AddCellsAliasResponse; + public static create(properties?: vtctldata.IBackupResponse): vtctldata.BackupResponse; /** - * Encodes the specified AddCellsAliasResponse message. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. - * @param message AddCellsAliasResponse message or plain object to encode + * Encodes the specified BackupResponse message. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. + * @param message BackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IAddCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified AddCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. - * @param message AddCellsAliasResponse message or plain object to encode + * Encodes the specified BackupResponse message, length delimited. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. + * @param message BackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IAddCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an AddCellsAliasResponse message from the specified reader or buffer. + * Decodes a BackupResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns AddCellsAliasResponse + * @returns BackupResponse * @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.AddCellsAliasResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.BackupResponse; /** - * Decodes an AddCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes a BackupResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns AddCellsAliasResponse + * @returns BackupResponse * @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.AddCellsAliasResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.BackupResponse; /** - * Verifies an AddCellsAliasResponse message. + * Verifies a BackupResponse 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 AddCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BackupResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns AddCellsAliasResponse + * @returns BackupResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.AddCellsAliasResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.BackupResponse; /** - * Creates a plain object from an AddCellsAliasResponse message. Also converts values to other types if specified. - * @param message AddCellsAliasResponse + * Creates a plain object from a BackupResponse message. Also converts values to other types if specified. + * @param message BackupResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.AddCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.BackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this AddCellsAliasResponse to JSON. + * Converts this BackupResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for AddCellsAliasResponse + * Gets the default type url for BackupResponse * @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 { + /** Properties of a BackupShardRequest. */ + interface IBackupShardRequest { - /** ApplyRoutingRulesRequest routing_rules */ - routing_rules?: (vschema.IRoutingRules|null); + /** BackupShardRequest keyspace */ + keyspace?: (string|null); - /** ApplyRoutingRulesRequest skip_rebuild */ - skip_rebuild?: (boolean|null); + /** BackupShardRequest shard */ + shard?: (string|null); - /** ApplyRoutingRulesRequest rebuild_cells */ - rebuild_cells?: (string[]|null); + /** BackupShardRequest allow_primary */ + allow_primary?: (boolean|null); + + /** BackupShardRequest concurrency */ + concurrency?: (number|Long|null); + + /** BackupShardRequest upgrade_safe */ + upgrade_safe?: (boolean|null); } - /** Represents an ApplyRoutingRulesRequest. */ - class ApplyRoutingRulesRequest implements IApplyRoutingRulesRequest { + /** Represents a BackupShardRequest. */ + class BackupShardRequest implements IBackupShardRequest { /** - * Constructs a new ApplyRoutingRulesRequest. + * Constructs a new BackupShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyRoutingRulesRequest); + constructor(properties?: vtctldata.IBackupShardRequest); - /** ApplyRoutingRulesRequest routing_rules. */ - public routing_rules?: (vschema.IRoutingRules|null); + /** BackupShardRequest keyspace. */ + public keyspace: string; - /** ApplyRoutingRulesRequest skip_rebuild. */ - public skip_rebuild: boolean; + /** BackupShardRequest shard. */ + public shard: string; + + /** BackupShardRequest allow_primary. */ + public allow_primary: boolean; + + /** BackupShardRequest concurrency. */ + public concurrency: (number|Long); - /** ApplyRoutingRulesRequest rebuild_cells. */ - public rebuild_cells: string[]; + /** BackupShardRequest upgrade_safe. */ + public upgrade_safe: boolean; /** - * Creates a new ApplyRoutingRulesRequest instance using the specified properties. + * Creates a new BackupShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyRoutingRulesRequest instance + * @returns BackupShardRequest instance */ - public static create(properties?: vtctldata.IApplyRoutingRulesRequest): vtctldata.ApplyRoutingRulesRequest; + public static create(properties?: vtctldata.IBackupShardRequest): vtctldata.BackupShardRequest; /** - * Encodes the specified ApplyRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. - * @param message ApplyRoutingRulesRequest message or plain object to encode + * Encodes the specified BackupShardRequest message. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. + * @param message BackupShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IBackupShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. - * @param message ApplyRoutingRulesRequest message or plain object to encode + * Encodes the specified BackupShardRequest message, length delimited. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. + * @param message BackupShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IBackupShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer. + * Decodes a BackupShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyRoutingRulesRequest + * @returns BackupShardRequest * @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.ApplyRoutingRulesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.BackupShardRequest; /** - * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a BackupShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyRoutingRulesRequest + * @returns BackupShardRequest * @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.ApplyRoutingRulesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.BackupShardRequest; /** - * Verifies an ApplyRoutingRulesRequest message. + * Verifies a BackupShardRequest 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 ApplyRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BackupShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyRoutingRulesRequest + * @returns BackupShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyRoutingRulesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.BackupShardRequest; /** - * Creates a plain object from an ApplyRoutingRulesRequest message. Also converts values to other types if specified. - * @param message ApplyRoutingRulesRequest + * Creates a plain object from a BackupShardRequest message. Also converts values to other types if specified. + * @param message BackupShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.BackupShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyRoutingRulesRequest to JSON. + * Converts this BackupShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyRoutingRulesRequest + * Gets the default type url for BackupShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplyRoutingRulesResponse. */ - interface IApplyRoutingRulesResponse { + /** Properties of a ChangeTabletTypeRequest. */ + interface IChangeTabletTypeRequest { + + /** ChangeTabletTypeRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** ChangeTabletTypeRequest db_type */ + db_type?: (topodata.TabletType|null); + + /** ChangeTabletTypeRequest dry_run */ + dry_run?: (boolean|null); } - /** Represents an ApplyRoutingRulesResponse. */ - class ApplyRoutingRulesResponse implements IApplyRoutingRulesResponse { + /** Represents a ChangeTabletTypeRequest. */ + class ChangeTabletTypeRequest implements IChangeTabletTypeRequest { /** - * Constructs a new ApplyRoutingRulesResponse. + * Constructs a new ChangeTabletTypeRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyRoutingRulesResponse); + constructor(properties?: vtctldata.IChangeTabletTypeRequest); + + /** ChangeTabletTypeRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** ChangeTabletTypeRequest db_type. */ + public db_type: topodata.TabletType; + + /** ChangeTabletTypeRequest dry_run. */ + public dry_run: boolean; /** - * Creates a new ApplyRoutingRulesResponse instance using the specified properties. + * Creates a new ChangeTabletTypeRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyRoutingRulesResponse instance + * @returns ChangeTabletTypeRequest instance */ - public static create(properties?: vtctldata.IApplyRoutingRulesResponse): vtctldata.ApplyRoutingRulesResponse; + public static create(properties?: vtctldata.IChangeTabletTypeRequest): vtctldata.ChangeTabletTypeRequest; /** - * Encodes the specified ApplyRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. - * @param message ApplyRoutingRulesResponse message or plain object to encode + * Encodes the specified ChangeTabletTypeRequest message. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. + * @param message ChangeTabletTypeRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IChangeTabletTypeRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. - * @param message ApplyRoutingRulesResponse message or plain object to encode + * Encodes the specified ChangeTabletTypeRequest message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. + * @param message ChangeTabletTypeRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IChangeTabletTypeRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer. + * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyRoutingRulesResponse + * @returns ChangeTabletTypeRequest * @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.ApplyRoutingRulesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ChangeTabletTypeRequest; /** - * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyRoutingRulesResponse + * @returns ChangeTabletTypeRequest * @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.ApplyRoutingRulesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ChangeTabletTypeRequest; /** - * Verifies an ApplyRoutingRulesResponse message. + * Verifies a ChangeTabletTypeRequest 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 ApplyRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ChangeTabletTypeRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyRoutingRulesResponse + * @returns ChangeTabletTypeRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyRoutingRulesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ChangeTabletTypeRequest; /** - * Creates a plain object from an ApplyRoutingRulesResponse message. Also converts values to other types if specified. - * @param message ApplyRoutingRulesResponse + * Creates a plain object from a ChangeTabletTypeRequest message. Also converts values to other types if specified. + * @param message ChangeTabletTypeRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ChangeTabletTypeRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyRoutingRulesResponse to JSON. + * Converts this ChangeTabletTypeRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyRoutingRulesResponse + * Gets the default type url for ChangeTabletTypeRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplyShardRoutingRulesRequest. */ - interface IApplyShardRoutingRulesRequest { + /** Properties of a ChangeTabletTypeResponse. */ + interface IChangeTabletTypeResponse { - /** ApplyShardRoutingRulesRequest shard_routing_rules */ - shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** ChangeTabletTypeResponse before_tablet */ + before_tablet?: (topodata.ITablet|null); - /** ApplyShardRoutingRulesRequest skip_rebuild */ - skip_rebuild?: (boolean|null); + /** ChangeTabletTypeResponse after_tablet */ + after_tablet?: (topodata.ITablet|null); - /** ApplyShardRoutingRulesRequest rebuild_cells */ - rebuild_cells?: (string[]|null); + /** ChangeTabletTypeResponse was_dry_run */ + was_dry_run?: (boolean|null); } - /** Represents an ApplyShardRoutingRulesRequest. */ - class ApplyShardRoutingRulesRequest implements IApplyShardRoutingRulesRequest { + /** Represents a ChangeTabletTypeResponse. */ + class ChangeTabletTypeResponse implements IChangeTabletTypeResponse { /** - * Constructs a new ApplyShardRoutingRulesRequest. + * Constructs a new ChangeTabletTypeResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyShardRoutingRulesRequest); + constructor(properties?: vtctldata.IChangeTabletTypeResponse); - /** ApplyShardRoutingRulesRequest shard_routing_rules. */ - public shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** ChangeTabletTypeResponse before_tablet. */ + public before_tablet?: (topodata.ITablet|null); - /** ApplyShardRoutingRulesRequest skip_rebuild. */ - public skip_rebuild: boolean; + /** ChangeTabletTypeResponse after_tablet. */ + public after_tablet?: (topodata.ITablet|null); - /** ApplyShardRoutingRulesRequest rebuild_cells. */ - public rebuild_cells: string[]; + /** ChangeTabletTypeResponse was_dry_run. */ + public was_dry_run: boolean; /** - * Creates a new ApplyShardRoutingRulesRequest instance using the specified properties. + * Creates a new ChangeTabletTypeResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyShardRoutingRulesRequest instance + * @returns ChangeTabletTypeResponse instance */ - public static create(properties?: vtctldata.IApplyShardRoutingRulesRequest): vtctldata.ApplyShardRoutingRulesRequest; + public static create(properties?: vtctldata.IChangeTabletTypeResponse): vtctldata.ChangeTabletTypeResponse; /** - * Encodes the specified ApplyShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. - * @param message ApplyShardRoutingRulesRequest message or plain object to encode + * Encodes the specified ChangeTabletTypeResponse message. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. + * @param message ChangeTabletTypeResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IChangeTabletTypeResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. - * @param message ApplyShardRoutingRulesRequest message or plain object to encode + * Encodes the specified ChangeTabletTypeResponse message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. + * @param message ChangeTabletTypeResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IChangeTabletTypeResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer. + * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyShardRoutingRulesRequest + * @returns ChangeTabletTypeResponse * @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.ApplyShardRoutingRulesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ChangeTabletTypeResponse; /** - * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyShardRoutingRulesRequest + * @returns ChangeTabletTypeResponse * @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.ApplyShardRoutingRulesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ChangeTabletTypeResponse; /** - * Verifies an ApplyShardRoutingRulesRequest message. + * Verifies a ChangeTabletTypeResponse 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 ApplyShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ChangeTabletTypeResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyShardRoutingRulesRequest + * @returns ChangeTabletTypeResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyShardRoutingRulesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ChangeTabletTypeResponse; /** - * Creates a plain object from an ApplyShardRoutingRulesRequest message. Also converts values to other types if specified. - * @param message ApplyShardRoutingRulesRequest + * Creates a plain object from a ChangeTabletTypeResponse message. Also converts values to other types if specified. + * @param message ChangeTabletTypeResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyShardRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ChangeTabletTypeResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyShardRoutingRulesRequest to JSON. + * Converts this ChangeTabletTypeResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyShardRoutingRulesRequest + * Gets the default type url for ChangeTabletTypeResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplyShardRoutingRulesResponse. */ - interface IApplyShardRoutingRulesResponse { + /** Properties of a CreateKeyspaceRequest. */ + interface ICreateKeyspaceRequest { + + /** CreateKeyspaceRequest name */ + name?: (string|null); + + /** CreateKeyspaceRequest force */ + force?: (boolean|null); + + /** CreateKeyspaceRequest allow_empty_v_schema */ + allow_empty_v_schema?: (boolean|null); + + /** CreateKeyspaceRequest served_froms */ + served_froms?: (topodata.Keyspace.IServedFrom[]|null); + + /** CreateKeyspaceRequest type */ + type?: (topodata.KeyspaceType|null); + + /** CreateKeyspaceRequest base_keyspace */ + base_keyspace?: (string|null); + + /** CreateKeyspaceRequest snapshot_time */ + snapshot_time?: (vttime.ITime|null); + + /** CreateKeyspaceRequest durability_policy */ + durability_policy?: (string|null); + + /** CreateKeyspaceRequest sidecar_db_name */ + sidecar_db_name?: (string|null); } - /** Represents an ApplyShardRoutingRulesResponse. */ - class ApplyShardRoutingRulesResponse implements IApplyShardRoutingRulesResponse { + /** Represents a CreateKeyspaceRequest. */ + class CreateKeyspaceRequest implements ICreateKeyspaceRequest { /** - * Constructs a new ApplyShardRoutingRulesResponse. + * Constructs a new CreateKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyShardRoutingRulesResponse); + constructor(properties?: vtctldata.ICreateKeyspaceRequest); + + /** CreateKeyspaceRequest name. */ + public name: string; + + /** CreateKeyspaceRequest force. */ + public force: boolean; + + /** CreateKeyspaceRequest allow_empty_v_schema. */ + public allow_empty_v_schema: boolean; + + /** CreateKeyspaceRequest served_froms. */ + public served_froms: topodata.Keyspace.IServedFrom[]; + + /** CreateKeyspaceRequest type. */ + public type: topodata.KeyspaceType; + + /** CreateKeyspaceRequest base_keyspace. */ + public base_keyspace: string; + + /** CreateKeyspaceRequest snapshot_time. */ + public snapshot_time?: (vttime.ITime|null); + + /** CreateKeyspaceRequest durability_policy. */ + public durability_policy: string; + + /** CreateKeyspaceRequest sidecar_db_name. */ + public sidecar_db_name: string; /** - * Creates a new ApplyShardRoutingRulesResponse instance using the specified properties. + * Creates a new CreateKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyShardRoutingRulesResponse instance + * @returns CreateKeyspaceRequest instance */ - public static create(properties?: vtctldata.IApplyShardRoutingRulesResponse): vtctldata.ApplyShardRoutingRulesResponse; + public static create(properties?: vtctldata.ICreateKeyspaceRequest): vtctldata.CreateKeyspaceRequest; /** - * Encodes the specified ApplyShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. - * @param message ApplyShardRoutingRulesResponse message or plain object to encode + * Encodes the specified CreateKeyspaceRequest message. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. + * @param message CreateKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ICreateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. - * @param message ApplyShardRoutingRulesResponse message or plain object to encode + * Encodes the specified CreateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. + * @param message CreateKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ICreateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer. + * Decodes a CreateKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyShardRoutingRulesResponse + * @returns CreateKeyspaceRequest * @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.ApplyShardRoutingRulesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.CreateKeyspaceRequest; /** - * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyShardRoutingRulesResponse + * @returns CreateKeyspaceRequest * @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.ApplyShardRoutingRulesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.CreateKeyspaceRequest; /** - * Verifies an ApplyShardRoutingRulesResponse message. + * Verifies a CreateKeyspaceRequest 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 ApplyShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyShardRoutingRulesResponse + * @returns CreateKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyShardRoutingRulesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.CreateKeyspaceRequest; /** - * Creates a plain object from an ApplyShardRoutingRulesResponse message. Also converts values to other types if specified. - * @param message ApplyShardRoutingRulesResponse + * Creates a plain object from a CreateKeyspaceRequest message. Also converts values to other types if specified. + * @param message CreateKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyShardRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.CreateKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyShardRoutingRulesResponse to JSON. + * Converts this CreateKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyShardRoutingRulesResponse + * Gets the default type url for CreateKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplySchemaRequest. */ - interface IApplySchemaRequest { - - /** ApplySchemaRequest keyspace */ - keyspace?: (string|null); - - /** ApplySchemaRequest sql */ - sql?: (string[]|null); - - /** ApplySchemaRequest ddl_strategy */ - ddl_strategy?: (string|null); - - /** ApplySchemaRequest uuid_list */ - uuid_list?: (string[]|null); - - /** ApplySchemaRequest migration_context */ - migration_context?: (string|null); - - /** ApplySchemaRequest wait_replicas_timeout */ - wait_replicas_timeout?: (vttime.IDuration|null); - - /** ApplySchemaRequest skip_preflight */ - skip_preflight?: (boolean|null); + /** Properties of a CreateKeyspaceResponse. */ + interface ICreateKeyspaceResponse { - /** ApplySchemaRequest caller_id */ - caller_id?: (vtrpc.ICallerID|null); + /** CreateKeyspaceResponse keyspace */ + keyspace?: (vtctldata.IKeyspace|null); } - /** Represents an ApplySchemaRequest. */ - class ApplySchemaRequest implements IApplySchemaRequest { + /** Represents a CreateKeyspaceResponse. */ + class CreateKeyspaceResponse implements ICreateKeyspaceResponse { /** - * Constructs a new ApplySchemaRequest. + * Constructs a new CreateKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplySchemaRequest); - - /** ApplySchemaRequest keyspace. */ - public keyspace: string; - - /** ApplySchemaRequest sql. */ - public sql: string[]; - - /** ApplySchemaRequest ddl_strategy. */ - public ddl_strategy: string; - - /** ApplySchemaRequest uuid_list. */ - public uuid_list: string[]; - - /** ApplySchemaRequest migration_context. */ - public migration_context: string; - - /** ApplySchemaRequest wait_replicas_timeout. */ - public wait_replicas_timeout?: (vttime.IDuration|null); - - /** ApplySchemaRequest skip_preflight. */ - public skip_preflight: boolean; + constructor(properties?: vtctldata.ICreateKeyspaceResponse); - /** ApplySchemaRequest caller_id. */ - public caller_id?: (vtrpc.ICallerID|null); + /** CreateKeyspaceResponse keyspace. */ + public keyspace?: (vtctldata.IKeyspace|null); /** - * Creates a new ApplySchemaRequest instance using the specified properties. + * Creates a new CreateKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ApplySchemaRequest instance + * @returns CreateKeyspaceResponse instance */ - public static create(properties?: vtctldata.IApplySchemaRequest): vtctldata.ApplySchemaRequest; + public static create(properties?: vtctldata.ICreateKeyspaceResponse): vtctldata.CreateKeyspaceResponse; /** - * Encodes the specified ApplySchemaRequest message. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. - * @param message ApplySchemaRequest message or plain object to encode + * Encodes the specified CreateKeyspaceResponse message. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. + * @param message CreateKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplySchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ICreateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplySchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. - * @param message ApplySchemaRequest message or plain object to encode + * Encodes the specified CreateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. + * @param message CreateKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplySchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ICreateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplySchemaRequest message from the specified reader or buffer. + * Decodes a CreateKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplySchemaRequest + * @returns CreateKeyspaceResponse * @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.ApplySchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.CreateKeyspaceResponse; /** - * Decodes an ApplySchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplySchemaRequest + * @returns CreateKeyspaceResponse * @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.ApplySchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.CreateKeyspaceResponse; /** - * Verifies an ApplySchemaRequest message. + * Verifies a CreateKeyspaceResponse 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 ApplySchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplySchemaRequest + * @returns CreateKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplySchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.CreateKeyspaceResponse; /** - * Creates a plain object from an ApplySchemaRequest message. Also converts values to other types if specified. - * @param message ApplySchemaRequest + * Creates a plain object from a CreateKeyspaceResponse message. Also converts values to other types if specified. + * @param message CreateKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplySchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.CreateKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplySchemaRequest to JSON. + * Converts this CreateKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplySchemaRequest + * Gets the default type url for CreateKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplySchemaResponse. */ - interface IApplySchemaResponse { + /** Properties of a CreateShardRequest. */ + interface ICreateShardRequest { - /** ApplySchemaResponse uuid_list */ - uuid_list?: (string[]|null); + /** CreateShardRequest keyspace */ + keyspace?: (string|null); + + /** CreateShardRequest shard_name */ + shard_name?: (string|null); + + /** CreateShardRequest force */ + force?: (boolean|null); + + /** CreateShardRequest include_parent */ + include_parent?: (boolean|null); } - /** Represents an ApplySchemaResponse. */ - class ApplySchemaResponse implements IApplySchemaResponse { + /** Represents a CreateShardRequest. */ + class CreateShardRequest implements ICreateShardRequest { /** - * Constructs a new ApplySchemaResponse. + * Constructs a new CreateShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplySchemaResponse); + constructor(properties?: vtctldata.ICreateShardRequest); - /** ApplySchemaResponse uuid_list. */ - public uuid_list: string[]; + /** CreateShardRequest keyspace. */ + public keyspace: string; + + /** CreateShardRequest shard_name. */ + public shard_name: string; + + /** CreateShardRequest force. */ + public force: boolean; + + /** CreateShardRequest include_parent. */ + public include_parent: boolean; /** - * Creates a new ApplySchemaResponse instance using the specified properties. + * Creates a new CreateShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ApplySchemaResponse instance + * @returns CreateShardRequest instance */ - public static create(properties?: vtctldata.IApplySchemaResponse): vtctldata.ApplySchemaResponse; + public static create(properties?: vtctldata.ICreateShardRequest): vtctldata.CreateShardRequest; /** - * Encodes the specified ApplySchemaResponse message. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. - * @param message ApplySchemaResponse message or plain object to encode + * Encodes the specified CreateShardRequest message. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. + * @param message CreateShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplySchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ICreateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplySchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. - * @param message ApplySchemaResponse message or plain object to encode + * Encodes the specified CreateShardRequest message, length delimited. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. + * @param message CreateShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplySchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ICreateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplySchemaResponse message from the specified reader or buffer. + * Decodes a CreateShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplySchemaResponse + * @returns CreateShardRequest * @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.ApplySchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.CreateShardRequest; /** - * Decodes an ApplySchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplySchemaResponse + * @returns CreateShardRequest * @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.ApplySchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.CreateShardRequest; /** - * Verifies an ApplySchemaResponse message. + * Verifies a CreateShardRequest 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 ApplySchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplySchemaResponse + * @returns CreateShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplySchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.CreateShardRequest; /** - * Creates a plain object from an ApplySchemaResponse message. Also converts values to other types if specified. - * @param message ApplySchemaResponse + * Creates a plain object from a CreateShardRequest message. Also converts values to other types if specified. + * @param message CreateShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplySchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.CreateShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplySchemaResponse to JSON. + * Converts this CreateShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplySchemaResponse + * Gets the default type url for CreateShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplyVSchemaRequest. */ - interface IApplyVSchemaRequest { - - /** ApplyVSchemaRequest keyspace */ - keyspace?: (string|null); - - /** ApplyVSchemaRequest skip_rebuild */ - skip_rebuild?: (boolean|null); - - /** ApplyVSchemaRequest dry_run */ - dry_run?: (boolean|null); + /** Properties of a CreateShardResponse. */ + interface ICreateShardResponse { - /** ApplyVSchemaRequest cells */ - cells?: (string[]|null); + /** CreateShardResponse keyspace */ + keyspace?: (vtctldata.IKeyspace|null); - /** ApplyVSchemaRequest v_schema */ - v_schema?: (vschema.IKeyspace|null); + /** CreateShardResponse shard */ + shard?: (vtctldata.IShard|null); - /** ApplyVSchemaRequest sql */ - sql?: (string|null); + /** CreateShardResponse shard_already_exists */ + shard_already_exists?: (boolean|null); } - /** Represents an ApplyVSchemaRequest. */ - class ApplyVSchemaRequest implements IApplyVSchemaRequest { + /** Represents a CreateShardResponse. */ + class CreateShardResponse implements ICreateShardResponse { /** - * Constructs a new ApplyVSchemaRequest. + * Constructs a new CreateShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyVSchemaRequest); - - /** ApplyVSchemaRequest keyspace. */ - public keyspace: string; - - /** ApplyVSchemaRequest skip_rebuild. */ - public skip_rebuild: boolean; - - /** ApplyVSchemaRequest dry_run. */ - public dry_run: boolean; + constructor(properties?: vtctldata.ICreateShardResponse); - /** ApplyVSchemaRequest cells. */ - public cells: string[]; + /** CreateShardResponse keyspace. */ + public keyspace?: (vtctldata.IKeyspace|null); - /** ApplyVSchemaRequest v_schema. */ - public v_schema?: (vschema.IKeyspace|null); + /** CreateShardResponse shard. */ + public shard?: (vtctldata.IShard|null); - /** ApplyVSchemaRequest sql. */ - public sql: string; + /** CreateShardResponse shard_already_exists. */ + public shard_already_exists: boolean; /** - * Creates a new ApplyVSchemaRequest instance using the specified properties. + * Creates a new CreateShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyVSchemaRequest instance + * @returns CreateShardResponse instance */ - public static create(properties?: vtctldata.IApplyVSchemaRequest): vtctldata.ApplyVSchemaRequest; + public static create(properties?: vtctldata.ICreateShardResponse): vtctldata.CreateShardResponse; /** - * Encodes the specified ApplyVSchemaRequest message. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. - * @param message ApplyVSchemaRequest message or plain object to encode + * Encodes the specified CreateShardResponse message. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. + * @param message CreateShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ICreateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. - * @param message ApplyVSchemaRequest message or plain object to encode + * Encodes the specified CreateShardResponse message, length delimited. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. + * @param message CreateShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ICreateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyVSchemaRequest message from the specified reader or buffer. + * Decodes a CreateShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyVSchemaRequest + * @returns CreateShardResponse * @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.ApplyVSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.CreateShardResponse; /** - * Decodes an ApplyVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyVSchemaRequest + * @returns CreateShardResponse * @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.ApplyVSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.CreateShardResponse; /** - * Verifies an ApplyVSchemaRequest message. + * Verifies a CreateShardResponse 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 ApplyVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyVSchemaRequest + * @returns CreateShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyVSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.CreateShardResponse; /** - * Creates a plain object from an ApplyVSchemaRequest message. Also converts values to other types if specified. - * @param message ApplyVSchemaRequest + * Creates a plain object from a CreateShardResponse message. Also converts values to other types if specified. + * @param message CreateShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.CreateShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyVSchemaRequest to JSON. + * Converts this CreateShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyVSchemaRequest + * Gets the default type url for CreateShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ApplyVSchemaResponse. */ - interface IApplyVSchemaResponse { + /** Properties of a DeleteCellInfoRequest. */ + interface IDeleteCellInfoRequest { - /** ApplyVSchemaResponse v_schema */ - v_schema?: (vschema.IKeyspace|null); + /** DeleteCellInfoRequest name */ + name?: (string|null); + + /** DeleteCellInfoRequest force */ + force?: (boolean|null); } - /** Represents an ApplyVSchemaResponse. */ - class ApplyVSchemaResponse implements IApplyVSchemaResponse { + /** Represents a DeleteCellInfoRequest. */ + class DeleteCellInfoRequest implements IDeleteCellInfoRequest { /** - * Constructs a new ApplyVSchemaResponse. + * Constructs a new DeleteCellInfoRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IApplyVSchemaResponse); + constructor(properties?: vtctldata.IDeleteCellInfoRequest); - /** ApplyVSchemaResponse v_schema. */ - public v_schema?: (vschema.IKeyspace|null); + /** DeleteCellInfoRequest name. */ + public name: string; + + /** DeleteCellInfoRequest force. */ + public force: boolean; /** - * Creates a new ApplyVSchemaResponse instance using the specified properties. + * Creates a new DeleteCellInfoRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ApplyVSchemaResponse instance + * @returns DeleteCellInfoRequest instance */ - public static create(properties?: vtctldata.IApplyVSchemaResponse): vtctldata.ApplyVSchemaResponse; + public static create(properties?: vtctldata.IDeleteCellInfoRequest): vtctldata.DeleteCellInfoRequest; /** - * Encodes the specified ApplyVSchemaResponse message. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. - * @param message ApplyVSchemaResponse message or plain object to encode + * Encodes the specified DeleteCellInfoRequest message. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. + * @param message DeleteCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IApplyVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ApplyVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. - * @param message ApplyVSchemaResponse message or plain object to encode + * Encodes the specified DeleteCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. + * @param message DeleteCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IApplyVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ApplyVSchemaResponse message from the specified reader or buffer. + * Decodes a DeleteCellInfoRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ApplyVSchemaResponse + * @returns DeleteCellInfoRequest * @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.ApplyVSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteCellInfoRequest; /** - * Decodes an ApplyVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellInfoRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ApplyVSchemaResponse + * @returns DeleteCellInfoRequest * @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.ApplyVSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteCellInfoRequest; /** - * Verifies an ApplyVSchemaResponse message. + * Verifies a DeleteCellInfoRequest 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 ApplyVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ApplyVSchemaResponse + * @returns DeleteCellInfoRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ApplyVSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellInfoRequest; /** - * Creates a plain object from an ApplyVSchemaResponse message. Also converts values to other types if specified. - * @param message ApplyVSchemaResponse + * Creates a plain object from a DeleteCellInfoRequest message. Also converts values to other types if specified. + * @param message DeleteCellInfoRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ApplyVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ApplyVSchemaResponse to JSON. + * Converts this DeleteCellInfoRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ApplyVSchemaResponse + * Gets the default type url for DeleteCellInfoRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BackupRequest. */ - interface IBackupRequest { - - /** BackupRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** BackupRequest allow_primary */ - allow_primary?: (boolean|null); - - /** BackupRequest concurrency */ - concurrency?: (number|Long|null); - - /** BackupRequest incremental_from_pos */ - incremental_from_pos?: (string|null); - - /** BackupRequest upgrade_safe */ - upgrade_safe?: (boolean|null); + /** Properties of a DeleteCellInfoResponse. */ + interface IDeleteCellInfoResponse { } - /** Represents a BackupRequest. */ - class BackupRequest implements IBackupRequest { + /** Represents a DeleteCellInfoResponse. */ + class DeleteCellInfoResponse implements IDeleteCellInfoResponse { /** - * Constructs a new BackupRequest. + * Constructs a new DeleteCellInfoResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IBackupRequest); - - /** BackupRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** BackupRequest allow_primary. */ - public allow_primary: boolean; - - /** BackupRequest concurrency. */ - public concurrency: (number|Long); - - /** BackupRequest incremental_from_pos. */ - public incremental_from_pos: string; - - /** BackupRequest upgrade_safe. */ - public upgrade_safe: boolean; + constructor(properties?: vtctldata.IDeleteCellInfoResponse); /** - * Creates a new BackupRequest instance using the specified properties. + * Creates a new DeleteCellInfoResponse instance using the specified properties. * @param [properties] Properties to set - * @returns BackupRequest instance + * @returns DeleteCellInfoResponse instance */ - public static create(properties?: vtctldata.IBackupRequest): vtctldata.BackupRequest; + public static create(properties?: vtctldata.IDeleteCellInfoResponse): vtctldata.DeleteCellInfoResponse; /** - * Encodes the specified BackupRequest message. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. - * @param message BackupRequest message or plain object to encode + * Encodes the specified DeleteCellInfoResponse message. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. + * @param message DeleteCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BackupRequest message, length delimited. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. - * @param message BackupRequest message or plain object to encode + * Encodes the specified DeleteCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. + * @param message DeleteCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BackupRequest message from the specified reader or buffer. + * Decodes a DeleteCellInfoResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BackupRequest + * @returns DeleteCellInfoResponse * @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.BackupRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteCellInfoResponse; /** - * Decodes a BackupRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellInfoResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BackupRequest + * @returns DeleteCellInfoResponse * @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.BackupRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteCellInfoResponse; /** - * Verifies a BackupRequest message. + * Verifies a DeleteCellInfoResponse 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 BackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BackupRequest + * @returns DeleteCellInfoResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.BackupRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellInfoResponse; /** - * Creates a plain object from a BackupRequest message. Also converts values to other types if specified. - * @param message BackupRequest + * Creates a plain object from a DeleteCellInfoResponse message. Also converts values to other types if specified. + * @param message DeleteCellInfoResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.BackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BackupRequest to JSON. + * Converts this DeleteCellInfoResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BackupRequest + * Gets the default type url for DeleteCellInfoResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BackupResponse. */ - interface IBackupResponse { - - /** BackupResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** BackupResponse keyspace */ - keyspace?: (string|null); - - /** BackupResponse shard */ - shard?: (string|null); + /** Properties of a DeleteCellsAliasRequest. */ + interface IDeleteCellsAliasRequest { - /** BackupResponse event */ - event?: (logutil.IEvent|null); + /** DeleteCellsAliasRequest name */ + name?: (string|null); } - /** Represents a BackupResponse. */ - class BackupResponse implements IBackupResponse { + /** Represents a DeleteCellsAliasRequest. */ + class DeleteCellsAliasRequest implements IDeleteCellsAliasRequest { /** - * Constructs a new BackupResponse. + * Constructs a new DeleteCellsAliasRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IBackupResponse); - - /** BackupResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** BackupResponse keyspace. */ - public keyspace: string; - - /** BackupResponse shard. */ - public shard: string; + constructor(properties?: vtctldata.IDeleteCellsAliasRequest); - /** BackupResponse event. */ - public event?: (logutil.IEvent|null); + /** DeleteCellsAliasRequest name. */ + public name: string; /** - * Creates a new BackupResponse instance using the specified properties. + * Creates a new DeleteCellsAliasRequest instance using the specified properties. * @param [properties] Properties to set - * @returns BackupResponse instance + * @returns DeleteCellsAliasRequest instance */ - public static create(properties?: vtctldata.IBackupResponse): vtctldata.BackupResponse; + public static create(properties?: vtctldata.IDeleteCellsAliasRequest): vtctldata.DeleteCellsAliasRequest; /** - * Encodes the specified BackupResponse message. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. - * @param message BackupResponse message or plain object to encode + * Encodes the specified DeleteCellsAliasRequest message. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. + * @param message DeleteCellsAliasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BackupResponse message, length delimited. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. - * @param message BackupResponse message or plain object to encode + * Encodes the specified DeleteCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. + * @param message DeleteCellsAliasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BackupResponse message from the specified reader or buffer. + * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BackupResponse + * @returns DeleteCellsAliasRequest * @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.BackupResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteCellsAliasRequest; /** - * Decodes a BackupResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BackupResponse + * @returns DeleteCellsAliasRequest * @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.BackupResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteCellsAliasRequest; /** - * Verifies a BackupResponse message. + * Verifies a DeleteCellsAliasRequest 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 BackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellsAliasRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BackupResponse + * @returns DeleteCellsAliasRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.BackupResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellsAliasRequest; /** - * Creates a plain object from a BackupResponse message. Also converts values to other types if specified. - * @param message BackupResponse + * Creates a plain object from a DeleteCellsAliasRequest message. Also converts values to other types if specified. + * @param message DeleteCellsAliasRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.BackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BackupResponse to JSON. + * Converts this DeleteCellsAliasRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BackupResponse + * Gets the default type url for DeleteCellsAliasRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a BackupShardRequest. */ - interface IBackupShardRequest { - - /** BackupShardRequest keyspace */ - keyspace?: (string|null); - - /** BackupShardRequest shard */ - shard?: (string|null); - - /** BackupShardRequest allow_primary */ - allow_primary?: (boolean|null); - - /** BackupShardRequest concurrency */ - concurrency?: (number|Long|null); - - /** BackupShardRequest upgrade_safe */ - upgrade_safe?: (boolean|null); + /** Properties of a DeleteCellsAliasResponse. */ + interface IDeleteCellsAliasResponse { } - /** Represents a BackupShardRequest. */ - class BackupShardRequest implements IBackupShardRequest { + /** Represents a DeleteCellsAliasResponse. */ + class DeleteCellsAliasResponse implements IDeleteCellsAliasResponse { /** - * Constructs a new BackupShardRequest. + * Constructs a new DeleteCellsAliasResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IBackupShardRequest); - - /** BackupShardRequest keyspace. */ - public keyspace: string; - - /** BackupShardRequest shard. */ - public shard: string; - - /** BackupShardRequest allow_primary. */ - public allow_primary: boolean; - - /** BackupShardRequest concurrency. */ - public concurrency: (number|Long); - - /** BackupShardRequest upgrade_safe. */ - public upgrade_safe: boolean; + constructor(properties?: vtctldata.IDeleteCellsAliasResponse); /** - * Creates a new BackupShardRequest instance using the specified properties. + * Creates a new DeleteCellsAliasResponse instance using the specified properties. * @param [properties] Properties to set - * @returns BackupShardRequest instance + * @returns DeleteCellsAliasResponse instance */ - public static create(properties?: vtctldata.IBackupShardRequest): vtctldata.BackupShardRequest; + public static create(properties?: vtctldata.IDeleteCellsAliasResponse): vtctldata.DeleteCellsAliasResponse; /** - * Encodes the specified BackupShardRequest message. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. - * @param message BackupShardRequest message or plain object to encode + * Encodes the specified DeleteCellsAliasResponse message. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. + * @param message DeleteCellsAliasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IBackupShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified BackupShardRequest message, length delimited. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. - * @param message BackupShardRequest message or plain object to encode + * Encodes the specified DeleteCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. + * @param message DeleteCellsAliasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IBackupShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a BackupShardRequest message from the specified reader or buffer. + * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns BackupShardRequest + * @returns DeleteCellsAliasResponse * @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.BackupShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteCellsAliasResponse; /** - * Decodes a BackupShardRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns BackupShardRequest + * @returns DeleteCellsAliasResponse * @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.BackupShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteCellsAliasResponse; /** - * Verifies a BackupShardRequest message. + * Verifies a DeleteCellsAliasResponse 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 BackupShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellsAliasResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns BackupShardRequest + * @returns DeleteCellsAliasResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.BackupShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellsAliasResponse; /** - * Creates a plain object from a BackupShardRequest message. Also converts values to other types if specified. - * @param message BackupShardRequest + * Creates a plain object from a DeleteCellsAliasResponse message. Also converts values to other types if specified. + * @param message DeleteCellsAliasResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.BackupShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this BackupShardRequest to JSON. + * Converts this DeleteCellsAliasResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for BackupShardRequest + * Gets the default type url for DeleteCellsAliasResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ChangeTabletTypeRequest. */ - interface IChangeTabletTypeRequest { + /** Properties of a DeleteKeyspaceRequest. */ + interface IDeleteKeyspaceRequest { - /** ChangeTabletTypeRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** DeleteKeyspaceRequest keyspace */ + keyspace?: (string|null); - /** ChangeTabletTypeRequest db_type */ - db_type?: (topodata.TabletType|null); + /** DeleteKeyspaceRequest recursive */ + recursive?: (boolean|null); - /** ChangeTabletTypeRequest dry_run */ - dry_run?: (boolean|null); + /** DeleteKeyspaceRequest force */ + force?: (boolean|null); } - /** Represents a ChangeTabletTypeRequest. */ - class ChangeTabletTypeRequest implements IChangeTabletTypeRequest { + /** Represents a DeleteKeyspaceRequest. */ + class DeleteKeyspaceRequest implements IDeleteKeyspaceRequest { /** - * Constructs a new ChangeTabletTypeRequest. + * Constructs a new DeleteKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IChangeTabletTypeRequest); + constructor(properties?: vtctldata.IDeleteKeyspaceRequest); - /** ChangeTabletTypeRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** DeleteKeyspaceRequest keyspace. */ + public keyspace: string; - /** ChangeTabletTypeRequest db_type. */ - public db_type: topodata.TabletType; + /** DeleteKeyspaceRequest recursive. */ + public recursive: boolean; - /** ChangeTabletTypeRequest dry_run. */ - public dry_run: boolean; + /** DeleteKeyspaceRequest force. */ + public force: boolean; /** - * Creates a new ChangeTabletTypeRequest instance using the specified properties. + * Creates a new DeleteKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ChangeTabletTypeRequest instance + * @returns DeleteKeyspaceRequest instance */ - public static create(properties?: vtctldata.IChangeTabletTypeRequest): vtctldata.ChangeTabletTypeRequest; + public static create(properties?: vtctldata.IDeleteKeyspaceRequest): vtctldata.DeleteKeyspaceRequest; /** - * Encodes the specified ChangeTabletTypeRequest message. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. - * @param message ChangeTabletTypeRequest message or plain object to encode + * Encodes the specified DeleteKeyspaceRequest message. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. + * @param message DeleteKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IChangeTabletTypeRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ChangeTabletTypeRequest message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. - * @param message ChangeTabletTypeRequest message or plain object to encode + * Encodes the specified DeleteKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. + * @param message DeleteKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IChangeTabletTypeRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer. + * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ChangeTabletTypeRequest + * @returns DeleteKeyspaceRequest * @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.ChangeTabletTypeRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteKeyspaceRequest; /** - * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ChangeTabletTypeRequest + * @returns DeleteKeyspaceRequest * @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.ChangeTabletTypeRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteKeyspaceRequest; /** - * Verifies a ChangeTabletTypeRequest message. + * Verifies a DeleteKeyspaceRequest 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 ChangeTabletTypeRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ChangeTabletTypeRequest + * @returns DeleteKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ChangeTabletTypeRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteKeyspaceRequest; /** - * Creates a plain object from a ChangeTabletTypeRequest message. Also converts values to other types if specified. - * @param message ChangeTabletTypeRequest + * Creates a plain object from a DeleteKeyspaceRequest message. Also converts values to other types if specified. + * @param message DeleteKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ChangeTabletTypeRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ChangeTabletTypeRequest to JSON. + * Converts this DeleteKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ChangeTabletTypeRequest + * Gets the default type url for DeleteKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ChangeTabletTypeResponse. */ - interface IChangeTabletTypeResponse { - - /** ChangeTabletTypeResponse before_tablet */ - before_tablet?: (topodata.ITablet|null); - - /** ChangeTabletTypeResponse after_tablet */ - after_tablet?: (topodata.ITablet|null); - - /** ChangeTabletTypeResponse was_dry_run */ - was_dry_run?: (boolean|null); + /** Properties of a DeleteKeyspaceResponse. */ + interface IDeleteKeyspaceResponse { } - /** Represents a ChangeTabletTypeResponse. */ - class ChangeTabletTypeResponse implements IChangeTabletTypeResponse { + /** Represents a DeleteKeyspaceResponse. */ + class DeleteKeyspaceResponse implements IDeleteKeyspaceResponse { /** - * Constructs a new ChangeTabletTypeResponse. + * Constructs a new DeleteKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IChangeTabletTypeResponse); - - /** ChangeTabletTypeResponse before_tablet. */ - public before_tablet?: (topodata.ITablet|null); - - /** ChangeTabletTypeResponse after_tablet. */ - public after_tablet?: (topodata.ITablet|null); - - /** ChangeTabletTypeResponse was_dry_run. */ - public was_dry_run: boolean; + constructor(properties?: vtctldata.IDeleteKeyspaceResponse); /** - * Creates a new ChangeTabletTypeResponse instance using the specified properties. + * Creates a new DeleteKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ChangeTabletTypeResponse instance + * @returns DeleteKeyspaceResponse instance */ - public static create(properties?: vtctldata.IChangeTabletTypeResponse): vtctldata.ChangeTabletTypeResponse; + public static create(properties?: vtctldata.IDeleteKeyspaceResponse): vtctldata.DeleteKeyspaceResponse; /** - * Encodes the specified ChangeTabletTypeResponse message. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. - * @param message ChangeTabletTypeResponse message or plain object to encode + * Encodes the specified DeleteKeyspaceResponse message. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. + * @param message DeleteKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IChangeTabletTypeResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ChangeTabletTypeResponse message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. - * @param message ChangeTabletTypeResponse message or plain object to encode + * Encodes the specified DeleteKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. + * @param message DeleteKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IChangeTabletTypeResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer. + * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ChangeTabletTypeResponse + * @returns DeleteKeyspaceResponse * @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.ChangeTabletTypeResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteKeyspaceResponse; /** - * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ChangeTabletTypeResponse + * @returns DeleteKeyspaceResponse * @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.ChangeTabletTypeResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteKeyspaceResponse; /** - * Verifies a ChangeTabletTypeResponse message. + * Verifies a DeleteKeyspaceResponse 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 ChangeTabletTypeResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ChangeTabletTypeResponse + * @returns DeleteKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ChangeTabletTypeResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteKeyspaceResponse; /** - * Creates a plain object from a ChangeTabletTypeResponse message. Also converts values to other types if specified. - * @param message ChangeTabletTypeResponse + * Creates a plain object from a DeleteKeyspaceResponse message. Also converts values to other types if specified. + * @param message DeleteKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ChangeTabletTypeResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ChangeTabletTypeResponse to JSON. + * Converts this DeleteKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ChangeTabletTypeResponse + * Gets the default type url for DeleteKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateKeyspaceRequest. */ - interface ICreateKeyspaceRequest { - - /** CreateKeyspaceRequest name */ - name?: (string|null); - - /** CreateKeyspaceRequest force */ - force?: (boolean|null); - - /** CreateKeyspaceRequest allow_empty_v_schema */ - allow_empty_v_schema?: (boolean|null); - - /** CreateKeyspaceRequest served_froms */ - served_froms?: (topodata.Keyspace.IServedFrom[]|null); - - /** CreateKeyspaceRequest type */ - type?: (topodata.KeyspaceType|null); + /** Properties of a DeleteShardsRequest. */ + interface IDeleteShardsRequest { - /** CreateKeyspaceRequest base_keyspace */ - base_keyspace?: (string|null); + /** DeleteShardsRequest shards */ + shards?: (vtctldata.IShard[]|null); - /** CreateKeyspaceRequest snapshot_time */ - snapshot_time?: (vttime.ITime|null); + /** DeleteShardsRequest recursive */ + recursive?: (boolean|null); - /** CreateKeyspaceRequest durability_policy */ - durability_policy?: (string|null); + /** DeleteShardsRequest even_if_serving */ + even_if_serving?: (boolean|null); - /** CreateKeyspaceRequest sidecar_db_name */ - sidecar_db_name?: (string|null); + /** DeleteShardsRequest force */ + force?: (boolean|null); } - /** Represents a CreateKeyspaceRequest. */ - class CreateKeyspaceRequest implements ICreateKeyspaceRequest { + /** Represents a DeleteShardsRequest. */ + class DeleteShardsRequest implements IDeleteShardsRequest { /** - * Constructs a new CreateKeyspaceRequest. + * Constructs a new DeleteShardsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ICreateKeyspaceRequest); - - /** CreateKeyspaceRequest name. */ - public name: string; - - /** CreateKeyspaceRequest force. */ - public force: boolean; - - /** CreateKeyspaceRequest allow_empty_v_schema. */ - public allow_empty_v_schema: boolean; - - /** CreateKeyspaceRequest served_froms. */ - public served_froms: topodata.Keyspace.IServedFrom[]; - - /** CreateKeyspaceRequest type. */ - public type: topodata.KeyspaceType; + constructor(properties?: vtctldata.IDeleteShardsRequest); - /** CreateKeyspaceRequest base_keyspace. */ - public base_keyspace: string; + /** DeleteShardsRequest shards. */ + public shards: vtctldata.IShard[]; - /** CreateKeyspaceRequest snapshot_time. */ - public snapshot_time?: (vttime.ITime|null); + /** DeleteShardsRequest recursive. */ + public recursive: boolean; - /** CreateKeyspaceRequest durability_policy. */ - public durability_policy: string; + /** DeleteShardsRequest even_if_serving. */ + public even_if_serving: boolean; - /** CreateKeyspaceRequest sidecar_db_name. */ - public sidecar_db_name: string; + /** DeleteShardsRequest force. */ + public force: boolean; /** - * Creates a new CreateKeyspaceRequest instance using the specified properties. + * Creates a new DeleteShardsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns CreateKeyspaceRequest instance + * @returns DeleteShardsRequest instance */ - public static create(properties?: vtctldata.ICreateKeyspaceRequest): vtctldata.CreateKeyspaceRequest; + public static create(properties?: vtctldata.IDeleteShardsRequest): vtctldata.DeleteShardsRequest; /** - * Encodes the specified CreateKeyspaceRequest message. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. - * @param message CreateKeyspaceRequest message or plain object to encode + * Encodes the specified DeleteShardsRequest message. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. + * @param message DeleteShardsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ICreateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteShardsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. - * @param message CreateKeyspaceRequest message or plain object to encode + * Encodes the specified DeleteShardsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. + * @param message DeleteShardsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ICreateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteShardsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateKeyspaceRequest message from the specified reader or buffer. + * Decodes a DeleteShardsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateKeyspaceRequest + * @returns DeleteShardsRequest * @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.CreateKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteShardsRequest; /** - * Decodes a CreateKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteShardsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateKeyspaceRequest + * @returns DeleteShardsRequest * @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.CreateKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteShardsRequest; /** - * Verifies a CreateKeyspaceRequest message. + * Verifies a DeleteShardsRequest 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 CreateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteShardsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateKeyspaceRequest + * @returns DeleteShardsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.CreateKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteShardsRequest; /** - * Creates a plain object from a CreateKeyspaceRequest message. Also converts values to other types if specified. - * @param message CreateKeyspaceRequest + * Creates a plain object from a DeleteShardsRequest message. Also converts values to other types if specified. + * @param message DeleteShardsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.CreateKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteShardsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateKeyspaceRequest to JSON. + * Converts this DeleteShardsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateKeyspaceRequest + * Gets the default type url for DeleteShardsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateKeyspaceResponse. */ - interface ICreateKeyspaceResponse { - - /** CreateKeyspaceResponse keyspace */ - keyspace?: (vtctldata.IKeyspace|null); + /** Properties of a DeleteShardsResponse. */ + interface IDeleteShardsResponse { } - /** Represents a CreateKeyspaceResponse. */ - class CreateKeyspaceResponse implements ICreateKeyspaceResponse { + /** Represents a DeleteShardsResponse. */ + class DeleteShardsResponse implements IDeleteShardsResponse { /** - * Constructs a new CreateKeyspaceResponse. + * Constructs a new DeleteShardsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ICreateKeyspaceResponse); - - /** CreateKeyspaceResponse keyspace. */ - public keyspace?: (vtctldata.IKeyspace|null); + constructor(properties?: vtctldata.IDeleteShardsResponse); /** - * Creates a new CreateKeyspaceResponse instance using the specified properties. + * Creates a new DeleteShardsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns CreateKeyspaceResponse instance + * @returns DeleteShardsResponse instance */ - public static create(properties?: vtctldata.ICreateKeyspaceResponse): vtctldata.CreateKeyspaceResponse; + public static create(properties?: vtctldata.IDeleteShardsResponse): vtctldata.DeleteShardsResponse; /** - * Encodes the specified CreateKeyspaceResponse message. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. - * @param message CreateKeyspaceResponse message or plain object to encode + * Encodes the specified DeleteShardsResponse message. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. + * @param message DeleteShardsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ICreateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteShardsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. - * @param message CreateKeyspaceResponse message or plain object to encode + * Encodes the specified DeleteShardsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. + * @param message DeleteShardsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ICreateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteShardsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateKeyspaceResponse message from the specified reader or buffer. + * Decodes a DeleteShardsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateKeyspaceResponse + * @returns DeleteShardsResponse * @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.CreateKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteShardsResponse; /** - * Decodes a CreateKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteShardsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateKeyspaceResponse + * @returns DeleteShardsResponse * @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.CreateKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteShardsResponse; /** - * Verifies a CreateKeyspaceResponse message. + * Verifies a DeleteShardsResponse 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 CreateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteShardsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateKeyspaceResponse + * @returns DeleteShardsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.CreateKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteShardsResponse; /** - * Creates a plain object from a CreateKeyspaceResponse message. Also converts values to other types if specified. - * @param message CreateKeyspaceResponse + * Creates a plain object from a DeleteShardsResponse message. Also converts values to other types if specified. + * @param message DeleteShardsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.CreateKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteShardsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateKeyspaceResponse to JSON. + * Converts this DeleteShardsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateKeyspaceResponse + * Gets the default type url for DeleteShardsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateShardRequest. */ - interface ICreateShardRequest { - - /** CreateShardRequest keyspace */ - keyspace?: (string|null); - - /** CreateShardRequest shard_name */ - shard_name?: (string|null); - - /** CreateShardRequest force */ - force?: (boolean|null); + /** Properties of a DeleteSrvVSchemaRequest. */ + interface IDeleteSrvVSchemaRequest { - /** CreateShardRequest include_parent */ - include_parent?: (boolean|null); + /** DeleteSrvVSchemaRequest cell */ + cell?: (string|null); } - /** Represents a CreateShardRequest. */ - class CreateShardRequest implements ICreateShardRequest { + /** Represents a DeleteSrvVSchemaRequest. */ + class DeleteSrvVSchemaRequest implements IDeleteSrvVSchemaRequest { /** - * Constructs a new CreateShardRequest. + * Constructs a new DeleteSrvVSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ICreateShardRequest); - - /** CreateShardRequest keyspace. */ - public keyspace: string; - - /** CreateShardRequest shard_name. */ - public shard_name: string; - - /** CreateShardRequest force. */ - public force: boolean; + constructor(properties?: vtctldata.IDeleteSrvVSchemaRequest); - /** CreateShardRequest include_parent. */ - public include_parent: boolean; + /** DeleteSrvVSchemaRequest cell. */ + public cell: string; /** - * Creates a new CreateShardRequest instance using the specified properties. + * Creates a new DeleteSrvVSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns CreateShardRequest instance + * @returns DeleteSrvVSchemaRequest instance */ - public static create(properties?: vtctldata.ICreateShardRequest): vtctldata.CreateShardRequest; + public static create(properties?: vtctldata.IDeleteSrvVSchemaRequest): vtctldata.DeleteSrvVSchemaRequest; /** - * Encodes the specified CreateShardRequest message. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. - * @param message CreateShardRequest message or plain object to encode + * Encodes the specified DeleteSrvVSchemaRequest message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. + * @param message DeleteSrvVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ICreateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateShardRequest message, length delimited. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. - * @param message CreateShardRequest message or plain object to encode + * Encodes the specified DeleteSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. + * @param message DeleteSrvVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ICreateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateShardRequest message from the specified reader or buffer. + * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateShardRequest + * @returns DeleteSrvVSchemaRequest * @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.CreateShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteSrvVSchemaRequest; /** - * Decodes a CreateShardRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateShardRequest + * @returns DeleteSrvVSchemaRequest * @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.CreateShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteSrvVSchemaRequest; /** - * Verifies a CreateShardRequest message. + * Verifies a DeleteSrvVSchemaRequest 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 CreateShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateShardRequest + * @returns DeleteSrvVSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.CreateShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteSrvVSchemaRequest; /** - * Creates a plain object from a CreateShardRequest message. Also converts values to other types if specified. - * @param message CreateShardRequest + * Creates a plain object from a DeleteSrvVSchemaRequest message. Also converts values to other types if specified. + * @param message DeleteSrvVSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.CreateShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteSrvVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateShardRequest to JSON. + * Converts this DeleteSrvVSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateShardRequest + * Gets the default type url for DeleteSrvVSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a CreateShardResponse. */ - interface ICreateShardResponse { - - /** CreateShardResponse keyspace */ - keyspace?: (vtctldata.IKeyspace|null); - - /** CreateShardResponse shard */ - shard?: (vtctldata.IShard|null); - - /** CreateShardResponse shard_already_exists */ - shard_already_exists?: (boolean|null); + /** Properties of a DeleteSrvVSchemaResponse. */ + interface IDeleteSrvVSchemaResponse { } - /** Represents a CreateShardResponse. */ - class CreateShardResponse implements ICreateShardResponse { + /** Represents a DeleteSrvVSchemaResponse. */ + class DeleteSrvVSchemaResponse implements IDeleteSrvVSchemaResponse { /** - * Constructs a new CreateShardResponse. + * Constructs a new DeleteSrvVSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ICreateShardResponse); - - /** CreateShardResponse keyspace. */ - public keyspace?: (vtctldata.IKeyspace|null); - - /** CreateShardResponse shard. */ - public shard?: (vtctldata.IShard|null); - - /** CreateShardResponse shard_already_exists. */ - public shard_already_exists: boolean; + constructor(properties?: vtctldata.IDeleteSrvVSchemaResponse); /** - * Creates a new CreateShardResponse instance using the specified properties. + * Creates a new DeleteSrvVSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns CreateShardResponse instance + * @returns DeleteSrvVSchemaResponse instance */ - public static create(properties?: vtctldata.ICreateShardResponse): vtctldata.CreateShardResponse; + public static create(properties?: vtctldata.IDeleteSrvVSchemaResponse): vtctldata.DeleteSrvVSchemaResponse; /** - * Encodes the specified CreateShardResponse message. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. - * @param message CreateShardResponse message or plain object to encode + * Encodes the specified DeleteSrvVSchemaResponse message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. + * @param message DeleteSrvVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ICreateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified CreateShardResponse message, length delimited. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. - * @param message CreateShardResponse message or plain object to encode + * Encodes the specified DeleteSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. + * @param message DeleteSrvVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ICreateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a CreateShardResponse message from the specified reader or buffer. + * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns CreateShardResponse + * @returns DeleteSrvVSchemaResponse * @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.CreateShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteSrvVSchemaResponse; /** - * Decodes a CreateShardResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns CreateShardResponse + * @returns DeleteSrvVSchemaResponse * @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.CreateShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteSrvVSchemaResponse; /** - * Verifies a CreateShardResponse message. + * Verifies a DeleteSrvVSchemaResponse 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 CreateShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns CreateShardResponse + * @returns DeleteSrvVSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.CreateShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteSrvVSchemaResponse; /** - * Creates a plain object from a CreateShardResponse message. Also converts values to other types if specified. - * @param message CreateShardResponse + * Creates a plain object from a DeleteSrvVSchemaResponse message. Also converts values to other types if specified. + * @param message DeleteSrvVSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.CreateShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteSrvVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this CreateShardResponse to JSON. + * Converts this DeleteSrvVSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for CreateShardResponse + * Gets the default type url for DeleteSrvVSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteCellInfoRequest. */ - interface IDeleteCellInfoRequest { + /** Properties of a DeleteTabletsRequest. */ + interface IDeleteTabletsRequest { - /** DeleteCellInfoRequest name */ - name?: (string|null); + /** DeleteTabletsRequest tablet_aliases */ + tablet_aliases?: (topodata.ITabletAlias[]|null); - /** DeleteCellInfoRequest force */ - force?: (boolean|null); + /** DeleteTabletsRequest allow_primary */ + allow_primary?: (boolean|null); } - /** Represents a DeleteCellInfoRequest. */ - class DeleteCellInfoRequest implements IDeleteCellInfoRequest { + /** Represents a DeleteTabletsRequest. */ + class DeleteTabletsRequest implements IDeleteTabletsRequest { /** - * Constructs a new DeleteCellInfoRequest. + * Constructs a new DeleteTabletsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteCellInfoRequest); + constructor(properties?: vtctldata.IDeleteTabletsRequest); - /** DeleteCellInfoRequest name. */ - public name: string; + /** DeleteTabletsRequest tablet_aliases. */ + public tablet_aliases: topodata.ITabletAlias[]; - /** DeleteCellInfoRequest force. */ - public force: boolean; + /** DeleteTabletsRequest allow_primary. */ + public allow_primary: boolean; /** - * Creates a new DeleteCellInfoRequest instance using the specified properties. + * Creates a new DeleteTabletsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteCellInfoRequest instance + * @returns DeleteTabletsRequest instance */ - public static create(properties?: vtctldata.IDeleteCellInfoRequest): vtctldata.DeleteCellInfoRequest; + public static create(properties?: vtctldata.IDeleteTabletsRequest): vtctldata.DeleteTabletsRequest; /** - * Encodes the specified DeleteCellInfoRequest message. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. - * @param message DeleteCellInfoRequest message or plain object to encode + * Encodes the specified DeleteTabletsRequest message. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. + * @param message DeleteTabletsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. - * @param message DeleteCellInfoRequest message or plain object to encode + * Encodes the specified DeleteTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. + * @param message DeleteTabletsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteCellInfoRequest message from the specified reader or buffer. + * Decodes a DeleteTabletsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteCellInfoRequest + * @returns DeleteTabletsRequest * @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.DeleteCellInfoRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteTabletsRequest; /** - * Decodes a DeleteCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteTabletsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteCellInfoRequest + * @returns DeleteTabletsRequest * @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.DeleteCellInfoRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteTabletsRequest; /** - * Verifies a DeleteCellInfoRequest message. + * Verifies a DeleteTabletsRequest 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 DeleteCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteTabletsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteCellInfoRequest + * @returns DeleteTabletsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellInfoRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteTabletsRequest; /** - * Creates a plain object from a DeleteCellInfoRequest message. Also converts values to other types if specified. - * @param message DeleteCellInfoRequest + * Creates a plain object from a DeleteTabletsRequest message. Also converts values to other types if specified. + * @param message DeleteTabletsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteTabletsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteCellInfoRequest to JSON. + * Converts this DeleteTabletsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteCellInfoRequest + * Gets the default type url for DeleteTabletsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteCellInfoResponse. */ - interface IDeleteCellInfoResponse { + /** Properties of a DeleteTabletsResponse. */ + interface IDeleteTabletsResponse { } - /** Represents a DeleteCellInfoResponse. */ - class DeleteCellInfoResponse implements IDeleteCellInfoResponse { + /** Represents a DeleteTabletsResponse. */ + class DeleteTabletsResponse implements IDeleteTabletsResponse { /** - * Constructs a new DeleteCellInfoResponse. + * Constructs a new DeleteTabletsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteCellInfoResponse); + constructor(properties?: vtctldata.IDeleteTabletsResponse); /** - * Creates a new DeleteCellInfoResponse instance using the specified properties. + * Creates a new DeleteTabletsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteCellInfoResponse instance + * @returns DeleteTabletsResponse instance */ - public static create(properties?: vtctldata.IDeleteCellInfoResponse): vtctldata.DeleteCellInfoResponse; + public static create(properties?: vtctldata.IDeleteTabletsResponse): vtctldata.DeleteTabletsResponse; /** - * Encodes the specified DeleteCellInfoResponse message. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. - * @param message DeleteCellInfoResponse message or plain object to encode + * Encodes the specified DeleteTabletsResponse message. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. + * @param message DeleteTabletsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IDeleteTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. - * @param message DeleteCellInfoResponse message or plain object to encode + * Encodes the specified DeleteTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. + * @param message DeleteTabletsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IDeleteTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteCellInfoResponse message from the specified reader or buffer. + * Decodes a DeleteTabletsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteCellInfoResponse + * @returns DeleteTabletsResponse * @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.DeleteCellInfoResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.DeleteTabletsResponse; /** - * Decodes a DeleteCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteTabletsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteCellInfoResponse + * @returns DeleteTabletsResponse * @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.DeleteCellInfoResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.DeleteTabletsResponse; /** - * Verifies a DeleteCellInfoResponse message. + * Verifies a DeleteTabletsResponse 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 DeleteCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteTabletsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteCellInfoResponse + * @returns DeleteTabletsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellInfoResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.DeleteTabletsResponse; /** - * Creates a plain object from a DeleteCellInfoResponse message. Also converts values to other types if specified. - * @param message DeleteCellInfoResponse + * Creates a plain object from a DeleteTabletsResponse message. Also converts values to other types if specified. + * @param message DeleteTabletsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.DeleteTabletsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteCellInfoResponse to JSON. + * Converts this DeleteTabletsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteCellInfoResponse + * Gets the default type url for DeleteTabletsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteCellsAliasRequest. */ - interface IDeleteCellsAliasRequest { + /** Properties of an EmergencyReparentShardRequest. */ + interface IEmergencyReparentShardRequest { - /** DeleteCellsAliasRequest name */ - name?: (string|null); + /** EmergencyReparentShardRequest keyspace */ + keyspace?: (string|null); + + /** EmergencyReparentShardRequest shard */ + shard?: (string|null); + + /** EmergencyReparentShardRequest new_primary */ + new_primary?: (topodata.ITabletAlias|null); + + /** EmergencyReparentShardRequest ignore_replicas */ + ignore_replicas?: (topodata.ITabletAlias[]|null); + + /** EmergencyReparentShardRequest wait_replicas_timeout */ + wait_replicas_timeout?: (vttime.IDuration|null); + + /** EmergencyReparentShardRequest prevent_cross_cell_promotion */ + prevent_cross_cell_promotion?: (boolean|null); } - /** Represents a DeleteCellsAliasRequest. */ - class DeleteCellsAliasRequest implements IDeleteCellsAliasRequest { + /** Represents an EmergencyReparentShardRequest. */ + class EmergencyReparentShardRequest implements IEmergencyReparentShardRequest { /** - * Constructs a new DeleteCellsAliasRequest. + * Constructs a new EmergencyReparentShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteCellsAliasRequest); + constructor(properties?: vtctldata.IEmergencyReparentShardRequest); - /** DeleteCellsAliasRequest name. */ - public name: string; + /** EmergencyReparentShardRequest keyspace. */ + public keyspace: string; + + /** EmergencyReparentShardRequest shard. */ + public shard: string; + + /** EmergencyReparentShardRequest new_primary. */ + public new_primary?: (topodata.ITabletAlias|null); + + /** EmergencyReparentShardRequest ignore_replicas. */ + public ignore_replicas: topodata.ITabletAlias[]; + + /** EmergencyReparentShardRequest wait_replicas_timeout. */ + public wait_replicas_timeout?: (vttime.IDuration|null); + + /** EmergencyReparentShardRequest prevent_cross_cell_promotion. */ + public prevent_cross_cell_promotion: boolean; /** - * Creates a new DeleteCellsAliasRequest instance using the specified properties. + * Creates a new EmergencyReparentShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteCellsAliasRequest instance + * @returns EmergencyReparentShardRequest instance */ - public static create(properties?: vtctldata.IDeleteCellsAliasRequest): vtctldata.DeleteCellsAliasRequest; + public static create(properties?: vtctldata.IEmergencyReparentShardRequest): vtctldata.EmergencyReparentShardRequest; /** - * Encodes the specified DeleteCellsAliasRequest message. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. - * @param message DeleteCellsAliasRequest message or plain object to encode + * Encodes the specified EmergencyReparentShardRequest message. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. + * @param message EmergencyReparentShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IEmergencyReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. - * @param message DeleteCellsAliasRequest message or plain object to encode + * Encodes the specified EmergencyReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. + * @param message EmergencyReparentShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IEmergencyReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer. + * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteCellsAliasRequest + * @returns EmergencyReparentShardRequest * @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.DeleteCellsAliasRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.EmergencyReparentShardRequest; /** - * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteCellsAliasRequest + * @returns EmergencyReparentShardRequest * @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.DeleteCellsAliasRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.EmergencyReparentShardRequest; /** - * Verifies a DeleteCellsAliasRequest message. + * Verifies an EmergencyReparentShardRequest 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 DeleteCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates an EmergencyReparentShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteCellsAliasRequest + * @returns EmergencyReparentShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellsAliasRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.EmergencyReparentShardRequest; /** - * Creates a plain object from a DeleteCellsAliasRequest message. Also converts values to other types if specified. - * @param message DeleteCellsAliasRequest + * Creates a plain object from an EmergencyReparentShardRequest message. Also converts values to other types if specified. + * @param message EmergencyReparentShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.EmergencyReparentShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteCellsAliasRequest to JSON. + * Converts this EmergencyReparentShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteCellsAliasRequest + * Gets the default type url for EmergencyReparentShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteCellsAliasResponse. */ - interface IDeleteCellsAliasResponse { + /** Properties of an EmergencyReparentShardResponse. */ + interface IEmergencyReparentShardResponse { + + /** EmergencyReparentShardResponse keyspace */ + keyspace?: (string|null); + + /** EmergencyReparentShardResponse shard */ + shard?: (string|null); + + /** EmergencyReparentShardResponse promoted_primary */ + promoted_primary?: (topodata.ITabletAlias|null); + + /** EmergencyReparentShardResponse events */ + events?: (logutil.IEvent[]|null); } - /** Represents a DeleteCellsAliasResponse. */ - class DeleteCellsAliasResponse implements IDeleteCellsAliasResponse { + /** Represents an EmergencyReparentShardResponse. */ + class EmergencyReparentShardResponse implements IEmergencyReparentShardResponse { /** - * Constructs a new DeleteCellsAliasResponse. + * Constructs a new EmergencyReparentShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteCellsAliasResponse); + constructor(properties?: vtctldata.IEmergencyReparentShardResponse); + + /** EmergencyReparentShardResponse keyspace. */ + public keyspace: string; + + /** EmergencyReparentShardResponse shard. */ + public shard: string; + + /** EmergencyReparentShardResponse promoted_primary. */ + public promoted_primary?: (topodata.ITabletAlias|null); + + /** EmergencyReparentShardResponse events. */ + public events: logutil.IEvent[]; /** - * Creates a new DeleteCellsAliasResponse instance using the specified properties. + * Creates a new EmergencyReparentShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteCellsAliasResponse instance + * @returns EmergencyReparentShardResponse instance */ - public static create(properties?: vtctldata.IDeleteCellsAliasResponse): vtctldata.DeleteCellsAliasResponse; + public static create(properties?: vtctldata.IEmergencyReparentShardResponse): vtctldata.EmergencyReparentShardResponse; /** - * Encodes the specified DeleteCellsAliasResponse message. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. - * @param message DeleteCellsAliasResponse message or plain object to encode + * Encodes the specified EmergencyReparentShardResponse message. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. + * @param message EmergencyReparentShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IEmergencyReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. - * @param message DeleteCellsAliasResponse message or plain object to encode + * Encodes the specified EmergencyReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. + * @param message EmergencyReparentShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IEmergencyReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer. + * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteCellsAliasResponse + * @returns EmergencyReparentShardResponse * @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.DeleteCellsAliasResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.EmergencyReparentShardResponse; /** - * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteCellsAliasResponse + * @returns EmergencyReparentShardResponse * @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.DeleteCellsAliasResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.EmergencyReparentShardResponse; /** - * Verifies a DeleteCellsAliasResponse message. + * Verifies an EmergencyReparentShardResponse 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 DeleteCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates an EmergencyReparentShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteCellsAliasResponse + * @returns EmergencyReparentShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteCellsAliasResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.EmergencyReparentShardResponse; /** - * Creates a plain object from a DeleteCellsAliasResponse message. Also converts values to other types if specified. - * @param message DeleteCellsAliasResponse + * Creates a plain object from an EmergencyReparentShardResponse message. Also converts values to other types if specified. + * @param message EmergencyReparentShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.EmergencyReparentShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteCellsAliasResponse to JSON. + * Converts this EmergencyReparentShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteCellsAliasResponse + * Gets the default type url for EmergencyReparentShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteKeyspaceRequest. */ - interface IDeleteKeyspaceRequest { + /** Properties of an ExecuteFetchAsAppRequest. */ + interface IExecuteFetchAsAppRequest { - /** DeleteKeyspaceRequest keyspace */ - keyspace?: (string|null); + /** ExecuteFetchAsAppRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** DeleteKeyspaceRequest recursive */ - recursive?: (boolean|null); + /** ExecuteFetchAsAppRequest query */ + query?: (string|null); - /** DeleteKeyspaceRequest force */ - force?: (boolean|null); + /** ExecuteFetchAsAppRequest max_rows */ + max_rows?: (number|Long|null); + + /** ExecuteFetchAsAppRequest use_pool */ + use_pool?: (boolean|null); } - /** Represents a DeleteKeyspaceRequest. */ - class DeleteKeyspaceRequest implements IDeleteKeyspaceRequest { + /** Represents an ExecuteFetchAsAppRequest. */ + class ExecuteFetchAsAppRequest implements IExecuteFetchAsAppRequest { /** - * Constructs a new DeleteKeyspaceRequest. + * Constructs a new ExecuteFetchAsAppRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteKeyspaceRequest); + constructor(properties?: vtctldata.IExecuteFetchAsAppRequest); - /** DeleteKeyspaceRequest keyspace. */ - public keyspace: string; + /** ExecuteFetchAsAppRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** DeleteKeyspaceRequest recursive. */ - public recursive: boolean; + /** ExecuteFetchAsAppRequest query. */ + public query: string; - /** DeleteKeyspaceRequest force. */ - public force: boolean; + /** ExecuteFetchAsAppRequest max_rows. */ + public max_rows: (number|Long); + + /** ExecuteFetchAsAppRequest use_pool. */ + public use_pool: boolean; /** - * Creates a new DeleteKeyspaceRequest instance using the specified properties. + * Creates a new ExecuteFetchAsAppRequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteKeyspaceRequest instance + * @returns ExecuteFetchAsAppRequest instance */ - public static create(properties?: vtctldata.IDeleteKeyspaceRequest): vtctldata.DeleteKeyspaceRequest; + public static create(properties?: vtctldata.IExecuteFetchAsAppRequest): vtctldata.ExecuteFetchAsAppRequest; /** - * Encodes the specified DeleteKeyspaceRequest message. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. - * @param message DeleteKeyspaceRequest message or plain object to encode + * Encodes the specified ExecuteFetchAsAppRequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. + * @param message ExecuteFetchAsAppRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteFetchAsAppRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. - * @param message DeleteKeyspaceRequest message or plain object to encode + * Encodes the specified ExecuteFetchAsAppRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. + * @param message ExecuteFetchAsAppRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteFetchAsAppRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer. + * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteKeyspaceRequest + * @returns ExecuteFetchAsAppRequest * @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.DeleteKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteFetchAsAppRequest; /** - * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteKeyspaceRequest + * @returns ExecuteFetchAsAppRequest * @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.DeleteKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteFetchAsAppRequest; /** - * Verifies a DeleteKeyspaceRequest message. + * Verifies an ExecuteFetchAsAppRequest 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 DeleteKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsAppRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteKeyspaceRequest + * @returns ExecuteFetchAsAppRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsAppRequest; /** - * Creates a plain object from a DeleteKeyspaceRequest message. Also converts values to other types if specified. - * @param message DeleteKeyspaceRequest + * Creates a plain object from an ExecuteFetchAsAppRequest message. Also converts values to other types if specified. + * @param message ExecuteFetchAsAppRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteFetchAsAppRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteKeyspaceRequest to JSON. + * Converts this ExecuteFetchAsAppRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteKeyspaceRequest + * Gets the default type url for ExecuteFetchAsAppRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteKeyspaceResponse. */ - interface IDeleteKeyspaceResponse { + /** Properties of an ExecuteFetchAsAppResponse. */ + interface IExecuteFetchAsAppResponse { + + /** ExecuteFetchAsAppResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a DeleteKeyspaceResponse. */ - class DeleteKeyspaceResponse implements IDeleteKeyspaceResponse { + /** Represents an ExecuteFetchAsAppResponse. */ + class ExecuteFetchAsAppResponse implements IExecuteFetchAsAppResponse { /** - * Constructs a new DeleteKeyspaceResponse. + * Constructs a new ExecuteFetchAsAppResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteKeyspaceResponse); + constructor(properties?: vtctldata.IExecuteFetchAsAppResponse); + + /** ExecuteFetchAsAppResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new DeleteKeyspaceResponse instance using the specified properties. + * Creates a new ExecuteFetchAsAppResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteKeyspaceResponse instance + * @returns ExecuteFetchAsAppResponse instance */ - public static create(properties?: vtctldata.IDeleteKeyspaceResponse): vtctldata.DeleteKeyspaceResponse; + public static create(properties?: vtctldata.IExecuteFetchAsAppResponse): vtctldata.ExecuteFetchAsAppResponse; /** - * Encodes the specified DeleteKeyspaceResponse message. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. - * @param message DeleteKeyspaceResponse message or plain object to encode + * Encodes the specified ExecuteFetchAsAppResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. + * @param message ExecuteFetchAsAppResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteFetchAsAppResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. - * @param message DeleteKeyspaceResponse message or plain object to encode + * Encodes the specified ExecuteFetchAsAppResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. + * @param message ExecuteFetchAsAppResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteFetchAsAppResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer. + * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteKeyspaceResponse + * @returns ExecuteFetchAsAppResponse * @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.DeleteKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteFetchAsAppResponse; /** - * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteKeyspaceResponse + * @returns ExecuteFetchAsAppResponse * @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.DeleteKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteFetchAsAppResponse; /** - * Verifies a DeleteKeyspaceResponse message. + * Verifies an ExecuteFetchAsAppResponse 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 DeleteKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsAppResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteKeyspaceResponse + * @returns ExecuteFetchAsAppResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsAppResponse; /** - * Creates a plain object from a DeleteKeyspaceResponse message. Also converts values to other types if specified. - * @param message DeleteKeyspaceResponse + * Creates a plain object from an ExecuteFetchAsAppResponse message. Also converts values to other types if specified. + * @param message ExecuteFetchAsAppResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteFetchAsAppResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteKeyspaceResponse to JSON. + * Converts this ExecuteFetchAsAppResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteKeyspaceResponse + * Gets the default type url for ExecuteFetchAsAppResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteShardsRequest. */ - interface IDeleteShardsRequest { + /** Properties of an ExecuteFetchAsDBARequest. */ + interface IExecuteFetchAsDBARequest { - /** DeleteShardsRequest shards */ - shards?: (vtctldata.IShard[]|null); + /** ExecuteFetchAsDBARequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** DeleteShardsRequest recursive */ - recursive?: (boolean|null); + /** ExecuteFetchAsDBARequest query */ + query?: (string|null); - /** DeleteShardsRequest even_if_serving */ - even_if_serving?: (boolean|null); + /** ExecuteFetchAsDBARequest max_rows */ + max_rows?: (number|Long|null); - /** DeleteShardsRequest force */ - force?: (boolean|null); + /** ExecuteFetchAsDBARequest disable_binlogs */ + disable_binlogs?: (boolean|null); + + /** ExecuteFetchAsDBARequest reload_schema */ + reload_schema?: (boolean|null); } - /** Represents a DeleteShardsRequest. */ - class DeleteShardsRequest implements IDeleteShardsRequest { + /** Represents an ExecuteFetchAsDBARequest. */ + class ExecuteFetchAsDBARequest implements IExecuteFetchAsDBARequest { /** - * Constructs a new DeleteShardsRequest. + * Constructs a new ExecuteFetchAsDBARequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteShardsRequest); + constructor(properties?: vtctldata.IExecuteFetchAsDBARequest); - /** DeleteShardsRequest shards. */ - public shards: vtctldata.IShard[]; + /** ExecuteFetchAsDBARequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** DeleteShardsRequest recursive. */ - public recursive: boolean; + /** ExecuteFetchAsDBARequest query. */ + public query: string; - /** DeleteShardsRequest even_if_serving. */ - public even_if_serving: boolean; + /** ExecuteFetchAsDBARequest max_rows. */ + public max_rows: (number|Long); - /** DeleteShardsRequest force. */ - public force: boolean; + /** ExecuteFetchAsDBARequest disable_binlogs. */ + public disable_binlogs: boolean; + + /** ExecuteFetchAsDBARequest reload_schema. */ + public reload_schema: boolean; /** - * Creates a new DeleteShardsRequest instance using the specified properties. + * Creates a new ExecuteFetchAsDBARequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteShardsRequest instance + * @returns ExecuteFetchAsDBARequest instance */ - public static create(properties?: vtctldata.IDeleteShardsRequest): vtctldata.DeleteShardsRequest; + public static create(properties?: vtctldata.IExecuteFetchAsDBARequest): vtctldata.ExecuteFetchAsDBARequest; /** - * Encodes the specified DeleteShardsRequest message. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. - * @param message DeleteShardsRequest message or plain object to encode + * Encodes the specified ExecuteFetchAsDBARequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. + * @param message ExecuteFetchAsDBARequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteShardsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteFetchAsDBARequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteShardsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. - * @param message DeleteShardsRequest message or plain object to encode + * Encodes the specified ExecuteFetchAsDBARequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. + * @param message ExecuteFetchAsDBARequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteShardsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteFetchAsDBARequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteShardsRequest message from the specified reader or buffer. + * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteShardsRequest + * @returns ExecuteFetchAsDBARequest * @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.DeleteShardsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteFetchAsDBARequest; /** - * Decodes a DeleteShardsRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteShardsRequest + * @returns ExecuteFetchAsDBARequest * @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.DeleteShardsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteFetchAsDBARequest; /** - * Verifies a DeleteShardsRequest message. + * Verifies an ExecuteFetchAsDBARequest 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 DeleteShardsRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsDBARequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteShardsRequest + * @returns ExecuteFetchAsDBARequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteShardsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsDBARequest; /** - * Creates a plain object from a DeleteShardsRequest message. Also converts values to other types if specified. - * @param message DeleteShardsRequest + * Creates a plain object from an ExecuteFetchAsDBARequest message. Also converts values to other types if specified. + * @param message ExecuteFetchAsDBARequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteShardsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteFetchAsDBARequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteShardsRequest to JSON. + * Converts this ExecuteFetchAsDBARequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteShardsRequest + * Gets the default type url for ExecuteFetchAsDBARequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteShardsResponse. */ - interface IDeleteShardsResponse { + /** Properties of an ExecuteFetchAsDBAResponse. */ + interface IExecuteFetchAsDBAResponse { + + /** ExecuteFetchAsDBAResponse result */ + result?: (query.IQueryResult|null); } - /** Represents a DeleteShardsResponse. */ - class DeleteShardsResponse implements IDeleteShardsResponse { + /** Represents an ExecuteFetchAsDBAResponse. */ + class ExecuteFetchAsDBAResponse implements IExecuteFetchAsDBAResponse { /** - * Constructs a new DeleteShardsResponse. + * Constructs a new ExecuteFetchAsDBAResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteShardsResponse); + constructor(properties?: vtctldata.IExecuteFetchAsDBAResponse); + + /** ExecuteFetchAsDBAResponse result. */ + public result?: (query.IQueryResult|null); /** - * Creates a new DeleteShardsResponse instance using the specified properties. + * Creates a new ExecuteFetchAsDBAResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteShardsResponse instance + * @returns ExecuteFetchAsDBAResponse instance */ - public static create(properties?: vtctldata.IDeleteShardsResponse): vtctldata.DeleteShardsResponse; + public static create(properties?: vtctldata.IExecuteFetchAsDBAResponse): vtctldata.ExecuteFetchAsDBAResponse; /** - * Encodes the specified DeleteShardsResponse message. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. - * @param message DeleteShardsResponse message or plain object to encode + * Encodes the specified ExecuteFetchAsDBAResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. + * @param message ExecuteFetchAsDBAResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteShardsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteFetchAsDBAResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteShardsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. - * @param message DeleteShardsResponse message or plain object to encode + * Encodes the specified ExecuteFetchAsDBAResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. + * @param message ExecuteFetchAsDBAResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteShardsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteFetchAsDBAResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteShardsResponse message from the specified reader or buffer. + * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteShardsResponse + * @returns ExecuteFetchAsDBAResponse * @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.DeleteShardsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteFetchAsDBAResponse; /** - * Decodes a DeleteShardsResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteShardsResponse + * @returns ExecuteFetchAsDBAResponse * @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.DeleteShardsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteFetchAsDBAResponse; /** - * Verifies a DeleteShardsResponse message. + * Verifies an ExecuteFetchAsDBAResponse 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 DeleteShardsResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsDBAResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteShardsResponse + * @returns ExecuteFetchAsDBAResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteShardsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsDBAResponse; /** - * Creates a plain object from a DeleteShardsResponse message. Also converts values to other types if specified. - * @param message DeleteShardsResponse + * Creates a plain object from an ExecuteFetchAsDBAResponse message. Also converts values to other types if specified. + * @param message ExecuteFetchAsDBAResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteShardsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteFetchAsDBAResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteShardsResponse to JSON. + * Converts this ExecuteFetchAsDBAResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteShardsResponse + * Gets the default type url for ExecuteFetchAsDBAResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteSrvVSchemaRequest. */ - interface IDeleteSrvVSchemaRequest { + /** Properties of an ExecuteHookRequest. */ + interface IExecuteHookRequest { - /** DeleteSrvVSchemaRequest cell */ - cell?: (string|null); + /** ExecuteHookRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** ExecuteHookRequest tablet_hook_request */ + tablet_hook_request?: (tabletmanagerdata.IExecuteHookRequest|null); } - /** Represents a DeleteSrvVSchemaRequest. */ - class DeleteSrvVSchemaRequest implements IDeleteSrvVSchemaRequest { + /** Represents an ExecuteHookRequest. */ + class ExecuteHookRequest implements IExecuteHookRequest { /** - * Constructs a new DeleteSrvVSchemaRequest. + * Constructs a new ExecuteHookRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteSrvVSchemaRequest); + constructor(properties?: vtctldata.IExecuteHookRequest); - /** DeleteSrvVSchemaRequest cell. */ - public cell: string; + /** ExecuteHookRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** ExecuteHookRequest tablet_hook_request. */ + public tablet_hook_request?: (tabletmanagerdata.IExecuteHookRequest|null); /** - * Creates a new DeleteSrvVSchemaRequest instance using the specified properties. + * Creates a new ExecuteHookRequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteSrvVSchemaRequest instance + * @returns ExecuteHookRequest instance */ - public static create(properties?: vtctldata.IDeleteSrvVSchemaRequest): vtctldata.DeleteSrvVSchemaRequest; + public static create(properties?: vtctldata.IExecuteHookRequest): vtctldata.ExecuteHookRequest; /** - * Encodes the specified DeleteSrvVSchemaRequest message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. - * @param message DeleteSrvVSchemaRequest message or plain object to encode + * Encodes the specified ExecuteHookRequest message. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. + * @param message ExecuteHookRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteHookRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. - * @param message DeleteSrvVSchemaRequest message or plain object to encode + * Encodes the specified ExecuteHookRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. + * @param message ExecuteHookRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteHookRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer. + * Decodes an ExecuteHookRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteSrvVSchemaRequest + * @returns ExecuteHookRequest * @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.DeleteSrvVSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteHookRequest; /** - * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteHookRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteSrvVSchemaRequest + * @returns ExecuteHookRequest * @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.DeleteSrvVSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteHookRequest; /** - * Verifies a DeleteSrvVSchemaRequest message. + * Verifies an ExecuteHookRequest 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 DeleteSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteHookRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteSrvVSchemaRequest + * @returns ExecuteHookRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteSrvVSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteHookRequest; /** - * Creates a plain object from a DeleteSrvVSchemaRequest message. Also converts values to other types if specified. - * @param message DeleteSrvVSchemaRequest + * Creates a plain object from an ExecuteHookRequest message. Also converts values to other types if specified. + * @param message ExecuteHookRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteSrvVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteHookRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteSrvVSchemaRequest to JSON. + * Converts this ExecuteHookRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteSrvVSchemaRequest + * Gets the default type url for ExecuteHookRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteSrvVSchemaResponse. */ - interface IDeleteSrvVSchemaResponse { + /** Properties of an ExecuteHookResponse. */ + interface IExecuteHookResponse { + + /** ExecuteHookResponse hook_result */ + hook_result?: (tabletmanagerdata.IExecuteHookResponse|null); } - /** Represents a DeleteSrvVSchemaResponse. */ - class DeleteSrvVSchemaResponse implements IDeleteSrvVSchemaResponse { + /** Represents an ExecuteHookResponse. */ + class ExecuteHookResponse implements IExecuteHookResponse { /** - * Constructs a new DeleteSrvVSchemaResponse. + * Constructs a new ExecuteHookResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteSrvVSchemaResponse); + constructor(properties?: vtctldata.IExecuteHookResponse); + + /** ExecuteHookResponse hook_result. */ + public hook_result?: (tabletmanagerdata.IExecuteHookResponse|null); /** - * Creates a new DeleteSrvVSchemaResponse instance using the specified properties. + * Creates a new ExecuteHookResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteSrvVSchemaResponse instance + * @returns ExecuteHookResponse instance */ - public static create(properties?: vtctldata.IDeleteSrvVSchemaResponse): vtctldata.DeleteSrvVSchemaResponse; + public static create(properties?: vtctldata.IExecuteHookResponse): vtctldata.ExecuteHookResponse; /** - * Encodes the specified DeleteSrvVSchemaResponse message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. - * @param message DeleteSrvVSchemaResponse message or plain object to encode + * Encodes the specified ExecuteHookResponse message. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. + * @param message ExecuteHookResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IExecuteHookResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. - * @param message DeleteSrvVSchemaResponse message or plain object to encode + * Encodes the specified ExecuteHookResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. + * @param message ExecuteHookResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IExecuteHookResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer. + * Decodes an ExecuteHookResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteSrvVSchemaResponse + * @returns ExecuteHookResponse * @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.DeleteSrvVSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ExecuteHookResponse; /** - * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteHookResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteSrvVSchemaResponse + * @returns ExecuteHookResponse * @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.DeleteSrvVSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ExecuteHookResponse; /** - * Verifies a DeleteSrvVSchemaResponse message. + * Verifies an ExecuteHookResponse 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 DeleteSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteHookResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteSrvVSchemaResponse + * @returns ExecuteHookResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteSrvVSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteHookResponse; /** - * Creates a plain object from a DeleteSrvVSchemaResponse message. Also converts values to other types if specified. - * @param message DeleteSrvVSchemaResponse + * Creates a plain object from an ExecuteHookResponse message. Also converts values to other types if specified. + * @param message ExecuteHookResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteSrvVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ExecuteHookResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteSrvVSchemaResponse to JSON. + * Converts this ExecuteHookResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteSrvVSchemaResponse + * Gets the default type url for ExecuteHookResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteTabletsRequest. */ - interface IDeleteTabletsRequest { - - /** DeleteTabletsRequest tablet_aliases */ - tablet_aliases?: (topodata.ITabletAlias[]|null); + /** Properties of a FindAllShardsInKeyspaceRequest. */ + interface IFindAllShardsInKeyspaceRequest { - /** DeleteTabletsRequest allow_primary */ - allow_primary?: (boolean|null); + /** FindAllShardsInKeyspaceRequest keyspace */ + keyspace?: (string|null); } - /** Represents a DeleteTabletsRequest. */ - class DeleteTabletsRequest implements IDeleteTabletsRequest { + /** Represents a FindAllShardsInKeyspaceRequest. */ + class FindAllShardsInKeyspaceRequest implements IFindAllShardsInKeyspaceRequest { /** - * Constructs a new DeleteTabletsRequest. + * Constructs a new FindAllShardsInKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteTabletsRequest); - - /** DeleteTabletsRequest tablet_aliases. */ - public tablet_aliases: topodata.ITabletAlias[]; + constructor(properties?: vtctldata.IFindAllShardsInKeyspaceRequest); - /** DeleteTabletsRequest allow_primary. */ - public allow_primary: boolean; + /** FindAllShardsInKeyspaceRequest keyspace. */ + public keyspace: string; /** - * Creates a new DeleteTabletsRequest instance using the specified properties. + * Creates a new FindAllShardsInKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteTabletsRequest instance + * @returns FindAllShardsInKeyspaceRequest instance */ - public static create(properties?: vtctldata.IDeleteTabletsRequest): vtctldata.DeleteTabletsRequest; + public static create(properties?: vtctldata.IFindAllShardsInKeyspaceRequest): vtctldata.FindAllShardsInKeyspaceRequest; /** - * Encodes the specified DeleteTabletsRequest message. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. - * @param message DeleteTabletsRequest message or plain object to encode + * Encodes the specified FindAllShardsInKeyspaceRequest message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. + * @param message FindAllShardsInKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IFindAllShardsInKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. - * @param message DeleteTabletsRequest message or plain object to encode + * Encodes the specified FindAllShardsInKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. + * @param message FindAllShardsInKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IFindAllShardsInKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteTabletsRequest message from the specified reader or buffer. + * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteTabletsRequest + * @returns FindAllShardsInKeyspaceRequest * @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.DeleteTabletsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.FindAllShardsInKeyspaceRequest; /** - * Decodes a DeleteTabletsRequest message from the specified reader or buffer, length delimited. + * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteTabletsRequest + * @returns FindAllShardsInKeyspaceRequest * @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.DeleteTabletsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.FindAllShardsInKeyspaceRequest; /** - * Verifies a DeleteTabletsRequest message. + * Verifies a FindAllShardsInKeyspaceRequest 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 DeleteTabletsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a FindAllShardsInKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteTabletsRequest + * @returns FindAllShardsInKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteTabletsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.FindAllShardsInKeyspaceRequest; /** - * Creates a plain object from a DeleteTabletsRequest message. Also converts values to other types if specified. - * @param message DeleteTabletsRequest + * Creates a plain object from a FindAllShardsInKeyspaceRequest message. Also converts values to other types if specified. + * @param message FindAllShardsInKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteTabletsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.FindAllShardsInKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteTabletsRequest to JSON. + * Converts this FindAllShardsInKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteTabletsRequest + * Gets the default type url for FindAllShardsInKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a DeleteTabletsResponse. */ - interface IDeleteTabletsResponse { + /** Properties of a FindAllShardsInKeyspaceResponse. */ + interface IFindAllShardsInKeyspaceResponse { + + /** FindAllShardsInKeyspaceResponse shards */ + shards?: ({ [k: string]: vtctldata.IShard }|null); } - /** Represents a DeleteTabletsResponse. */ - class DeleteTabletsResponse implements IDeleteTabletsResponse { + /** Represents a FindAllShardsInKeyspaceResponse. */ + class FindAllShardsInKeyspaceResponse implements IFindAllShardsInKeyspaceResponse { /** - * Constructs a new DeleteTabletsResponse. + * Constructs a new FindAllShardsInKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IDeleteTabletsResponse); + constructor(properties?: vtctldata.IFindAllShardsInKeyspaceResponse); + + /** FindAllShardsInKeyspaceResponse shards. */ + public shards: { [k: string]: vtctldata.IShard }; /** - * Creates a new DeleteTabletsResponse instance using the specified properties. + * Creates a new FindAllShardsInKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns DeleteTabletsResponse instance + * @returns FindAllShardsInKeyspaceResponse instance */ - public static create(properties?: vtctldata.IDeleteTabletsResponse): vtctldata.DeleteTabletsResponse; + public static create(properties?: vtctldata.IFindAllShardsInKeyspaceResponse): vtctldata.FindAllShardsInKeyspaceResponse; /** - * Encodes the specified DeleteTabletsResponse message. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. - * @param message DeleteTabletsResponse message or plain object to encode + * Encodes the specified FindAllShardsInKeyspaceResponse message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. + * @param message FindAllShardsInKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IDeleteTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IFindAllShardsInKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified DeleteTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. - * @param message DeleteTabletsResponse message or plain object to encode + * Encodes the specified FindAllShardsInKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. + * @param message FindAllShardsInKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IDeleteTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IFindAllShardsInKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a DeleteTabletsResponse message from the specified reader or buffer. + * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns DeleteTabletsResponse + * @returns FindAllShardsInKeyspaceResponse * @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.DeleteTabletsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.FindAllShardsInKeyspaceResponse; /** - * Decodes a DeleteTabletsResponse message from the specified reader or buffer, length delimited. + * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns DeleteTabletsResponse + * @returns FindAllShardsInKeyspaceResponse * @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.DeleteTabletsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.FindAllShardsInKeyspaceResponse; /** - * Verifies a DeleteTabletsResponse message. + * Verifies a FindAllShardsInKeyspaceResponse 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 DeleteTabletsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a FindAllShardsInKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns DeleteTabletsResponse + * @returns FindAllShardsInKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.DeleteTabletsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.FindAllShardsInKeyspaceResponse; /** - * Creates a plain object from a DeleteTabletsResponse message. Also converts values to other types if specified. - * @param message DeleteTabletsResponse + * Creates a plain object from a FindAllShardsInKeyspaceResponse message. Also converts values to other types if specified. + * @param message FindAllShardsInKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.DeleteTabletsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.FindAllShardsInKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this DeleteTabletsResponse to JSON. + * Converts this FindAllShardsInKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for DeleteTabletsResponse + * Gets the default type url for FindAllShardsInKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an EmergencyReparentShardRequest. */ - interface IEmergencyReparentShardRequest { + /** Properties of a GetBackupsRequest. */ + interface IGetBackupsRequest { - /** EmergencyReparentShardRequest keyspace */ + /** GetBackupsRequest keyspace */ keyspace?: (string|null); - /** EmergencyReparentShardRequest shard */ + /** GetBackupsRequest shard */ shard?: (string|null); - /** EmergencyReparentShardRequest new_primary */ - new_primary?: (topodata.ITabletAlias|null); - - /** EmergencyReparentShardRequest ignore_replicas */ - ignore_replicas?: (topodata.ITabletAlias[]|null); + /** GetBackupsRequest limit */ + limit?: (number|null); - /** EmergencyReparentShardRequest wait_replicas_timeout */ - wait_replicas_timeout?: (vttime.IDuration|null); + /** GetBackupsRequest detailed */ + detailed?: (boolean|null); - /** EmergencyReparentShardRequest prevent_cross_cell_promotion */ - prevent_cross_cell_promotion?: (boolean|null); + /** GetBackupsRequest detailed_limit */ + detailed_limit?: (number|null); } - /** Represents an EmergencyReparentShardRequest. */ - class EmergencyReparentShardRequest implements IEmergencyReparentShardRequest { + /** Represents a GetBackupsRequest. */ + class GetBackupsRequest implements IGetBackupsRequest { /** - * Constructs a new EmergencyReparentShardRequest. + * Constructs a new GetBackupsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IEmergencyReparentShardRequest); + constructor(properties?: vtctldata.IGetBackupsRequest); - /** EmergencyReparentShardRequest keyspace. */ + /** GetBackupsRequest keyspace. */ public keyspace: string; - /** EmergencyReparentShardRequest shard. */ + /** GetBackupsRequest shard. */ public shard: string; - /** EmergencyReparentShardRequest new_primary. */ - public new_primary?: (topodata.ITabletAlias|null); - - /** EmergencyReparentShardRequest ignore_replicas. */ - public ignore_replicas: topodata.ITabletAlias[]; + /** GetBackupsRequest limit. */ + public limit: number; - /** EmergencyReparentShardRequest wait_replicas_timeout. */ - public wait_replicas_timeout?: (vttime.IDuration|null); + /** GetBackupsRequest detailed. */ + public detailed: boolean; - /** EmergencyReparentShardRequest prevent_cross_cell_promotion. */ - public prevent_cross_cell_promotion: boolean; + /** GetBackupsRequest detailed_limit. */ + public detailed_limit: number; /** - * Creates a new EmergencyReparentShardRequest instance using the specified properties. + * Creates a new GetBackupsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns EmergencyReparentShardRequest instance + * @returns GetBackupsRequest instance */ - public static create(properties?: vtctldata.IEmergencyReparentShardRequest): vtctldata.EmergencyReparentShardRequest; + public static create(properties?: vtctldata.IGetBackupsRequest): vtctldata.GetBackupsRequest; /** - * Encodes the specified EmergencyReparentShardRequest message. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. - * @param message EmergencyReparentShardRequest message or plain object to encode + * Encodes the specified GetBackupsRequest message. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. + * @param message GetBackupsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IEmergencyReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetBackupsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified EmergencyReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. - * @param message EmergencyReparentShardRequest message or plain object to encode + * Encodes the specified GetBackupsRequest message, length delimited. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. + * @param message GetBackupsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IEmergencyReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetBackupsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer. + * Decodes a GetBackupsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns EmergencyReparentShardRequest + * @returns GetBackupsRequest * @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.EmergencyReparentShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetBackupsRequest; /** - * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer, length delimited. + * Decodes a GetBackupsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns EmergencyReparentShardRequest + * @returns GetBackupsRequest * @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.EmergencyReparentShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetBackupsRequest; /** - * Verifies an EmergencyReparentShardRequest message. + * Verifies a GetBackupsRequest 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 EmergencyReparentShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetBackupsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns EmergencyReparentShardRequest + * @returns GetBackupsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.EmergencyReparentShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetBackupsRequest; /** - * Creates a plain object from an EmergencyReparentShardRequest message. Also converts values to other types if specified. - * @param message EmergencyReparentShardRequest + * Creates a plain object from a GetBackupsRequest message. Also converts values to other types if specified. + * @param message GetBackupsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.EmergencyReparentShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetBackupsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this EmergencyReparentShardRequest to JSON. + * Converts this GetBackupsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for EmergencyReparentShardRequest + * Gets the default type url for GetBackupsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an EmergencyReparentShardResponse. */ - interface IEmergencyReparentShardResponse { - - /** EmergencyReparentShardResponse keyspace */ - keyspace?: (string|null); - - /** EmergencyReparentShardResponse shard */ - shard?: (string|null); - - /** EmergencyReparentShardResponse promoted_primary */ - promoted_primary?: (topodata.ITabletAlias|null); + /** Properties of a GetBackupsResponse. */ + interface IGetBackupsResponse { - /** EmergencyReparentShardResponse events */ - events?: (logutil.IEvent[]|null); + /** GetBackupsResponse backups */ + backups?: (mysqlctl.IBackupInfo[]|null); } - /** Represents an EmergencyReparentShardResponse. */ - class EmergencyReparentShardResponse implements IEmergencyReparentShardResponse { + /** Represents a GetBackupsResponse. */ + class GetBackupsResponse implements IGetBackupsResponse { /** - * Constructs a new EmergencyReparentShardResponse. + * Constructs a new GetBackupsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IEmergencyReparentShardResponse); - - /** EmergencyReparentShardResponse keyspace. */ - public keyspace: string; - - /** EmergencyReparentShardResponse shard. */ - public shard: string; - - /** EmergencyReparentShardResponse promoted_primary. */ - public promoted_primary?: (topodata.ITabletAlias|null); + constructor(properties?: vtctldata.IGetBackupsResponse); - /** EmergencyReparentShardResponse events. */ - public events: logutil.IEvent[]; + /** GetBackupsResponse backups. */ + public backups: mysqlctl.IBackupInfo[]; /** - * Creates a new EmergencyReparentShardResponse instance using the specified properties. + * Creates a new GetBackupsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns EmergencyReparentShardResponse instance + * @returns GetBackupsResponse instance */ - public static create(properties?: vtctldata.IEmergencyReparentShardResponse): vtctldata.EmergencyReparentShardResponse; + public static create(properties?: vtctldata.IGetBackupsResponse): vtctldata.GetBackupsResponse; /** - * Encodes the specified EmergencyReparentShardResponse message. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. - * @param message EmergencyReparentShardResponse message or plain object to encode + * Encodes the specified GetBackupsResponse message. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. + * @param message GetBackupsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IEmergencyReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetBackupsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified EmergencyReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. - * @param message EmergencyReparentShardResponse message or plain object to encode + * Encodes the specified GetBackupsResponse message, length delimited. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. + * @param message GetBackupsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IEmergencyReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetBackupsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer. + * Decodes a GetBackupsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns EmergencyReparentShardResponse + * @returns GetBackupsResponse * @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.EmergencyReparentShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetBackupsResponse; /** - * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer, length delimited. + * Decodes a GetBackupsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns EmergencyReparentShardResponse + * @returns GetBackupsResponse * @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.EmergencyReparentShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetBackupsResponse; /** - * Verifies an EmergencyReparentShardResponse message. + * Verifies a GetBackupsResponse 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 EmergencyReparentShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetBackupsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns EmergencyReparentShardResponse + * @returns GetBackupsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.EmergencyReparentShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetBackupsResponse; /** - * Creates a plain object from an EmergencyReparentShardResponse message. Also converts values to other types if specified. - * @param message EmergencyReparentShardResponse + * Creates a plain object from a GetBackupsResponse message. Also converts values to other types if specified. + * @param message GetBackupsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.EmergencyReparentShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetBackupsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this EmergencyReparentShardResponse to JSON. + * Converts this GetBackupsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for EmergencyReparentShardResponse + * Gets the default type url for GetBackupsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteFetchAsAppRequest. */ - interface IExecuteFetchAsAppRequest { - - /** ExecuteFetchAsAppRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteFetchAsAppRequest query */ - query?: (string|null); - - /** ExecuteFetchAsAppRequest max_rows */ - max_rows?: (number|Long|null); + /** Properties of a GetCellInfoRequest. */ + interface IGetCellInfoRequest { - /** ExecuteFetchAsAppRequest use_pool */ - use_pool?: (boolean|null); + /** GetCellInfoRequest cell */ + cell?: (string|null); } - /** Represents an ExecuteFetchAsAppRequest. */ - class ExecuteFetchAsAppRequest implements IExecuteFetchAsAppRequest { + /** Represents a GetCellInfoRequest. */ + class GetCellInfoRequest implements IGetCellInfoRequest { /** - * Constructs a new ExecuteFetchAsAppRequest. + * Constructs a new GetCellInfoRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteFetchAsAppRequest); - - /** ExecuteFetchAsAppRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteFetchAsAppRequest query. */ - public query: string; - - /** ExecuteFetchAsAppRequest max_rows. */ - public max_rows: (number|Long); + constructor(properties?: vtctldata.IGetCellInfoRequest); - /** ExecuteFetchAsAppRequest use_pool. */ - public use_pool: boolean; + /** GetCellInfoRequest cell. */ + public cell: string; /** - * Creates a new ExecuteFetchAsAppRequest instance using the specified properties. + * Creates a new GetCellInfoRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteFetchAsAppRequest instance + * @returns GetCellInfoRequest instance */ - public static create(properties?: vtctldata.IExecuteFetchAsAppRequest): vtctldata.ExecuteFetchAsAppRequest; + public static create(properties?: vtctldata.IGetCellInfoRequest): vtctldata.GetCellInfoRequest; /** - * Encodes the specified ExecuteFetchAsAppRequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. - * @param message ExecuteFetchAsAppRequest message or plain object to encode + * Encodes the specified GetCellInfoRequest message. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. + * @param message GetCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteFetchAsAppRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteFetchAsAppRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. - * @param message ExecuteFetchAsAppRequest message or plain object to encode + * Encodes the specified GetCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. + * @param message GetCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteFetchAsAppRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer. + * Decodes a GetCellInfoRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteFetchAsAppRequest + * @returns GetCellInfoRequest * @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.ExecuteFetchAsAppRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellInfoRequest; /** - * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteFetchAsAppRequest + * @returns GetCellInfoRequest * @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.ExecuteFetchAsAppRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellInfoRequest; /** - * Verifies an ExecuteFetchAsAppRequest message. + * Verifies a GetCellInfoRequest 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 ExecuteFetchAsAppRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteFetchAsAppRequest + * @returns GetCellInfoRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsAppRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoRequest; /** - * Creates a plain object from an ExecuteFetchAsAppRequest message. Also converts values to other types if specified. - * @param message ExecuteFetchAsAppRequest + * Creates a plain object from a GetCellInfoRequest message. Also converts values to other types if specified. + * @param message GetCellInfoRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteFetchAsAppRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteFetchAsAppRequest to JSON. + * Converts this GetCellInfoRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteFetchAsAppRequest + * Gets the default type url for GetCellInfoRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteFetchAsAppResponse. */ - interface IExecuteFetchAsAppResponse { + /** Properties of a GetCellInfoResponse. */ + interface IGetCellInfoResponse { - /** ExecuteFetchAsAppResponse result */ - result?: (query.IQueryResult|null); + /** GetCellInfoResponse cell_info */ + cell_info?: (topodata.ICellInfo|null); } - /** Represents an ExecuteFetchAsAppResponse. */ - class ExecuteFetchAsAppResponse implements IExecuteFetchAsAppResponse { + /** Represents a GetCellInfoResponse. */ + class GetCellInfoResponse implements IGetCellInfoResponse { /** - * Constructs a new ExecuteFetchAsAppResponse. + * Constructs a new GetCellInfoResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteFetchAsAppResponse); + constructor(properties?: vtctldata.IGetCellInfoResponse); - /** ExecuteFetchAsAppResponse result. */ - public result?: (query.IQueryResult|null); + /** GetCellInfoResponse cell_info. */ + public cell_info?: (topodata.ICellInfo|null); /** - * Creates a new ExecuteFetchAsAppResponse instance using the specified properties. + * Creates a new GetCellInfoResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteFetchAsAppResponse instance + * @returns GetCellInfoResponse instance */ - public static create(properties?: vtctldata.IExecuteFetchAsAppResponse): vtctldata.ExecuteFetchAsAppResponse; + public static create(properties?: vtctldata.IGetCellInfoResponse): vtctldata.GetCellInfoResponse; /** - * Encodes the specified ExecuteFetchAsAppResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. - * @param message ExecuteFetchAsAppResponse message or plain object to encode + * Encodes the specified GetCellInfoResponse message. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. + * @param message GetCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteFetchAsAppResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteFetchAsAppResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. - * @param message ExecuteFetchAsAppResponse message or plain object to encode + * Encodes the specified GetCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. + * @param message GetCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteFetchAsAppResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer. + * Decodes a GetCellInfoResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteFetchAsAppResponse + * @returns GetCellInfoResponse * @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.ExecuteFetchAsAppResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellInfoResponse; /** - * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteFetchAsAppResponse + * @returns GetCellInfoResponse * @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.ExecuteFetchAsAppResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellInfoResponse; /** - * Verifies an ExecuteFetchAsAppResponse message. + * Verifies a GetCellInfoResponse 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 ExecuteFetchAsAppResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteFetchAsAppResponse + * @returns GetCellInfoResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsAppResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoResponse; /** - * Creates a plain object from an ExecuteFetchAsAppResponse message. Also converts values to other types if specified. - * @param message ExecuteFetchAsAppResponse + * Creates a plain object from a GetCellInfoResponse message. Also converts values to other types if specified. + * @param message GetCellInfoResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteFetchAsAppResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteFetchAsAppResponse to JSON. + * Converts this GetCellInfoResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteFetchAsAppResponse + * Gets the default type url for GetCellInfoResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteFetchAsDBARequest. */ - interface IExecuteFetchAsDBARequest { - - /** ExecuteFetchAsDBARequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteFetchAsDBARequest query */ - query?: (string|null); - - /** ExecuteFetchAsDBARequest max_rows */ - max_rows?: (number|Long|null); - - /** ExecuteFetchAsDBARequest disable_binlogs */ - disable_binlogs?: (boolean|null); - - /** ExecuteFetchAsDBARequest reload_schema */ - reload_schema?: (boolean|null); + /** Properties of a GetCellInfoNamesRequest. */ + interface IGetCellInfoNamesRequest { } - /** Represents an ExecuteFetchAsDBARequest. */ - class ExecuteFetchAsDBARequest implements IExecuteFetchAsDBARequest { + /** Represents a GetCellInfoNamesRequest. */ + class GetCellInfoNamesRequest implements IGetCellInfoNamesRequest { /** - * Constructs a new ExecuteFetchAsDBARequest. + * Constructs a new GetCellInfoNamesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteFetchAsDBARequest); - - /** ExecuteFetchAsDBARequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteFetchAsDBARequest query. */ - public query: string; - - /** ExecuteFetchAsDBARequest max_rows. */ - public max_rows: (number|Long); - - /** ExecuteFetchAsDBARequest disable_binlogs. */ - public disable_binlogs: boolean; - - /** ExecuteFetchAsDBARequest reload_schema. */ - public reload_schema: boolean; + constructor(properties?: vtctldata.IGetCellInfoNamesRequest); /** - * Creates a new ExecuteFetchAsDBARequest instance using the specified properties. + * Creates a new GetCellInfoNamesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteFetchAsDBARequest instance + * @returns GetCellInfoNamesRequest instance */ - public static create(properties?: vtctldata.IExecuteFetchAsDBARequest): vtctldata.ExecuteFetchAsDBARequest; + public static create(properties?: vtctldata.IGetCellInfoNamesRequest): vtctldata.GetCellInfoNamesRequest; /** - * Encodes the specified ExecuteFetchAsDBARequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. - * @param message ExecuteFetchAsDBARequest message or plain object to encode + * Encodes the specified GetCellInfoNamesRequest message. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. + * @param message GetCellInfoNamesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteFetchAsDBARequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellInfoNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteFetchAsDBARequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. - * @param message ExecuteFetchAsDBARequest message or plain object to encode + * Encodes the specified GetCellInfoNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. + * @param message GetCellInfoNamesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteFetchAsDBARequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellInfoNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer. + * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteFetchAsDBARequest + * @returns GetCellInfoNamesRequest * @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.ExecuteFetchAsDBARequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellInfoNamesRequest; /** - * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteFetchAsDBARequest + * @returns GetCellInfoNamesRequest * @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.ExecuteFetchAsDBARequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellInfoNamesRequest; /** - * Verifies an ExecuteFetchAsDBARequest message. + * Verifies a GetCellInfoNamesRequest 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 ExecuteFetchAsDBARequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoNamesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteFetchAsDBARequest + * @returns GetCellInfoNamesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsDBARequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoNamesRequest; /** - * Creates a plain object from an ExecuteFetchAsDBARequest message. Also converts values to other types if specified. - * @param message ExecuteFetchAsDBARequest + * Creates a plain object from a GetCellInfoNamesRequest message. Also converts values to other types if specified. + * @param message GetCellInfoNamesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteFetchAsDBARequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellInfoNamesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteFetchAsDBARequest to JSON. + * Converts this GetCellInfoNamesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteFetchAsDBARequest + * Gets the default type url for GetCellInfoNamesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteFetchAsDBAResponse. */ - interface IExecuteFetchAsDBAResponse { + /** Properties of a GetCellInfoNamesResponse. */ + interface IGetCellInfoNamesResponse { - /** ExecuteFetchAsDBAResponse result */ - result?: (query.IQueryResult|null); + /** GetCellInfoNamesResponse names */ + names?: (string[]|null); } - /** Represents an ExecuteFetchAsDBAResponse. */ - class ExecuteFetchAsDBAResponse implements IExecuteFetchAsDBAResponse { + /** Represents a GetCellInfoNamesResponse. */ + class GetCellInfoNamesResponse implements IGetCellInfoNamesResponse { /** - * Constructs a new ExecuteFetchAsDBAResponse. + * Constructs a new GetCellInfoNamesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteFetchAsDBAResponse); + constructor(properties?: vtctldata.IGetCellInfoNamesResponse); - /** ExecuteFetchAsDBAResponse result. */ - public result?: (query.IQueryResult|null); + /** GetCellInfoNamesResponse names. */ + public names: string[]; /** - * Creates a new ExecuteFetchAsDBAResponse instance using the specified properties. + * Creates a new GetCellInfoNamesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteFetchAsDBAResponse instance + * @returns GetCellInfoNamesResponse instance */ - public static create(properties?: vtctldata.IExecuteFetchAsDBAResponse): vtctldata.ExecuteFetchAsDBAResponse; + public static create(properties?: vtctldata.IGetCellInfoNamesResponse): vtctldata.GetCellInfoNamesResponse; /** - * Encodes the specified ExecuteFetchAsDBAResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. - * @param message ExecuteFetchAsDBAResponse message or plain object to encode + * Encodes the specified GetCellInfoNamesResponse message. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. + * @param message GetCellInfoNamesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteFetchAsDBAResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellInfoNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteFetchAsDBAResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. - * @param message ExecuteFetchAsDBAResponse message or plain object to encode + * Encodes the specified GetCellInfoNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. + * @param message GetCellInfoNamesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteFetchAsDBAResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellInfoNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer. + * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteFetchAsDBAResponse + * @returns GetCellInfoNamesResponse * @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.ExecuteFetchAsDBAResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellInfoNamesResponse; /** - * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteFetchAsDBAResponse + * @returns GetCellInfoNamesResponse * @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.ExecuteFetchAsDBAResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellInfoNamesResponse; /** - * Verifies an ExecuteFetchAsDBAResponse message. + * Verifies a GetCellInfoNamesResponse 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 ExecuteFetchAsDBAResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoNamesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteFetchAsDBAResponse + * @returns GetCellInfoNamesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteFetchAsDBAResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoNamesResponse; /** - * Creates a plain object from an ExecuteFetchAsDBAResponse message. Also converts values to other types if specified. - * @param message ExecuteFetchAsDBAResponse + * Creates a plain object from a GetCellInfoNamesResponse message. Also converts values to other types if specified. + * @param message GetCellInfoNamesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteFetchAsDBAResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellInfoNamesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteFetchAsDBAResponse to JSON. + * Converts this GetCellInfoNamesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteFetchAsDBAResponse + * Gets the default type url for GetCellInfoNamesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteHookRequest. */ - interface IExecuteHookRequest { - - /** ExecuteHookRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteHookRequest tablet_hook_request */ - tablet_hook_request?: (tabletmanagerdata.IExecuteHookRequest|null); + /** Properties of a GetCellsAliasesRequest. */ + interface IGetCellsAliasesRequest { } - /** Represents an ExecuteHookRequest. */ - class ExecuteHookRequest implements IExecuteHookRequest { + /** Represents a GetCellsAliasesRequest. */ + class GetCellsAliasesRequest implements IGetCellsAliasesRequest { /** - * Constructs a new ExecuteHookRequest. + * Constructs a new GetCellsAliasesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteHookRequest); - - /** ExecuteHookRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** ExecuteHookRequest tablet_hook_request. */ - public tablet_hook_request?: (tabletmanagerdata.IExecuteHookRequest|null); + constructor(properties?: vtctldata.IGetCellsAliasesRequest); /** - * Creates a new ExecuteHookRequest instance using the specified properties. + * Creates a new GetCellsAliasesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteHookRequest instance + * @returns GetCellsAliasesRequest instance */ - public static create(properties?: vtctldata.IExecuteHookRequest): vtctldata.ExecuteHookRequest; + public static create(properties?: vtctldata.IGetCellsAliasesRequest): vtctldata.GetCellsAliasesRequest; /** - * Encodes the specified ExecuteHookRequest message. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. - * @param message ExecuteHookRequest message or plain object to encode + * Encodes the specified GetCellsAliasesRequest message. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. + * @param message GetCellsAliasesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteHookRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellsAliasesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteHookRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. - * @param message ExecuteHookRequest message or plain object to encode + * Encodes the specified GetCellsAliasesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. + * @param message GetCellsAliasesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteHookRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellsAliasesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteHookRequest message from the specified reader or buffer. + * Decodes a GetCellsAliasesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteHookRequest + * @returns GetCellsAliasesRequest * @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.ExecuteHookRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellsAliasesRequest; /** - * Decodes an ExecuteHookRequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellsAliasesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteHookRequest + * @returns GetCellsAliasesRequest * @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.ExecuteHookRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellsAliasesRequest; /** - * Verifies an ExecuteHookRequest message. + * Verifies a GetCellsAliasesRequest 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 ExecuteHookRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellsAliasesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteHookRequest + * @returns GetCellsAliasesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteHookRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellsAliasesRequest; /** - * Creates a plain object from an ExecuteHookRequest message. Also converts values to other types if specified. - * @param message ExecuteHookRequest + * Creates a plain object from a GetCellsAliasesRequest message. Also converts values to other types if specified. + * @param message GetCellsAliasesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteHookRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellsAliasesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteHookRequest to JSON. + * Converts this GetCellsAliasesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteHookRequest + * Gets the default type url for GetCellsAliasesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an ExecuteHookResponse. */ - interface IExecuteHookResponse { + /** Properties of a GetCellsAliasesResponse. */ + interface IGetCellsAliasesResponse { - /** ExecuteHookResponse hook_result */ - hook_result?: (tabletmanagerdata.IExecuteHookResponse|null); + /** GetCellsAliasesResponse aliases */ + aliases?: ({ [k: string]: topodata.ICellsAlias }|null); } - /** Represents an ExecuteHookResponse. */ - class ExecuteHookResponse implements IExecuteHookResponse { + /** Represents a GetCellsAliasesResponse. */ + class GetCellsAliasesResponse implements IGetCellsAliasesResponse { /** - * Constructs a new ExecuteHookResponse. + * Constructs a new GetCellsAliasesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IExecuteHookResponse); + constructor(properties?: vtctldata.IGetCellsAliasesResponse); - /** ExecuteHookResponse hook_result. */ - public hook_result?: (tabletmanagerdata.IExecuteHookResponse|null); + /** GetCellsAliasesResponse aliases. */ + public aliases: { [k: string]: topodata.ICellsAlias }; /** - * Creates a new ExecuteHookResponse instance using the specified properties. + * Creates a new GetCellsAliasesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ExecuteHookResponse instance + * @returns GetCellsAliasesResponse instance */ - public static create(properties?: vtctldata.IExecuteHookResponse): vtctldata.ExecuteHookResponse; + public static create(properties?: vtctldata.IGetCellsAliasesResponse): vtctldata.GetCellsAliasesResponse; /** - * Encodes the specified ExecuteHookResponse message. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. - * @param message ExecuteHookResponse message or plain object to encode + * Encodes the specified GetCellsAliasesResponse message. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. + * @param message GetCellsAliasesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IExecuteHookResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetCellsAliasesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ExecuteHookResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. - * @param message ExecuteHookResponse message or plain object to encode + * Encodes the specified GetCellsAliasesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. + * @param message GetCellsAliasesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IExecuteHookResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetCellsAliasesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an ExecuteHookResponse message from the specified reader or buffer. + * Decodes a GetCellsAliasesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ExecuteHookResponse + * @returns GetCellsAliasesResponse * @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.ExecuteHookResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetCellsAliasesResponse; /** - * Decodes an ExecuteHookResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellsAliasesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ExecuteHookResponse + * @returns GetCellsAliasesResponse * @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.ExecuteHookResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetCellsAliasesResponse; /** - * Verifies an ExecuteHookResponse message. + * Verifies a GetCellsAliasesResponse 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 ExecuteHookResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellsAliasesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ExecuteHookResponse + * @returns GetCellsAliasesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ExecuteHookResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetCellsAliasesResponse; /** - * Creates a plain object from an ExecuteHookResponse message. Also converts values to other types if specified. - * @param message ExecuteHookResponse + * Creates a plain object from a GetCellsAliasesResponse message. Also converts values to other types if specified. + * @param message GetCellsAliasesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ExecuteHookResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetCellsAliasesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ExecuteHookResponse to JSON. + * Converts this GetCellsAliasesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ExecuteHookResponse + * Gets the default type url for GetCellsAliasesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a FindAllShardsInKeyspaceRequest. */ - interface IFindAllShardsInKeyspaceRequest { + /** Properties of a GetFullStatusRequest. */ + interface IGetFullStatusRequest { - /** FindAllShardsInKeyspaceRequest keyspace */ - keyspace?: (string|null); + /** GetFullStatusRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a FindAllShardsInKeyspaceRequest. */ - class FindAllShardsInKeyspaceRequest implements IFindAllShardsInKeyspaceRequest { + /** Represents a GetFullStatusRequest. */ + class GetFullStatusRequest implements IGetFullStatusRequest { /** - * Constructs a new FindAllShardsInKeyspaceRequest. + * Constructs a new GetFullStatusRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IFindAllShardsInKeyspaceRequest); + constructor(properties?: vtctldata.IGetFullStatusRequest); - /** FindAllShardsInKeyspaceRequest keyspace. */ - public keyspace: string; + /** GetFullStatusRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new FindAllShardsInKeyspaceRequest instance using the specified properties. + * Creates a new GetFullStatusRequest instance using the specified properties. * @param [properties] Properties to set - * @returns FindAllShardsInKeyspaceRequest instance + * @returns GetFullStatusRequest instance */ - public static create(properties?: vtctldata.IFindAllShardsInKeyspaceRequest): vtctldata.FindAllShardsInKeyspaceRequest; + public static create(properties?: vtctldata.IGetFullStatusRequest): vtctldata.GetFullStatusRequest; /** - * Encodes the specified FindAllShardsInKeyspaceRequest message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. - * @param message FindAllShardsInKeyspaceRequest message or plain object to encode + * Encodes the specified GetFullStatusRequest message. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. + * @param message GetFullStatusRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IFindAllShardsInKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetFullStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified FindAllShardsInKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. - * @param message FindAllShardsInKeyspaceRequest message or plain object to encode + * Encodes the specified GetFullStatusRequest message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. + * @param message GetFullStatusRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IFindAllShardsInKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetFullStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer. + * Decodes a GetFullStatusRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns FindAllShardsInKeyspaceRequest + * @returns GetFullStatusRequest * @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.FindAllShardsInKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetFullStatusRequest; /** - * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a GetFullStatusRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns FindAllShardsInKeyspaceRequest + * @returns GetFullStatusRequest * @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.FindAllShardsInKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetFullStatusRequest; /** - * Verifies a FindAllShardsInKeyspaceRequest message. + * Verifies a GetFullStatusRequest 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 FindAllShardsInKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetFullStatusRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns FindAllShardsInKeyspaceRequest + * @returns GetFullStatusRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.FindAllShardsInKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetFullStatusRequest; /** - * Creates a plain object from a FindAllShardsInKeyspaceRequest message. Also converts values to other types if specified. - * @param message FindAllShardsInKeyspaceRequest + * Creates a plain object from a GetFullStatusRequest message. Also converts values to other types if specified. + * @param message GetFullStatusRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.FindAllShardsInKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetFullStatusRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this FindAllShardsInKeyspaceRequest to JSON. + * Converts this GetFullStatusRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for FindAllShardsInKeyspaceRequest + * Gets the default type url for GetFullStatusRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a FindAllShardsInKeyspaceResponse. */ - interface IFindAllShardsInKeyspaceResponse { + /** Properties of a GetFullStatusResponse. */ + interface IGetFullStatusResponse { - /** FindAllShardsInKeyspaceResponse shards */ - shards?: ({ [k: string]: vtctldata.IShard }|null); + /** GetFullStatusResponse status */ + status?: (replicationdata.IFullStatus|null); } - /** Represents a FindAllShardsInKeyspaceResponse. */ - class FindAllShardsInKeyspaceResponse implements IFindAllShardsInKeyspaceResponse { + /** Represents a GetFullStatusResponse. */ + class GetFullStatusResponse implements IGetFullStatusResponse { /** - * Constructs a new FindAllShardsInKeyspaceResponse. + * Constructs a new GetFullStatusResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IFindAllShardsInKeyspaceResponse); + constructor(properties?: vtctldata.IGetFullStatusResponse); - /** FindAllShardsInKeyspaceResponse shards. */ - public shards: { [k: string]: vtctldata.IShard }; + /** GetFullStatusResponse status. */ + public status?: (replicationdata.IFullStatus|null); /** - * Creates a new FindAllShardsInKeyspaceResponse instance using the specified properties. + * Creates a new GetFullStatusResponse instance using the specified properties. * @param [properties] Properties to set - * @returns FindAllShardsInKeyspaceResponse instance + * @returns GetFullStatusResponse instance */ - public static create(properties?: vtctldata.IFindAllShardsInKeyspaceResponse): vtctldata.FindAllShardsInKeyspaceResponse; + public static create(properties?: vtctldata.IGetFullStatusResponse): vtctldata.GetFullStatusResponse; /** - * Encodes the specified FindAllShardsInKeyspaceResponse message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. - * @param message FindAllShardsInKeyspaceResponse message or plain object to encode + * Encodes the specified GetFullStatusResponse message. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. + * @param message GetFullStatusResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IFindAllShardsInKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetFullStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified FindAllShardsInKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. - * @param message FindAllShardsInKeyspaceResponse message or plain object to encode + * Encodes the specified GetFullStatusResponse message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. + * @param message GetFullStatusResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IFindAllShardsInKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetFullStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer. + * Decodes a GetFullStatusResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns FindAllShardsInKeyspaceResponse + * @returns GetFullStatusResponse * @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.FindAllShardsInKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetFullStatusResponse; /** - * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a GetFullStatusResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns FindAllShardsInKeyspaceResponse + * @returns GetFullStatusResponse * @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.FindAllShardsInKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetFullStatusResponse; /** - * Verifies a FindAllShardsInKeyspaceResponse message. + * Verifies a GetFullStatusResponse 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 FindAllShardsInKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetFullStatusResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns FindAllShardsInKeyspaceResponse + * @returns GetFullStatusResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.FindAllShardsInKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetFullStatusResponse; /** - * Creates a plain object from a FindAllShardsInKeyspaceResponse message. Also converts values to other types if specified. - * @param message FindAllShardsInKeyspaceResponse + * Creates a plain object from a GetFullStatusResponse message. Also converts values to other types if specified. + * @param message GetFullStatusResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.FindAllShardsInKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetFullStatusResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this FindAllShardsInKeyspaceResponse to JSON. + * Converts this GetFullStatusResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for FindAllShardsInKeyspaceResponse + * Gets the default type url for GetFullStatusResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetBackupsRequest. */ - interface IGetBackupsRequest { - - /** GetBackupsRequest keyspace */ - keyspace?: (string|null); - - /** GetBackupsRequest shard */ - shard?: (string|null); - - /** GetBackupsRequest limit */ - limit?: (number|null); - - /** GetBackupsRequest detailed */ - detailed?: (boolean|null); - - /** GetBackupsRequest detailed_limit */ - detailed_limit?: (number|null); + /** Properties of a GetKeyspacesRequest. */ + interface IGetKeyspacesRequest { } - /** Represents a GetBackupsRequest. */ - class GetBackupsRequest implements IGetBackupsRequest { + /** Represents a GetKeyspacesRequest. */ + class GetKeyspacesRequest implements IGetKeyspacesRequest { /** - * Constructs a new GetBackupsRequest. + * Constructs a new GetKeyspacesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetBackupsRequest); - - /** GetBackupsRequest keyspace. */ - public keyspace: string; - - /** GetBackupsRequest shard. */ - public shard: string; - - /** GetBackupsRequest limit. */ - public limit: number; - - /** GetBackupsRequest detailed. */ - public detailed: boolean; - - /** GetBackupsRequest detailed_limit. */ - public detailed_limit: number; + constructor(properties?: vtctldata.IGetKeyspacesRequest); /** - * Creates a new GetBackupsRequest instance using the specified properties. + * Creates a new GetKeyspacesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetBackupsRequest instance + * @returns GetKeyspacesRequest instance */ - public static create(properties?: vtctldata.IGetBackupsRequest): vtctldata.GetBackupsRequest; + public static create(properties?: vtctldata.IGetKeyspacesRequest): vtctldata.GetKeyspacesRequest; /** - * Encodes the specified GetBackupsRequest message. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. - * @param message GetBackupsRequest message or plain object to encode + * Encodes the specified GetKeyspacesRequest message. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. + * @param message GetKeyspacesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetBackupsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetBackupsRequest message, length delimited. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. - * @param message GetBackupsRequest message or plain object to encode + * Encodes the specified GetKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. + * @param message GetKeyspacesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetBackupsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetBackupsRequest message from the specified reader or buffer. + * Decodes a GetKeyspacesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetBackupsRequest + * @returns GetKeyspacesRequest * @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.GetBackupsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetKeyspacesRequest; /** - * Decodes a GetBackupsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspacesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetBackupsRequest + * @returns GetKeyspacesRequest * @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.GetBackupsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetKeyspacesRequest; /** - * Verifies a GetBackupsRequest message. + * Verifies a GetKeyspacesRequest 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 GetBackupsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspacesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetBackupsRequest + * @returns GetKeyspacesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetBackupsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspacesRequest; /** - * Creates a plain object from a GetBackupsRequest message. Also converts values to other types if specified. - * @param message GetBackupsRequest + * Creates a plain object from a GetKeyspacesRequest message. Also converts values to other types if specified. + * @param message GetKeyspacesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetBackupsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetKeyspacesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetBackupsRequest to JSON. + * Converts this GetKeyspacesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetBackupsRequest + * Gets the default type url for GetKeyspacesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetBackupsResponse. */ - interface IGetBackupsResponse { + /** Properties of a GetKeyspacesResponse. */ + interface IGetKeyspacesResponse { - /** GetBackupsResponse backups */ - backups?: (mysqlctl.IBackupInfo[]|null); + /** GetKeyspacesResponse keyspaces */ + keyspaces?: (vtctldata.IKeyspace[]|null); } - /** Represents a GetBackupsResponse. */ - class GetBackupsResponse implements IGetBackupsResponse { + /** Represents a GetKeyspacesResponse. */ + class GetKeyspacesResponse implements IGetKeyspacesResponse { /** - * Constructs a new GetBackupsResponse. + * Constructs a new GetKeyspacesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetBackupsResponse); + constructor(properties?: vtctldata.IGetKeyspacesResponse); - /** GetBackupsResponse backups. */ - public backups: mysqlctl.IBackupInfo[]; + /** GetKeyspacesResponse keyspaces. */ + public keyspaces: vtctldata.IKeyspace[]; /** - * Creates a new GetBackupsResponse instance using the specified properties. + * Creates a new GetKeyspacesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetBackupsResponse instance + * @returns GetKeyspacesResponse instance */ - public static create(properties?: vtctldata.IGetBackupsResponse): vtctldata.GetBackupsResponse; + public static create(properties?: vtctldata.IGetKeyspacesResponse): vtctldata.GetKeyspacesResponse; /** - * Encodes the specified GetBackupsResponse message. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. - * @param message GetBackupsResponse message or plain object to encode + * Encodes the specified GetKeyspacesResponse message. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. + * @param message GetKeyspacesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetBackupsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetBackupsResponse message, length delimited. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. - * @param message GetBackupsResponse message or plain object to encode + * Encodes the specified GetKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. + * @param message GetKeyspacesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetBackupsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetBackupsResponse message from the specified reader or buffer. + * Decodes a GetKeyspacesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetBackupsResponse + * @returns GetKeyspacesResponse * @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.GetBackupsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetKeyspacesResponse; /** - * Decodes a GetBackupsResponse message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspacesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetBackupsResponse + * @returns GetKeyspacesResponse * @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.GetBackupsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetKeyspacesResponse; /** - * Verifies a GetBackupsResponse message. + * Verifies a GetKeyspacesResponse 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 GetBackupsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspacesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetBackupsResponse + * @returns GetKeyspacesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetBackupsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspacesResponse; /** - * Creates a plain object from a GetBackupsResponse message. Also converts values to other types if specified. - * @param message GetBackupsResponse + * Creates a plain object from a GetKeyspacesResponse message. Also converts values to other types if specified. + * @param message GetKeyspacesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetBackupsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetKeyspacesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetBackupsResponse to JSON. + * Converts this GetKeyspacesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetBackupsResponse + * Gets the default type url for GetKeyspacesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellInfoRequest. */ - interface IGetCellInfoRequest { + /** Properties of a GetKeyspaceRequest. */ + interface IGetKeyspaceRequest { - /** GetCellInfoRequest cell */ - cell?: (string|null); + /** GetKeyspaceRequest keyspace */ + keyspace?: (string|null); } - /** Represents a GetCellInfoRequest. */ - class GetCellInfoRequest implements IGetCellInfoRequest { + /** Represents a GetKeyspaceRequest. */ + class GetKeyspaceRequest implements IGetKeyspaceRequest { /** - * Constructs a new GetCellInfoRequest. + * Constructs a new GetKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellInfoRequest); + constructor(properties?: vtctldata.IGetKeyspaceRequest); - /** GetCellInfoRequest cell. */ - public cell: string; + /** GetKeyspaceRequest keyspace. */ + public keyspace: string; /** - * Creates a new GetCellInfoRequest instance using the specified properties. + * Creates a new GetKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellInfoRequest instance + * @returns GetKeyspaceRequest instance */ - public static create(properties?: vtctldata.IGetCellInfoRequest): vtctldata.GetCellInfoRequest; + public static create(properties?: vtctldata.IGetKeyspaceRequest): vtctldata.GetKeyspaceRequest; /** - * Encodes the specified GetCellInfoRequest message. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. - * @param message GetCellInfoRequest message or plain object to encode + * Encodes the specified GetKeyspaceRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. + * @param message GetKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. - * @param message GetCellInfoRequest message or plain object to encode + * Encodes the specified GetKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. + * @param message GetKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellInfoRequest message from the specified reader or buffer. + * Decodes a GetKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellInfoRequest + * @returns GetKeyspaceRequest * @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.GetCellInfoRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetKeyspaceRequest; /** - * Decodes a GetCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellInfoRequest + * @returns GetKeyspaceRequest * @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.GetCellInfoRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetKeyspaceRequest; /** - * Verifies a GetCellInfoRequest message. + * Verifies a GetKeyspaceRequest 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 GetCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellInfoRequest + * @returns GetKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceRequest; /** - * Creates a plain object from a GetCellInfoRequest message. Also converts values to other types if specified. - * @param message GetCellInfoRequest + * Creates a plain object from a GetKeyspaceRequest message. Also converts values to other types if specified. + * @param message GetKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellInfoRequest to JSON. + * Converts this GetKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellInfoRequest + * Gets the default type url for GetKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellInfoResponse. */ - interface IGetCellInfoResponse { + /** Properties of a GetKeyspaceResponse. */ + interface IGetKeyspaceResponse { - /** GetCellInfoResponse cell_info */ - cell_info?: (topodata.ICellInfo|null); + /** GetKeyspaceResponse keyspace */ + keyspace?: (vtctldata.IKeyspace|null); } - /** Represents a GetCellInfoResponse. */ - class GetCellInfoResponse implements IGetCellInfoResponse { + /** Represents a GetKeyspaceResponse. */ + class GetKeyspaceResponse implements IGetKeyspaceResponse { /** - * Constructs a new GetCellInfoResponse. + * Constructs a new GetKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellInfoResponse); + constructor(properties?: vtctldata.IGetKeyspaceResponse); - /** GetCellInfoResponse cell_info. */ - public cell_info?: (topodata.ICellInfo|null); + /** GetKeyspaceResponse keyspace. */ + public keyspace?: (vtctldata.IKeyspace|null); /** - * Creates a new GetCellInfoResponse instance using the specified properties. + * Creates a new GetKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellInfoResponse instance + * @returns GetKeyspaceResponse instance */ - public static create(properties?: vtctldata.IGetCellInfoResponse): vtctldata.GetCellInfoResponse; + public static create(properties?: vtctldata.IGetKeyspaceResponse): vtctldata.GetKeyspaceResponse; /** - * Encodes the specified GetCellInfoResponse message. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. - * @param message GetCellInfoResponse message or plain object to encode + * Encodes the specified GetKeyspaceResponse message. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. + * @param message GetKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. - * @param message GetCellInfoResponse message or plain object to encode + * Encodes the specified GetKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. + * @param message GetKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellInfoResponse message from the specified reader or buffer. + * Decodes a GetKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellInfoResponse + * @returns GetKeyspaceResponse * @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.GetCellInfoResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetKeyspaceResponse; /** - * Decodes a GetCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellInfoResponse + * @returns GetKeyspaceResponse * @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.GetCellInfoResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetKeyspaceResponse; /** - * Verifies a GetCellInfoResponse message. + * Verifies a GetKeyspaceResponse 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 GetCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellInfoResponse + * @returns GetKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceResponse; /** - * Creates a plain object from a GetCellInfoResponse message. Also converts values to other types if specified. - * @param message GetCellInfoResponse + * Creates a plain object from a GetKeyspaceResponse message. Also converts values to other types if specified. + * @param message GetKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellInfoResponse to JSON. + * Converts this GetKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellInfoResponse + * Gets the default type url for GetKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellInfoNamesRequest. */ - interface IGetCellInfoNamesRequest { + /** Properties of a GetPermissionsRequest. */ + interface IGetPermissionsRequest { + + /** GetPermissionsRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a GetCellInfoNamesRequest. */ - class GetCellInfoNamesRequest implements IGetCellInfoNamesRequest { + /** Represents a GetPermissionsRequest. */ + class GetPermissionsRequest implements IGetPermissionsRequest { /** - * Constructs a new GetCellInfoNamesRequest. + * Constructs a new GetPermissionsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellInfoNamesRequest); + constructor(properties?: vtctldata.IGetPermissionsRequest); + + /** GetPermissionsRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new GetCellInfoNamesRequest instance using the specified properties. + * Creates a new GetPermissionsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellInfoNamesRequest instance + * @returns GetPermissionsRequest instance */ - public static create(properties?: vtctldata.IGetCellInfoNamesRequest): vtctldata.GetCellInfoNamesRequest; + public static create(properties?: vtctldata.IGetPermissionsRequest): vtctldata.GetPermissionsRequest; /** - * Encodes the specified GetCellInfoNamesRequest message. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. - * @param message GetCellInfoNamesRequest message or plain object to encode + * Encodes the specified GetPermissionsRequest message. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. + * @param message GetPermissionsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellInfoNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetPermissionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellInfoNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. - * @param message GetCellInfoNamesRequest message or plain object to encode + * Encodes the specified GetPermissionsRequest message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. + * @param message GetPermissionsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellInfoNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetPermissionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer. + * Decodes a GetPermissionsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellInfoNamesRequest + * @returns GetPermissionsRequest * @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.GetCellInfoNamesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetPermissionsRequest; /** - * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetPermissionsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellInfoNamesRequest + * @returns GetPermissionsRequest * @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.GetCellInfoNamesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetPermissionsRequest; /** - * Verifies a GetCellInfoNamesRequest message. + * Verifies a GetPermissionsRequest 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 GetCellInfoNamesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetPermissionsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellInfoNamesRequest + * @returns GetPermissionsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoNamesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetPermissionsRequest; /** - * Creates a plain object from a GetCellInfoNamesRequest message. Also converts values to other types if specified. - * @param message GetCellInfoNamesRequest + * Creates a plain object from a GetPermissionsRequest message. Also converts values to other types if specified. + * @param message GetPermissionsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellInfoNamesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetPermissionsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellInfoNamesRequest to JSON. + * Converts this GetPermissionsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellInfoNamesRequest + * Gets the default type url for GetPermissionsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellInfoNamesResponse. */ - interface IGetCellInfoNamesResponse { + /** Properties of a GetPermissionsResponse. */ + interface IGetPermissionsResponse { - /** GetCellInfoNamesResponse names */ - names?: (string[]|null); + /** GetPermissionsResponse permissions */ + permissions?: (tabletmanagerdata.IPermissions|null); } - /** Represents a GetCellInfoNamesResponse. */ - class GetCellInfoNamesResponse implements IGetCellInfoNamesResponse { + /** Represents a GetPermissionsResponse. */ + class GetPermissionsResponse implements IGetPermissionsResponse { /** - * Constructs a new GetCellInfoNamesResponse. + * Constructs a new GetPermissionsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellInfoNamesResponse); + constructor(properties?: vtctldata.IGetPermissionsResponse); - /** GetCellInfoNamesResponse names. */ - public names: string[]; + /** GetPermissionsResponse permissions. */ + public permissions?: (tabletmanagerdata.IPermissions|null); /** - * Creates a new GetCellInfoNamesResponse instance using the specified properties. + * Creates a new GetPermissionsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellInfoNamesResponse instance + * @returns GetPermissionsResponse instance */ - public static create(properties?: vtctldata.IGetCellInfoNamesResponse): vtctldata.GetCellInfoNamesResponse; + public static create(properties?: vtctldata.IGetPermissionsResponse): vtctldata.GetPermissionsResponse; /** - * Encodes the specified GetCellInfoNamesResponse message. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. - * @param message GetCellInfoNamesResponse message or plain object to encode + * Encodes the specified GetPermissionsResponse message. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. + * @param message GetPermissionsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellInfoNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetPermissionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellInfoNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. - * @param message GetCellInfoNamesResponse message or plain object to encode + * Encodes the specified GetPermissionsResponse message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. + * @param message GetPermissionsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellInfoNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetPermissionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer. + * Decodes a GetPermissionsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellInfoNamesResponse + * @returns GetPermissionsResponse * @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.GetCellInfoNamesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetPermissionsResponse; /** - * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetPermissionsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellInfoNamesResponse + * @returns GetPermissionsResponse * @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.GetCellInfoNamesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetPermissionsResponse; /** - * Verifies a GetCellInfoNamesResponse message. + * Verifies a GetPermissionsResponse 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 GetCellInfoNamesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetPermissionsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellInfoNamesResponse + * @returns GetPermissionsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellInfoNamesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetPermissionsResponse; /** - * Creates a plain object from a GetCellInfoNamesResponse message. Also converts values to other types if specified. - * @param message GetCellInfoNamesResponse + * Creates a plain object from a GetPermissionsResponse message. Also converts values to other types if specified. + * @param message GetPermissionsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellInfoNamesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetPermissionsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellInfoNamesResponse to JSON. + * Converts this GetPermissionsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellInfoNamesResponse + * Gets the default type url for GetPermissionsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellsAliasesRequest. */ - interface IGetCellsAliasesRequest { + /** Properties of a GetRoutingRulesRequest. */ + interface IGetRoutingRulesRequest { } - /** Represents a GetCellsAliasesRequest. */ - class GetCellsAliasesRequest implements IGetCellsAliasesRequest { + /** Represents a GetRoutingRulesRequest. */ + class GetRoutingRulesRequest implements IGetRoutingRulesRequest { /** - * Constructs a new GetCellsAliasesRequest. + * Constructs a new GetRoutingRulesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellsAliasesRequest); + constructor(properties?: vtctldata.IGetRoutingRulesRequest); /** - * Creates a new GetCellsAliasesRequest instance using the specified properties. + * Creates a new GetRoutingRulesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellsAliasesRequest instance + * @returns GetRoutingRulesRequest instance */ - public static create(properties?: vtctldata.IGetCellsAliasesRequest): vtctldata.GetCellsAliasesRequest; + public static create(properties?: vtctldata.IGetRoutingRulesRequest): vtctldata.GetRoutingRulesRequest; /** - * Encodes the specified GetCellsAliasesRequest message. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. - * @param message GetCellsAliasesRequest message or plain object to encode + * Encodes the specified GetRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. + * @param message GetRoutingRulesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellsAliasesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellsAliasesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. - * @param message GetCellsAliasesRequest message or plain object to encode + * Encodes the specified GetRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. + * @param message GetRoutingRulesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellsAliasesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellsAliasesRequest message from the specified reader or buffer. + * Decodes a GetRoutingRulesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellsAliasesRequest + * @returns GetRoutingRulesRequest * @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.GetCellsAliasesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetRoutingRulesRequest; /** - * Decodes a GetCellsAliasesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetRoutingRulesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellsAliasesRequest + * @returns GetRoutingRulesRequest * @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.GetCellsAliasesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetRoutingRulesRequest; /** - * Verifies a GetCellsAliasesRequest message. + * Verifies a GetRoutingRulesRequest 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 GetCellsAliasesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellsAliasesRequest + * @returns GetRoutingRulesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellsAliasesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetRoutingRulesRequest; /** - * Creates a plain object from a GetCellsAliasesRequest message. Also converts values to other types if specified. - * @param message GetCellsAliasesRequest + * Creates a plain object from a GetRoutingRulesRequest message. Also converts values to other types if specified. + * @param message GetRoutingRulesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellsAliasesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellsAliasesRequest to JSON. + * Converts this GetRoutingRulesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellsAliasesRequest + * Gets the default type url for GetRoutingRulesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetCellsAliasesResponse. */ - interface IGetCellsAliasesResponse { + /** Properties of a GetRoutingRulesResponse. */ + interface IGetRoutingRulesResponse { - /** GetCellsAliasesResponse aliases */ - aliases?: ({ [k: string]: topodata.ICellsAlias }|null); + /** GetRoutingRulesResponse routing_rules */ + routing_rules?: (vschema.IRoutingRules|null); } - /** Represents a GetCellsAliasesResponse. */ - class GetCellsAliasesResponse implements IGetCellsAliasesResponse { + /** Represents a GetRoutingRulesResponse. */ + class GetRoutingRulesResponse implements IGetRoutingRulesResponse { /** - * Constructs a new GetCellsAliasesResponse. + * Constructs a new GetRoutingRulesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetCellsAliasesResponse); + constructor(properties?: vtctldata.IGetRoutingRulesResponse); - /** GetCellsAliasesResponse aliases. */ - public aliases: { [k: string]: topodata.ICellsAlias }; + /** GetRoutingRulesResponse routing_rules. */ + public routing_rules?: (vschema.IRoutingRules|null); /** - * Creates a new GetCellsAliasesResponse instance using the specified properties. + * Creates a new GetRoutingRulesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetCellsAliasesResponse instance + * @returns GetRoutingRulesResponse instance */ - public static create(properties?: vtctldata.IGetCellsAliasesResponse): vtctldata.GetCellsAliasesResponse; + public static create(properties?: vtctldata.IGetRoutingRulesResponse): vtctldata.GetRoutingRulesResponse; /** - * Encodes the specified GetCellsAliasesResponse message. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. - * @param message GetCellsAliasesResponse message or plain object to encode + * Encodes the specified GetRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. + * @param message GetRoutingRulesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetCellsAliasesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetCellsAliasesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. - * @param message GetCellsAliasesResponse message or plain object to encode + * Encodes the specified GetRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. + * @param message GetRoutingRulesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetCellsAliasesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetCellsAliasesResponse message from the specified reader or buffer. + * Decodes a GetRoutingRulesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetCellsAliasesResponse + * @returns GetRoutingRulesResponse * @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.GetCellsAliasesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetRoutingRulesResponse; /** - * Decodes a GetCellsAliasesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetRoutingRulesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetCellsAliasesResponse + * @returns GetRoutingRulesResponse * @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.GetCellsAliasesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetRoutingRulesResponse; /** - * Verifies a GetCellsAliasesResponse message. + * Verifies a GetRoutingRulesResponse 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 GetCellsAliasesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetCellsAliasesResponse + * @returns GetRoutingRulesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetCellsAliasesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetRoutingRulesResponse; /** - * Creates a plain object from a GetCellsAliasesResponse message. Also converts values to other types if specified. - * @param message GetCellsAliasesResponse + * Creates a plain object from a GetRoutingRulesResponse message. Also converts values to other types if specified. + * @param message GetRoutingRulesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetCellsAliasesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetCellsAliasesResponse to JSON. + * Converts this GetRoutingRulesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetCellsAliasesResponse + * Gets the default type url for GetRoutingRulesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetFullStatusRequest. */ - interface IGetFullStatusRequest { + /** Properties of a GetSchemaRequest. */ + interface IGetSchemaRequest { - /** GetFullStatusRequest tablet_alias */ + /** GetSchemaRequest tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); + + /** GetSchemaRequest tables */ + tables?: (string[]|null); + + /** GetSchemaRequest exclude_tables */ + exclude_tables?: (string[]|null); + + /** GetSchemaRequest include_views */ + include_views?: (boolean|null); + + /** GetSchemaRequest table_names_only */ + table_names_only?: (boolean|null); + + /** GetSchemaRequest table_sizes_only */ + table_sizes_only?: (boolean|null); + + /** GetSchemaRequest table_schema_only */ + table_schema_only?: (boolean|null); } - /** Represents a GetFullStatusRequest. */ - class GetFullStatusRequest implements IGetFullStatusRequest { + /** Represents a GetSchemaRequest. */ + class GetSchemaRequest implements IGetSchemaRequest { /** - * Constructs a new GetFullStatusRequest. + * Constructs a new GetSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetFullStatusRequest); + constructor(properties?: vtctldata.IGetSchemaRequest); - /** GetFullStatusRequest tablet_alias. */ + /** GetSchemaRequest tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); + /** GetSchemaRequest tables. */ + public tables: string[]; + + /** GetSchemaRequest exclude_tables. */ + public exclude_tables: string[]; + + /** GetSchemaRequest include_views. */ + public include_views: boolean; + + /** GetSchemaRequest table_names_only. */ + public table_names_only: boolean; + + /** GetSchemaRequest table_sizes_only. */ + public table_sizes_only: boolean; + + /** GetSchemaRequest table_schema_only. */ + public table_schema_only: boolean; + /** - * Creates a new GetFullStatusRequest instance using the specified properties. + * Creates a new GetSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetFullStatusRequest instance + * @returns GetSchemaRequest instance */ - public static create(properties?: vtctldata.IGetFullStatusRequest): vtctldata.GetFullStatusRequest; + public static create(properties?: vtctldata.IGetSchemaRequest): vtctldata.GetSchemaRequest; /** - * Encodes the specified GetFullStatusRequest message. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. - * @param message GetFullStatusRequest message or plain object to encode + * Encodes the specified GetSchemaRequest message. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. + * @param message GetSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetFullStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetFullStatusRequest message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. - * @param message GetFullStatusRequest message or plain object to encode + * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. + * @param message GetSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetFullStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetFullStatusRequest message from the specified reader or buffer. + * Decodes a GetSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetFullStatusRequest + * @returns GetSchemaRequest * @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.GetFullStatusRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSchemaRequest; /** - * Decodes a GetFullStatusRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetFullStatusRequest + * @returns GetSchemaRequest * @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.GetFullStatusRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSchemaRequest; /** - * Verifies a GetFullStatusRequest message. + * Verifies a GetSchemaRequest 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 GetFullStatusRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetFullStatusRequest + * @returns GetSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetFullStatusRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSchemaRequest; /** - * Creates a plain object from a GetFullStatusRequest message. Also converts values to other types if specified. - * @param message GetFullStatusRequest + * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. + * @param message GetSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetFullStatusRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetFullStatusRequest to JSON. + * Converts this GetSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetFullStatusRequest + * Gets the default type url for GetSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetFullStatusResponse. */ - interface IGetFullStatusResponse { + /** Properties of a GetSchemaResponse. */ + interface IGetSchemaResponse { - /** GetFullStatusResponse status */ - status?: (replicationdata.IFullStatus|null); + /** GetSchemaResponse schema */ + schema?: (tabletmanagerdata.ISchemaDefinition|null); } - /** Represents a GetFullStatusResponse. */ - class GetFullStatusResponse implements IGetFullStatusResponse { + /** Represents a GetSchemaResponse. */ + class GetSchemaResponse implements IGetSchemaResponse { /** - * Constructs a new GetFullStatusResponse. + * Constructs a new GetSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetFullStatusResponse); + constructor(properties?: vtctldata.IGetSchemaResponse); - /** GetFullStatusResponse status. */ - public status?: (replicationdata.IFullStatus|null); + /** GetSchemaResponse schema. */ + public schema?: (tabletmanagerdata.ISchemaDefinition|null); /** - * Creates a new GetFullStatusResponse instance using the specified properties. + * Creates a new GetSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetFullStatusResponse instance + * @returns GetSchemaResponse instance */ - public static create(properties?: vtctldata.IGetFullStatusResponse): vtctldata.GetFullStatusResponse; + public static create(properties?: vtctldata.IGetSchemaResponse): vtctldata.GetSchemaResponse; /** - * Encodes the specified GetFullStatusResponse message. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. - * @param message GetFullStatusResponse message or plain object to encode + * Encodes the specified GetSchemaResponse message. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. + * @param message GetSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetFullStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetFullStatusResponse message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. - * @param message GetFullStatusResponse message or plain object to encode + * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. + * @param message GetSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetFullStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetFullStatusResponse message from the specified reader or buffer. + * Decodes a GetSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetFullStatusResponse + * @returns GetSchemaResponse * @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.GetFullStatusResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSchemaResponse; /** - * Decodes a GetFullStatusResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetFullStatusResponse + * @returns GetSchemaResponse * @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.GetFullStatusResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSchemaResponse; /** - * Verifies a GetFullStatusResponse message. + * Verifies a GetSchemaResponse 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 GetFullStatusResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetFullStatusResponse + * @returns GetSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetFullStatusResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSchemaResponse; /** - * Creates a plain object from a GetFullStatusResponse message. Also converts values to other types if specified. - * @param message GetFullStatusResponse + * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. + * @param message GetSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetFullStatusResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetFullStatusResponse to JSON. + * Converts this GetSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetFullStatusResponse + * Gets the default type url for GetSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetKeyspacesRequest. */ - interface IGetKeyspacesRequest { + /** Properties of a GetShardRequest. */ + interface IGetShardRequest { + + /** GetShardRequest keyspace */ + keyspace?: (string|null); + + /** GetShardRequest shard_name */ + shard_name?: (string|null); } - /** Represents a GetKeyspacesRequest. */ - class GetKeyspacesRequest implements IGetKeyspacesRequest { + /** Represents a GetShardRequest. */ + class GetShardRequest implements IGetShardRequest { /** - * Constructs a new GetKeyspacesRequest. + * Constructs a new GetShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetKeyspacesRequest); + constructor(properties?: vtctldata.IGetShardRequest); + + /** GetShardRequest keyspace. */ + public keyspace: string; + + /** GetShardRequest shard_name. */ + public shard_name: string; /** - * Creates a new GetKeyspacesRequest instance using the specified properties. + * Creates a new GetShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetKeyspacesRequest instance + * @returns GetShardRequest instance */ - public static create(properties?: vtctldata.IGetKeyspacesRequest): vtctldata.GetKeyspacesRequest; + public static create(properties?: vtctldata.IGetShardRequest): vtctldata.GetShardRequest; /** - * Encodes the specified GetKeyspacesRequest message. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. - * @param message GetKeyspacesRequest message or plain object to encode + * Encodes the specified GetShardRequest message. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. + * @param message GetShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. - * @param message GetKeyspacesRequest message or plain object to encode + * Encodes the specified GetShardRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. + * @param message GetShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetKeyspacesRequest message from the specified reader or buffer. + * Decodes a GetShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetKeyspacesRequest + * @returns GetShardRequest * @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.GetKeyspacesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetShardRequest; /** - * Decodes a GetKeyspacesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetKeyspacesRequest + * @returns GetShardRequest * @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.GetKeyspacesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetShardRequest; /** - * Verifies a GetKeyspacesRequest message. + * Verifies a GetShardRequest 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 GetKeyspacesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetKeyspacesRequest + * @returns GetShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspacesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRequest; /** - * Creates a plain object from a GetKeyspacesRequest message. Also converts values to other types if specified. - * @param message GetKeyspacesRequest + * Creates a plain object from a GetShardRequest message. Also converts values to other types if specified. + * @param message GetShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetKeyspacesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetKeyspacesRequest to JSON. + * Converts this GetShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetKeyspacesRequest + * Gets the default type url for GetShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetKeyspacesResponse. */ - interface IGetKeyspacesResponse { + /** Properties of a GetShardResponse. */ + interface IGetShardResponse { - /** GetKeyspacesResponse keyspaces */ - keyspaces?: (vtctldata.IKeyspace[]|null); + /** GetShardResponse shard */ + shard?: (vtctldata.IShard|null); } - /** Represents a GetKeyspacesResponse. */ - class GetKeyspacesResponse implements IGetKeyspacesResponse { + /** Represents a GetShardResponse. */ + class GetShardResponse implements IGetShardResponse { /** - * Constructs a new GetKeyspacesResponse. + * Constructs a new GetShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetKeyspacesResponse); + constructor(properties?: vtctldata.IGetShardResponse); - /** GetKeyspacesResponse keyspaces. */ - public keyspaces: vtctldata.IKeyspace[]; + /** GetShardResponse shard. */ + public shard?: (vtctldata.IShard|null); /** - * Creates a new GetKeyspacesResponse instance using the specified properties. + * Creates a new GetShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetKeyspacesResponse instance + * @returns GetShardResponse instance */ - public static create(properties?: vtctldata.IGetKeyspacesResponse): vtctldata.GetKeyspacesResponse; + public static create(properties?: vtctldata.IGetShardResponse): vtctldata.GetShardResponse; /** - * Encodes the specified GetKeyspacesResponse message. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. - * @param message GetKeyspacesResponse message or plain object to encode + * Encodes the specified GetShardResponse message. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. + * @param message GetShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. - * @param message GetKeyspacesResponse message or plain object to encode + * Encodes the specified GetShardResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. + * @param message GetShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetKeyspacesResponse message from the specified reader or buffer. + * Decodes a GetShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetKeyspacesResponse + * @returns GetShardResponse * @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.GetKeyspacesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetShardResponse; /** - * Decodes a GetKeyspacesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetKeyspacesResponse + * @returns GetShardResponse * @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.GetKeyspacesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetShardResponse; /** - * Verifies a GetKeyspacesResponse message. + * Verifies a GetShardResponse 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 GetKeyspacesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetKeyspacesResponse + * @returns GetShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspacesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetShardResponse; /** - * Creates a plain object from a GetKeyspacesResponse message. Also converts values to other types if specified. - * @param message GetKeyspacesResponse + * Creates a plain object from a GetShardResponse message. Also converts values to other types if specified. + * @param message GetShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetKeyspacesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetKeyspacesResponse to JSON. + * Converts this GetShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetKeyspacesResponse + * Gets the default type url for GetShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetKeyspaceRequest. */ - interface IGetKeyspaceRequest { - - /** GetKeyspaceRequest keyspace */ - keyspace?: (string|null); + /** Properties of a GetShardRoutingRulesRequest. */ + interface IGetShardRoutingRulesRequest { } - /** Represents a GetKeyspaceRequest. */ - class GetKeyspaceRequest implements IGetKeyspaceRequest { + /** Represents a GetShardRoutingRulesRequest. */ + class GetShardRoutingRulesRequest implements IGetShardRoutingRulesRequest { /** - * Constructs a new GetKeyspaceRequest. + * Constructs a new GetShardRoutingRulesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetKeyspaceRequest); - - /** GetKeyspaceRequest keyspace. */ - public keyspace: string; + constructor(properties?: vtctldata.IGetShardRoutingRulesRequest); /** - * Creates a new GetKeyspaceRequest instance using the specified properties. + * Creates a new GetShardRoutingRulesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetKeyspaceRequest instance + * @returns GetShardRoutingRulesRequest instance */ - public static create(properties?: vtctldata.IGetKeyspaceRequest): vtctldata.GetKeyspaceRequest; + public static create(properties?: vtctldata.IGetShardRoutingRulesRequest): vtctldata.GetShardRoutingRulesRequest; /** - * Encodes the specified GetKeyspaceRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. - * @param message GetKeyspaceRequest message or plain object to encode + * Encodes the specified GetShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. + * @param message GetShardRoutingRulesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. - * @param message GetKeyspaceRequest message or plain object to encode + * Encodes the specified GetShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. + * @param message GetShardRoutingRulesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetKeyspaceRequest message from the specified reader or buffer. + * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetKeyspaceRequest + * @returns GetShardRoutingRulesRequest * @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.GetKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetShardRoutingRulesRequest; /** - * Decodes a GetKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetKeyspaceRequest + * @returns GetShardRoutingRulesRequest * @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.GetKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetShardRoutingRulesRequest; /** - * Verifies a GetKeyspaceRequest message. + * Verifies a GetShardRoutingRulesRequest 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 GetKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetKeyspaceRequest + * @returns GetShardRoutingRulesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRoutingRulesRequest; /** - * Creates a plain object from a GetKeyspaceRequest message. Also converts values to other types if specified. - * @param message GetKeyspaceRequest + * Creates a plain object from a GetShardRoutingRulesRequest message. Also converts values to other types if specified. + * @param message GetShardRoutingRulesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetShardRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetKeyspaceRequest to JSON. + * Converts this GetShardRoutingRulesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetKeyspaceRequest + * Gets the default type url for GetShardRoutingRulesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetKeyspaceResponse. */ - interface IGetKeyspaceResponse { + /** Properties of a GetShardRoutingRulesResponse. */ + interface IGetShardRoutingRulesResponse { - /** GetKeyspaceResponse keyspace */ - keyspace?: (vtctldata.IKeyspace|null); + /** GetShardRoutingRulesResponse shard_routing_rules */ + shard_routing_rules?: (vschema.IShardRoutingRules|null); } - /** Represents a GetKeyspaceResponse. */ - class GetKeyspaceResponse implements IGetKeyspaceResponse { + /** Represents a GetShardRoutingRulesResponse. */ + class GetShardRoutingRulesResponse implements IGetShardRoutingRulesResponse { /** - * Constructs a new GetKeyspaceResponse. + * Constructs a new GetShardRoutingRulesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetKeyspaceResponse); + constructor(properties?: vtctldata.IGetShardRoutingRulesResponse); - /** GetKeyspaceResponse keyspace. */ - public keyspace?: (vtctldata.IKeyspace|null); + /** GetShardRoutingRulesResponse shard_routing_rules. */ + public shard_routing_rules?: (vschema.IShardRoutingRules|null); /** - * Creates a new GetKeyspaceResponse instance using the specified properties. + * Creates a new GetShardRoutingRulesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetKeyspaceResponse instance + * @returns GetShardRoutingRulesResponse instance */ - public static create(properties?: vtctldata.IGetKeyspaceResponse): vtctldata.GetKeyspaceResponse; + public static create(properties?: vtctldata.IGetShardRoutingRulesResponse): vtctldata.GetShardRoutingRulesResponse; /** - * Encodes the specified GetKeyspaceResponse message. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. - * @param message GetKeyspaceResponse message or plain object to encode + * Encodes the specified GetShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. + * @param message GetShardRoutingRulesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. - * @param message GetKeyspaceResponse message or plain object to encode + * Encodes the specified GetShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. + * @param message GetShardRoutingRulesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetKeyspaceResponse message from the specified reader or buffer. + * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetKeyspaceResponse + * @returns GetShardRoutingRulesResponse * @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.GetKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetShardRoutingRulesResponse; /** - * Decodes a GetKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetKeyspaceResponse + * @returns GetShardRoutingRulesResponse * @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.GetKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetShardRoutingRulesResponse; /** - * Verifies a GetKeyspaceResponse message. + * Verifies a GetShardRoutingRulesResponse 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 GetKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetKeyspaceResponse + * @returns GetShardRoutingRulesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRoutingRulesResponse; /** - * Creates a plain object from a GetKeyspaceResponse message. Also converts values to other types if specified. - * @param message GetKeyspaceResponse + * Creates a plain object from a GetShardRoutingRulesResponse message. Also converts values to other types if specified. + * @param message GetShardRoutingRulesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetShardRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetKeyspaceResponse to JSON. + * Converts this GetShardRoutingRulesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetKeyspaceResponse + * Gets the default type url for GetShardRoutingRulesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetPermissionsRequest. */ - interface IGetPermissionsRequest { + /** Properties of a GetSrvKeyspaceNamesRequest. */ + interface IGetSrvKeyspaceNamesRequest { - /** GetPermissionsRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** GetSrvKeyspaceNamesRequest cells */ + cells?: (string[]|null); } - /** Represents a GetPermissionsRequest. */ - class GetPermissionsRequest implements IGetPermissionsRequest { + /** Represents a GetSrvKeyspaceNamesRequest. */ + class GetSrvKeyspaceNamesRequest implements IGetSrvKeyspaceNamesRequest { /** - * Constructs a new GetPermissionsRequest. + * Constructs a new GetSrvKeyspaceNamesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetPermissionsRequest); + constructor(properties?: vtctldata.IGetSrvKeyspaceNamesRequest); - /** GetPermissionsRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** GetSrvKeyspaceNamesRequest cells. */ + public cells: string[]; /** - * Creates a new GetPermissionsRequest instance using the specified properties. + * Creates a new GetSrvKeyspaceNamesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetPermissionsRequest instance + * @returns GetSrvKeyspaceNamesRequest instance */ - public static create(properties?: vtctldata.IGetPermissionsRequest): vtctldata.GetPermissionsRequest; + public static create(properties?: vtctldata.IGetSrvKeyspaceNamesRequest): vtctldata.GetSrvKeyspaceNamesRequest; /** - * Encodes the specified GetPermissionsRequest message. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. - * @param message GetPermissionsRequest message or plain object to encode + * Encodes the specified GetSrvKeyspaceNamesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. + * @param message GetSrvKeyspaceNamesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetPermissionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvKeyspaceNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetPermissionsRequest message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. - * @param message GetPermissionsRequest message or plain object to encode + * Encodes the specified GetSrvKeyspaceNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. + * @param message GetSrvKeyspaceNamesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetPermissionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvKeyspaceNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetPermissionsRequest message from the specified reader or buffer. + * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetPermissionsRequest + * @returns GetSrvKeyspaceNamesRequest * @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.GetPermissionsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvKeyspaceNamesRequest; /** - * Decodes a GetPermissionsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetPermissionsRequest + * @returns GetSrvKeyspaceNamesRequest * @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.GetPermissionsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvKeyspaceNamesRequest; /** - * Verifies a GetPermissionsRequest message. + * Verifies a GetSrvKeyspaceNamesRequest 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 GetPermissionsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspaceNamesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetPermissionsRequest + * @returns GetSrvKeyspaceNamesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetPermissionsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesRequest; /** - * Creates a plain object from a GetPermissionsRequest message. Also converts values to other types if specified. - * @param message GetPermissionsRequest + * Creates a plain object from a GetSrvKeyspaceNamesRequest message. Also converts values to other types if specified. + * @param message GetSrvKeyspaceNamesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetPermissionsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvKeyspaceNamesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetPermissionsRequest to JSON. + * Converts this GetSrvKeyspaceNamesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetPermissionsRequest + * Gets the default type url for GetSrvKeyspaceNamesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetPermissionsResponse. */ - interface IGetPermissionsResponse { + /** Properties of a GetSrvKeyspaceNamesResponse. */ + interface IGetSrvKeyspaceNamesResponse { - /** GetPermissionsResponse permissions */ - permissions?: (tabletmanagerdata.IPermissions|null); + /** GetSrvKeyspaceNamesResponse names */ + names?: ({ [k: string]: vtctldata.GetSrvKeyspaceNamesResponse.INameList }|null); } - /** Represents a GetPermissionsResponse. */ - class GetPermissionsResponse implements IGetPermissionsResponse { + /** Represents a GetSrvKeyspaceNamesResponse. */ + class GetSrvKeyspaceNamesResponse implements IGetSrvKeyspaceNamesResponse { /** - * Constructs a new GetPermissionsResponse. + * Constructs a new GetSrvKeyspaceNamesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetPermissionsResponse); + constructor(properties?: vtctldata.IGetSrvKeyspaceNamesResponse); - /** GetPermissionsResponse permissions. */ - public permissions?: (tabletmanagerdata.IPermissions|null); + /** GetSrvKeyspaceNamesResponse names. */ + public names: { [k: string]: vtctldata.GetSrvKeyspaceNamesResponse.INameList }; /** - * Creates a new GetPermissionsResponse instance using the specified properties. + * Creates a new GetSrvKeyspaceNamesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetPermissionsResponse instance + * @returns GetSrvKeyspaceNamesResponse instance */ - public static create(properties?: vtctldata.IGetPermissionsResponse): vtctldata.GetPermissionsResponse; + public static create(properties?: vtctldata.IGetSrvKeyspaceNamesResponse): vtctldata.GetSrvKeyspaceNamesResponse; /** - * Encodes the specified GetPermissionsResponse message. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. - * @param message GetPermissionsResponse message or plain object to encode + * Encodes the specified GetSrvKeyspaceNamesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. + * @param message GetSrvKeyspaceNamesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetPermissionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvKeyspaceNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetPermissionsResponse message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. - * @param message GetPermissionsResponse message or plain object to encode + * Encodes the specified GetSrvKeyspaceNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. + * @param message GetSrvKeyspaceNamesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetPermissionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvKeyspaceNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetPermissionsResponse message from the specified reader or buffer. + * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetPermissionsResponse + * @returns GetSrvKeyspaceNamesResponse * @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.GetPermissionsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvKeyspaceNamesResponse; /** - * Decodes a GetPermissionsResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetPermissionsResponse + * @returns GetSrvKeyspaceNamesResponse * @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.GetPermissionsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvKeyspaceNamesResponse; /** - * Verifies a GetPermissionsResponse message. + * Verifies a GetSrvKeyspaceNamesResponse 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 GetPermissionsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspaceNamesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetPermissionsResponse + * @returns GetSrvKeyspaceNamesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetPermissionsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesResponse; /** - * Creates a plain object from a GetPermissionsResponse message. Also converts values to other types if specified. - * @param message GetPermissionsResponse + * Creates a plain object from a GetSrvKeyspaceNamesResponse message. Also converts values to other types if specified. + * @param message GetSrvKeyspaceNamesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetPermissionsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvKeyspaceNamesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetPermissionsResponse to JSON. + * Converts this GetSrvKeyspaceNamesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetPermissionsResponse + * Gets the default type url for GetSrvKeyspaceNamesResponse * @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 { + namespace GetSrvKeyspaceNamesResponse { + + /** Properties of a NameList. */ + interface INameList { + + /** NameList names */ + names?: (string[]|null); + } + + /** Represents a NameList. */ + class NameList implements INameList { + + /** + * Constructs a new NameList. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.GetSrvKeyspaceNamesResponse.INameList); + + /** NameList names. */ + public names: string[]; + + /** + * Creates a new NameList instance using the specified properties. + * @param [properties] Properties to set + * @returns NameList instance + */ + public static create(properties?: vtctldata.GetSrvKeyspaceNamesResponse.INameList): vtctldata.GetSrvKeyspaceNamesResponse.NameList; + + /** + * Encodes the specified NameList message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. + * @param message NameList message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.GetSrvKeyspaceNamesResponse.INameList, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified NameList message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. + * @param message NameList message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.GetSrvKeyspaceNamesResponse.INameList, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a NameList message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns NameList + * @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.GetSrvKeyspaceNamesResponse.NameList; + + /** + * Decodes a NameList message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns NameList + * @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.GetSrvKeyspaceNamesResponse.NameList; + + /** + * Verifies a NameList 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 NameList message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns NameList + */ + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesResponse.NameList; + + /** + * Creates a plain object from a NameList message. Also converts values to other types if specified. + * @param message NameList + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.GetSrvKeyspaceNamesResponse.NameList, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this NameList to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for NameList + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } } - /** Represents a GetRoutingRulesRequest. */ - class GetRoutingRulesRequest implements IGetRoutingRulesRequest { + /** Properties of a GetSrvKeyspacesRequest. */ + interface IGetSrvKeyspacesRequest { + + /** GetSrvKeyspacesRequest keyspace */ + keyspace?: (string|null); + + /** GetSrvKeyspacesRequest cells */ + cells?: (string[]|null); + } + + /** Represents a GetSrvKeyspacesRequest. */ + class GetSrvKeyspacesRequest implements IGetSrvKeyspacesRequest { /** - * Constructs a new GetRoutingRulesRequest. + * Constructs a new GetSrvKeyspacesRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetRoutingRulesRequest); + constructor(properties?: vtctldata.IGetSrvKeyspacesRequest); + + /** GetSrvKeyspacesRequest keyspace. */ + public keyspace: string; + + /** GetSrvKeyspacesRequest cells. */ + public cells: string[]; /** - * Creates a new GetRoutingRulesRequest instance using the specified properties. + * Creates a new GetSrvKeyspacesRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetRoutingRulesRequest instance + * @returns GetSrvKeyspacesRequest instance */ - public static create(properties?: vtctldata.IGetRoutingRulesRequest): vtctldata.GetRoutingRulesRequest; + public static create(properties?: vtctldata.IGetSrvKeyspacesRequest): vtctldata.GetSrvKeyspacesRequest; /** - * Encodes the specified GetRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. - * @param message GetRoutingRulesRequest message or plain object to encode + * Encodes the specified GetSrvKeyspacesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. + * @param message GetSrvKeyspacesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. - * @param message GetRoutingRulesRequest message or plain object to encode + * Encodes the specified GetSrvKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. + * @param message GetSrvKeyspacesRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetRoutingRulesRequest message from the specified reader or buffer. + * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetRoutingRulesRequest + * @returns GetSrvKeyspacesRequest * @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.GetRoutingRulesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvKeyspacesRequest; /** - * Decodes a GetRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetRoutingRulesRequest + * @returns GetSrvKeyspacesRequest * @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.GetRoutingRulesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvKeyspacesRequest; /** - * Verifies a GetRoutingRulesRequest message. + * Verifies a GetSrvKeyspacesRequest 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 GetRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspacesRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetRoutingRulesRequest + * @returns GetSrvKeyspacesRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetRoutingRulesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspacesRequest; /** - * Creates a plain object from a GetRoutingRulesRequest message. Also converts values to other types if specified. - * @param message GetRoutingRulesRequest + * Creates a plain object from a GetSrvKeyspacesRequest message. Also converts values to other types if specified. + * @param message GetSrvKeyspacesRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvKeyspacesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetRoutingRulesRequest to JSON. + * Converts this GetSrvKeyspacesRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetRoutingRulesRequest + * Gets the default type url for GetSrvKeyspacesRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetRoutingRulesResponse. */ - interface IGetRoutingRulesResponse { + /** Properties of a GetSrvKeyspacesResponse. */ + interface IGetSrvKeyspacesResponse { - /** GetRoutingRulesResponse routing_rules */ - routing_rules?: (vschema.IRoutingRules|null); + /** GetSrvKeyspacesResponse srv_keyspaces */ + srv_keyspaces?: ({ [k: string]: topodata.ISrvKeyspace }|null); } - /** Represents a GetRoutingRulesResponse. */ - class GetRoutingRulesResponse implements IGetRoutingRulesResponse { + /** Represents a GetSrvKeyspacesResponse. */ + class GetSrvKeyspacesResponse implements IGetSrvKeyspacesResponse { /** - * Constructs a new GetRoutingRulesResponse. + * Constructs a new GetSrvKeyspacesResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetRoutingRulesResponse); + constructor(properties?: vtctldata.IGetSrvKeyspacesResponse); - /** GetRoutingRulesResponse routing_rules. */ - public routing_rules?: (vschema.IRoutingRules|null); + /** GetSrvKeyspacesResponse srv_keyspaces. */ + public srv_keyspaces: { [k: string]: topodata.ISrvKeyspace }; /** - * Creates a new GetRoutingRulesResponse instance using the specified properties. + * Creates a new GetSrvKeyspacesResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetRoutingRulesResponse instance + * @returns GetSrvKeyspacesResponse instance */ - public static create(properties?: vtctldata.IGetRoutingRulesResponse): vtctldata.GetRoutingRulesResponse; + public static create(properties?: vtctldata.IGetSrvKeyspacesResponse): vtctldata.GetSrvKeyspacesResponse; /** - * Encodes the specified GetRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. - * @param message GetRoutingRulesResponse message or plain object to encode + * Encodes the specified GetSrvKeyspacesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. + * @param message GetSrvKeyspacesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. - * @param message GetRoutingRulesResponse message or plain object to encode + * Encodes the specified GetSrvKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. + * @param message GetSrvKeyspacesResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetRoutingRulesResponse message from the specified reader or buffer. + * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetRoutingRulesResponse + * @returns GetSrvKeyspacesResponse * @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.GetRoutingRulesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvKeyspacesResponse; /** - * Decodes a GetRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetRoutingRulesResponse + * @returns GetSrvKeyspacesResponse * @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.GetRoutingRulesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvKeyspacesResponse; /** - * Verifies a GetRoutingRulesResponse message. + * Verifies a GetSrvKeyspacesResponse 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 GetRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspacesResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetRoutingRulesResponse + * @returns GetSrvKeyspacesResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetRoutingRulesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspacesResponse; /** - * Creates a plain object from a GetRoutingRulesResponse message. Also converts values to other types if specified. - * @param message GetRoutingRulesResponse + * Creates a plain object from a GetSrvKeyspacesResponse message. Also converts values to other types if specified. + * @param message GetSrvKeyspacesResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvKeyspacesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetRoutingRulesResponse to JSON. + * Converts this GetSrvKeyspacesResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetRoutingRulesResponse + * Gets the default type url for GetSrvKeyspacesResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSchemaRequest. */ - interface IGetSchemaRequest { + /** Properties of an UpdateThrottlerConfigRequest. */ + interface IUpdateThrottlerConfigRequest { - /** GetSchemaRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateThrottlerConfigRequest keyspace */ + keyspace?: (string|null); - /** GetSchemaRequest tables */ - tables?: (string[]|null); + /** UpdateThrottlerConfigRequest enable */ + enable?: (boolean|null); - /** GetSchemaRequest exclude_tables */ - exclude_tables?: (string[]|null); + /** UpdateThrottlerConfigRequest disable */ + disable?: (boolean|null); - /** GetSchemaRequest include_views */ - include_views?: (boolean|null); + /** UpdateThrottlerConfigRequest threshold */ + threshold?: (number|null); - /** GetSchemaRequest table_names_only */ - table_names_only?: (boolean|null); + /** UpdateThrottlerConfigRequest custom_query */ + custom_query?: (string|null); - /** GetSchemaRequest table_sizes_only */ - table_sizes_only?: (boolean|null); + /** UpdateThrottlerConfigRequest custom_query_set */ + custom_query_set?: (boolean|null); - /** GetSchemaRequest table_schema_only */ - table_schema_only?: (boolean|null); + /** UpdateThrottlerConfigRequest check_as_check_self */ + check_as_check_self?: (boolean|null); + + /** UpdateThrottlerConfigRequest check_as_check_shard */ + check_as_check_shard?: (boolean|null); + + /** UpdateThrottlerConfigRequest throttled_app */ + throttled_app?: (topodata.IThrottledAppRule|null); } - /** Represents a GetSchemaRequest. */ - class GetSchemaRequest implements IGetSchemaRequest { + /** Represents an UpdateThrottlerConfigRequest. */ + class UpdateThrottlerConfigRequest implements IUpdateThrottlerConfigRequest { /** - * Constructs a new GetSchemaRequest. + * Constructs a new UpdateThrottlerConfigRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSchemaRequest); + constructor(properties?: vtctldata.IUpdateThrottlerConfigRequest); - /** GetSchemaRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateThrottlerConfigRequest keyspace. */ + public keyspace: string; - /** GetSchemaRequest tables. */ - public tables: string[]; + /** UpdateThrottlerConfigRequest enable. */ + public enable: boolean; - /** GetSchemaRequest exclude_tables. */ - public exclude_tables: string[]; + /** UpdateThrottlerConfigRequest disable. */ + public disable: boolean; + + /** UpdateThrottlerConfigRequest threshold. */ + public threshold: number; + + /** UpdateThrottlerConfigRequest custom_query. */ + public custom_query: string; - /** GetSchemaRequest include_views. */ - public include_views: boolean; + /** UpdateThrottlerConfigRequest custom_query_set. */ + public custom_query_set: boolean; - /** GetSchemaRequest table_names_only. */ - public table_names_only: boolean; + /** UpdateThrottlerConfigRequest check_as_check_self. */ + public check_as_check_self: boolean; - /** GetSchemaRequest table_sizes_only. */ - public table_sizes_only: boolean; + /** UpdateThrottlerConfigRequest check_as_check_shard. */ + public check_as_check_shard: boolean; - /** GetSchemaRequest table_schema_only. */ - public table_schema_only: boolean; + /** UpdateThrottlerConfigRequest throttled_app. */ + public throttled_app?: (topodata.IThrottledAppRule|null); /** - * Creates a new GetSchemaRequest instance using the specified properties. + * Creates a new UpdateThrottlerConfigRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSchemaRequest instance + * @returns UpdateThrottlerConfigRequest instance */ - public static create(properties?: vtctldata.IGetSchemaRequest): vtctldata.GetSchemaRequest; + public static create(properties?: vtctldata.IUpdateThrottlerConfigRequest): vtctldata.UpdateThrottlerConfigRequest; /** - * Encodes the specified GetSchemaRequest message. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. - * @param message GetSchemaRequest message or plain object to encode + * Encodes the specified UpdateThrottlerConfigRequest message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. + * @param message UpdateThrottlerConfigRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateThrottlerConfigRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. - * @param message GetSchemaRequest message or plain object to encode + * Encodes the specified UpdateThrottlerConfigRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. + * @param message UpdateThrottlerConfigRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateThrottlerConfigRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer. + * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSchemaRequest + * @returns UpdateThrottlerConfigRequest * @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.GetSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateThrottlerConfigRequest; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSchemaRequest + * @returns UpdateThrottlerConfigRequest * @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.GetSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateThrottlerConfigRequest; /** - * Verifies a GetSchemaRequest message. + * Verifies an UpdateThrottlerConfigRequest 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 GetSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateThrottlerConfigRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSchemaRequest + * @returns UpdateThrottlerConfigRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateThrottlerConfigRequest; /** - * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. - * @param message GetSchemaRequest + * Creates a plain object from an UpdateThrottlerConfigRequest message. Also converts values to other types if specified. + * @param message UpdateThrottlerConfigRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateThrottlerConfigRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSchemaRequest to JSON. + * Converts this UpdateThrottlerConfigRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSchemaRequest + * Gets the default type url for UpdateThrottlerConfigRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSchemaResponse. */ - interface IGetSchemaResponse { - - /** GetSchemaResponse schema */ - schema?: (tabletmanagerdata.ISchemaDefinition|null); + /** Properties of an UpdateThrottlerConfigResponse. */ + interface IUpdateThrottlerConfigResponse { } - /** Represents a GetSchemaResponse. */ - class GetSchemaResponse implements IGetSchemaResponse { + /** Represents an UpdateThrottlerConfigResponse. */ + class UpdateThrottlerConfigResponse implements IUpdateThrottlerConfigResponse { /** - * Constructs a new GetSchemaResponse. + * Constructs a new UpdateThrottlerConfigResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSchemaResponse); - - /** GetSchemaResponse schema. */ - public schema?: (tabletmanagerdata.ISchemaDefinition|null); + constructor(properties?: vtctldata.IUpdateThrottlerConfigResponse); /** - * Creates a new GetSchemaResponse instance using the specified properties. + * Creates a new UpdateThrottlerConfigResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetSchemaResponse instance + * @returns UpdateThrottlerConfigResponse instance */ - public static create(properties?: vtctldata.IGetSchemaResponse): vtctldata.GetSchemaResponse; + public static create(properties?: vtctldata.IUpdateThrottlerConfigResponse): vtctldata.UpdateThrottlerConfigResponse; /** - * Encodes the specified GetSchemaResponse message. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. - * @param message GetSchemaResponse message or plain object to encode + * Encodes the specified UpdateThrottlerConfigResponse message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. + * @param message UpdateThrottlerConfigResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateThrottlerConfigResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. - * @param message GetSchemaResponse message or plain object to encode + * Encodes the specified UpdateThrottlerConfigResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. + * @param message UpdateThrottlerConfigResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateThrottlerConfigResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer. + * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSchemaResponse + * @returns UpdateThrottlerConfigResponse * @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.GetSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateThrottlerConfigResponse; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSchemaResponse + * @returns UpdateThrottlerConfigResponse * @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.GetSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateThrottlerConfigResponse; /** - * Verifies a GetSchemaResponse message. + * Verifies an UpdateThrottlerConfigResponse 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 GetSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateThrottlerConfigResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSchemaResponse + * @returns UpdateThrottlerConfigResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateThrottlerConfigResponse; /** - * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. - * @param message GetSchemaResponse + * Creates a plain object from an UpdateThrottlerConfigResponse message. Also converts values to other types if specified. + * @param message UpdateThrottlerConfigResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateThrottlerConfigResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSchemaResponse to JSON. + * Converts this UpdateThrottlerConfigResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSchemaResponse + * Gets the default type url for UpdateThrottlerConfigResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetShardRequest. */ - interface IGetShardRequest { - - /** GetShardRequest keyspace */ - keyspace?: (string|null); + /** Properties of a GetSrvVSchemaRequest. */ + interface IGetSrvVSchemaRequest { - /** GetShardRequest shard_name */ - shard_name?: (string|null); + /** GetSrvVSchemaRequest cell */ + cell?: (string|null); } - /** Represents a GetShardRequest. */ - class GetShardRequest implements IGetShardRequest { + /** Represents a GetSrvVSchemaRequest. */ + class GetSrvVSchemaRequest implements IGetSrvVSchemaRequest { /** - * Constructs a new GetShardRequest. + * Constructs a new GetSrvVSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetShardRequest); - - /** GetShardRequest keyspace. */ - public keyspace: string; + constructor(properties?: vtctldata.IGetSrvVSchemaRequest); - /** GetShardRequest shard_name. */ - public shard_name: string; + /** GetSrvVSchemaRequest cell. */ + public cell: string; /** - * Creates a new GetShardRequest instance using the specified properties. + * Creates a new GetSrvVSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetShardRequest instance + * @returns GetSrvVSchemaRequest instance */ - public static create(properties?: vtctldata.IGetShardRequest): vtctldata.GetShardRequest; + public static create(properties?: vtctldata.IGetSrvVSchemaRequest): vtctldata.GetSrvVSchemaRequest; /** - * Encodes the specified GetShardRequest message. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. - * @param message GetShardRequest message or plain object to encode + * Encodes the specified GetSrvVSchemaRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. + * @param message GetSrvVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetShardRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. - * @param message GetShardRequest message or plain object to encode + * Encodes the specified GetSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. + * @param message GetSrvVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetShardRequest message from the specified reader or buffer. + * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetShardRequest + * @returns GetSrvVSchemaRequest * @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.GetShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvVSchemaRequest; /** - * Decodes a GetShardRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetShardRequest + * @returns GetSrvVSchemaRequest * @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.GetShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvVSchemaRequest; /** - * Verifies a GetShardRequest message. + * Verifies a GetSrvVSchemaRequest 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 GetShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetShardRequest + * @returns GetSrvVSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemaRequest; /** - * Creates a plain object from a GetShardRequest message. Also converts values to other types if specified. - * @param message GetShardRequest + * Creates a plain object from a GetSrvVSchemaRequest message. Also converts values to other types if specified. + * @param message GetSrvVSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetShardRequest to JSON. + * Converts this GetSrvVSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetShardRequest + * Gets the default type url for GetSrvVSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetShardResponse. */ - interface IGetShardResponse { + /** Properties of a GetSrvVSchemaResponse. */ + interface IGetSrvVSchemaResponse { - /** GetShardResponse shard */ - shard?: (vtctldata.IShard|null); + /** GetSrvVSchemaResponse srv_v_schema */ + srv_v_schema?: (vschema.ISrvVSchema|null); } - /** Represents a GetShardResponse. */ - class GetShardResponse implements IGetShardResponse { + /** Represents a GetSrvVSchemaResponse. */ + class GetSrvVSchemaResponse implements IGetSrvVSchemaResponse { /** - * Constructs a new GetShardResponse. + * Constructs a new GetSrvVSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetShardResponse); + constructor(properties?: vtctldata.IGetSrvVSchemaResponse); - /** GetShardResponse shard. */ - public shard?: (vtctldata.IShard|null); + /** GetSrvVSchemaResponse srv_v_schema. */ + public srv_v_schema?: (vschema.ISrvVSchema|null); /** - * Creates a new GetShardResponse instance using the specified properties. + * Creates a new GetSrvVSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetShardResponse instance + * @returns GetSrvVSchemaResponse instance */ - public static create(properties?: vtctldata.IGetShardResponse): vtctldata.GetShardResponse; + public static create(properties?: vtctldata.IGetSrvVSchemaResponse): vtctldata.GetSrvVSchemaResponse; /** - * Encodes the specified GetShardResponse message. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. - * @param message GetShardResponse message or plain object to encode + * Encodes the specified GetSrvVSchemaResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. + * @param message GetSrvVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetShardResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. - * @param message GetShardResponse message or plain object to encode + * Encodes the specified GetSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. + * @param message GetSrvVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetShardResponse message from the specified reader or buffer. + * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetShardResponse + * @returns GetSrvVSchemaResponse * @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.GetShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvVSchemaResponse; /** - * Decodes a GetShardResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetShardResponse + * @returns GetSrvVSchemaResponse * @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.GetShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvVSchemaResponse; /** - * Verifies a GetShardResponse message. + * Verifies a GetSrvVSchemaResponse 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 GetShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetShardResponse + * @returns GetSrvVSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemaResponse; /** - * Creates a plain object from a GetShardResponse message. Also converts values to other types if specified. - * @param message GetShardResponse + * Creates a plain object from a GetSrvVSchemaResponse message. Also converts values to other types if specified. + * @param message GetSrvVSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetShardResponse to JSON. + * Converts this GetSrvVSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetShardResponse + * Gets the default type url for GetSrvVSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetShardRoutingRulesRequest. */ - interface IGetShardRoutingRulesRequest { + /** Properties of a GetSrvVSchemasRequest. */ + interface IGetSrvVSchemasRequest { + + /** GetSrvVSchemasRequest cells */ + cells?: (string[]|null); } - /** Represents a GetShardRoutingRulesRequest. */ - class GetShardRoutingRulesRequest implements IGetShardRoutingRulesRequest { + /** Represents a GetSrvVSchemasRequest. */ + class GetSrvVSchemasRequest implements IGetSrvVSchemasRequest { /** - * Constructs a new GetShardRoutingRulesRequest. + * Constructs a new GetSrvVSchemasRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetShardRoutingRulesRequest); + constructor(properties?: vtctldata.IGetSrvVSchemasRequest); + + /** GetSrvVSchemasRequest cells. */ + public cells: string[]; /** - * Creates a new GetShardRoutingRulesRequest instance using the specified properties. + * Creates a new GetSrvVSchemasRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetShardRoutingRulesRequest instance + * @returns GetSrvVSchemasRequest instance */ - public static create(properties?: vtctldata.IGetShardRoutingRulesRequest): vtctldata.GetShardRoutingRulesRequest; + public static create(properties?: vtctldata.IGetSrvVSchemasRequest): vtctldata.GetSrvVSchemasRequest; /** - * Encodes the specified GetShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. - * @param message GetShardRoutingRulesRequest message or plain object to encode + * Encodes the specified GetSrvVSchemasRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. + * @param message GetSrvVSchemasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvVSchemasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. - * @param message GetShardRoutingRulesRequest message or plain object to encode + * Encodes the specified GetSrvVSchemasRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. + * @param message GetSrvVSchemasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetShardRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvVSchemasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer. + * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetShardRoutingRulesRequest + * @returns GetSrvVSchemasRequest * @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.GetShardRoutingRulesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvVSchemasRequest; /** - * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetShardRoutingRulesRequest + * @returns GetSrvVSchemasRequest * @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.GetShardRoutingRulesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvVSchemasRequest; /** - * Verifies a GetShardRoutingRulesRequest message. + * Verifies a GetSrvVSchemasRequest 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 GetShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemasRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetShardRoutingRulesRequest + * @returns GetSrvVSchemasRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRoutingRulesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemasRequest; /** - * Creates a plain object from a GetShardRoutingRulesRequest message. Also converts values to other types if specified. - * @param message GetShardRoutingRulesRequest + * Creates a plain object from a GetSrvVSchemasRequest message. Also converts values to other types if specified. + * @param message GetSrvVSchemasRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetShardRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvVSchemasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetShardRoutingRulesRequest to JSON. + * Converts this GetSrvVSchemasRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetShardRoutingRulesRequest + * Gets the default type url for GetSrvVSchemasRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetShardRoutingRulesResponse. */ - interface IGetShardRoutingRulesResponse { + /** Properties of a GetSrvVSchemasResponse. */ + interface IGetSrvVSchemasResponse { - /** GetShardRoutingRulesResponse shard_routing_rules */ - shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** GetSrvVSchemasResponse srv_v_schemas */ + srv_v_schemas?: ({ [k: string]: vschema.ISrvVSchema }|null); } - /** Represents a GetShardRoutingRulesResponse. */ - class GetShardRoutingRulesResponse implements IGetShardRoutingRulesResponse { + /** Represents a GetSrvVSchemasResponse. */ + class GetSrvVSchemasResponse implements IGetSrvVSchemasResponse { /** - * Constructs a new GetShardRoutingRulesResponse. + * Constructs a new GetSrvVSchemasResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetShardRoutingRulesResponse); + constructor(properties?: vtctldata.IGetSrvVSchemasResponse); - /** GetShardRoutingRulesResponse shard_routing_rules. */ - public shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** GetSrvVSchemasResponse srv_v_schemas. */ + public srv_v_schemas: { [k: string]: vschema.ISrvVSchema }; /** - * Creates a new GetShardRoutingRulesResponse instance using the specified properties. + * Creates a new GetSrvVSchemasResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetShardRoutingRulesResponse instance + * @returns GetSrvVSchemasResponse instance */ - public static create(properties?: vtctldata.IGetShardRoutingRulesResponse): vtctldata.GetShardRoutingRulesResponse; + public static create(properties?: vtctldata.IGetSrvVSchemasResponse): vtctldata.GetSrvVSchemasResponse; /** - * Encodes the specified GetShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. - * @param message GetShardRoutingRulesResponse message or plain object to encode + * Encodes the specified GetSrvVSchemasResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. + * @param message GetSrvVSchemasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetSrvVSchemasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. - * @param message GetShardRoutingRulesResponse message or plain object to encode + * Encodes the specified GetSrvVSchemasResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. + * @param message GetSrvVSchemasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetShardRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetSrvVSchemasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer. + * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetShardRoutingRulesResponse + * @returns GetSrvVSchemasResponse * @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.GetShardRoutingRulesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetSrvVSchemasResponse; /** - * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetShardRoutingRulesResponse + * @returns GetSrvVSchemasResponse * @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.GetShardRoutingRulesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetSrvVSchemasResponse; /** - * Verifies a GetShardRoutingRulesResponse message. + * Verifies a GetSrvVSchemasResponse 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 GetShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemasResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetShardRoutingRulesResponse + * @returns GetSrvVSchemasResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetShardRoutingRulesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemasResponse; /** - * Creates a plain object from a GetShardRoutingRulesResponse message. Also converts values to other types if specified. - * @param message GetShardRoutingRulesResponse + * Creates a plain object from a GetSrvVSchemasResponse message. Also converts values to other types if specified. + * @param message GetSrvVSchemasResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetShardRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetSrvVSchemasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetShardRoutingRulesResponse to JSON. + * Converts this GetSrvVSchemasResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetShardRoutingRulesResponse + * Gets the default type url for GetSrvVSchemasResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvKeyspaceNamesRequest. */ - interface IGetSrvKeyspaceNamesRequest { + /** Properties of a GetTabletRequest. */ + interface IGetTabletRequest { - /** GetSrvKeyspaceNamesRequest cells */ - cells?: (string[]|null); + /** GetTabletRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a GetSrvKeyspaceNamesRequest. */ - class GetSrvKeyspaceNamesRequest implements IGetSrvKeyspaceNamesRequest { + /** Represents a GetTabletRequest. */ + class GetTabletRequest implements IGetTabletRequest { /** - * Constructs a new GetSrvKeyspaceNamesRequest. + * Constructs a new GetTabletRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvKeyspaceNamesRequest); + constructor(properties?: vtctldata.IGetTabletRequest); - /** GetSrvKeyspaceNamesRequest cells. */ - public cells: string[]; + /** GetTabletRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new GetSrvKeyspaceNamesRequest instance using the specified properties. + * Creates a new GetTabletRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvKeyspaceNamesRequest instance + * @returns GetTabletRequest instance */ - public static create(properties?: vtctldata.IGetSrvKeyspaceNamesRequest): vtctldata.GetSrvKeyspaceNamesRequest; + public static create(properties?: vtctldata.IGetTabletRequest): vtctldata.GetTabletRequest; /** - * Encodes the specified GetSrvKeyspaceNamesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. - * @param message GetSrvKeyspaceNamesRequest message or plain object to encode + * Encodes the specified GetTabletRequest message. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. + * @param message GetTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvKeyspaceNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvKeyspaceNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. - * @param message GetSrvKeyspaceNamesRequest message or plain object to encode + * Encodes the specified GetTabletRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. + * @param message GetTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvKeyspaceNamesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer. + * Decodes a GetTabletRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvKeyspaceNamesRequest + * @returns GetTabletRequest * @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.GetSrvKeyspaceNamesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTabletRequest; /** - * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTabletRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvKeyspaceNamesRequest + * @returns GetTabletRequest * @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.GetSrvKeyspaceNamesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTabletRequest; /** - * Verifies a GetSrvKeyspaceNamesRequest message. + * Verifies a GetTabletRequest 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 GetSrvKeyspaceNamesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvKeyspaceNamesRequest + * @returns GetTabletRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletRequest; /** - * Creates a plain object from a GetSrvKeyspaceNamesRequest message. Also converts values to other types if specified. - * @param message GetSrvKeyspaceNamesRequest + * Creates a plain object from a GetTabletRequest message. Also converts values to other types if specified. + * @param message GetTabletRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvKeyspaceNamesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvKeyspaceNamesRequest to JSON. + * Converts this GetTabletRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvKeyspaceNamesRequest + * Gets the default type url for GetTabletRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvKeyspaceNamesResponse. */ - interface IGetSrvKeyspaceNamesResponse { + /** Properties of a GetTabletResponse. */ + interface IGetTabletResponse { - /** GetSrvKeyspaceNamesResponse names */ - names?: ({ [k: string]: vtctldata.GetSrvKeyspaceNamesResponse.INameList }|null); + /** GetTabletResponse tablet */ + tablet?: (topodata.ITablet|null); } - /** Represents a GetSrvKeyspaceNamesResponse. */ - class GetSrvKeyspaceNamesResponse implements IGetSrvKeyspaceNamesResponse { + /** Represents a GetTabletResponse. */ + class GetTabletResponse implements IGetTabletResponse { /** - * Constructs a new GetSrvKeyspaceNamesResponse. + * Constructs a new GetTabletResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvKeyspaceNamesResponse); + constructor(properties?: vtctldata.IGetTabletResponse); - /** GetSrvKeyspaceNamesResponse names. */ - public names: { [k: string]: vtctldata.GetSrvKeyspaceNamesResponse.INameList }; + /** GetTabletResponse tablet. */ + public tablet?: (topodata.ITablet|null); /** - * Creates a new GetSrvKeyspaceNamesResponse instance using the specified properties. + * Creates a new GetTabletResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvKeyspaceNamesResponse instance + * @returns GetTabletResponse instance */ - public static create(properties?: vtctldata.IGetSrvKeyspaceNamesResponse): vtctldata.GetSrvKeyspaceNamesResponse; + public static create(properties?: vtctldata.IGetTabletResponse): vtctldata.GetTabletResponse; /** - * Encodes the specified GetSrvKeyspaceNamesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. - * @param message GetSrvKeyspaceNamesResponse message or plain object to encode + * Encodes the specified GetTabletResponse message. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. + * @param message GetTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvKeyspaceNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvKeyspaceNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. - * @param message GetSrvKeyspaceNamesResponse message or plain object to encode + * Encodes the specified GetTabletResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. + * @param message GetTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvKeyspaceNamesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer. + * Decodes a GetTabletResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvKeyspaceNamesResponse + * @returns GetTabletResponse * @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.GetSrvKeyspaceNamesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTabletResponse; /** - * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTabletResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvKeyspaceNamesResponse + * @returns GetTabletResponse * @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.GetSrvKeyspaceNamesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTabletResponse; /** - * Verifies a GetSrvKeyspaceNamesResponse message. + * Verifies a GetTabletResponse 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 GetSrvKeyspaceNamesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvKeyspaceNamesResponse + * @returns GetTabletResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletResponse; /** - * Creates a plain object from a GetSrvKeyspaceNamesResponse message. Also converts values to other types if specified. - * @param message GetSrvKeyspaceNamesResponse + * Creates a plain object from a GetTabletResponse message. Also converts values to other types if specified. + * @param message GetTabletResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvKeyspaceNamesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvKeyspaceNamesResponse to JSON. + * Converts this GetTabletResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvKeyspaceNamesResponse + * Gets the default type url for GetTabletResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - namespace GetSrvKeyspaceNamesResponse { - - /** Properties of a NameList. */ - interface INameList { - - /** NameList names */ - names?: (string[]|null); - } - - /** Represents a NameList. */ - class NameList implements INameList { - - /** - * Constructs a new NameList. - * @param [properties] Properties to set - */ - constructor(properties?: vtctldata.GetSrvKeyspaceNamesResponse.INameList); - - /** NameList names. */ - public names: string[]; - - /** - * Creates a new NameList instance using the specified properties. - * @param [properties] Properties to set - * @returns NameList instance - */ - public static create(properties?: vtctldata.GetSrvKeyspaceNamesResponse.INameList): vtctldata.GetSrvKeyspaceNamesResponse.NameList; - - /** - * Encodes the specified NameList message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. - * @param message NameList message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encode(message: vtctldata.GetSrvKeyspaceNamesResponse.INameList, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Encodes the specified NameList message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. - * @param message NameList message or plain object to encode - * @param [writer] Writer to encode to - * @returns Writer - */ - public static encodeDelimited(message: vtctldata.GetSrvKeyspaceNamesResponse.INameList, writer?: $protobuf.Writer): $protobuf.Writer; - - /** - * Decodes a NameList message from the specified reader or buffer. - * @param reader Reader or buffer to decode from - * @param [length] Message length if known beforehand - * @returns NameList - * @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.GetSrvKeyspaceNamesResponse.NameList; - - /** - * Decodes a NameList message from the specified reader or buffer, length delimited. - * @param reader Reader or buffer to decode from - * @returns NameList - * @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.GetSrvKeyspaceNamesResponse.NameList; - - /** - * Verifies a NameList 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 NameList message from a plain object. Also converts values to their respective internal types. - * @param object Plain object - * @returns NameList - */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspaceNamesResponse.NameList; + /** Properties of a GetTabletsRequest. */ + interface IGetTabletsRequest { - /** - * Creates a plain object from a NameList message. Also converts values to other types if specified. - * @param message NameList - * @param [options] Conversion options - * @returns Plain object - */ - public static toObject(message: vtctldata.GetSrvKeyspaceNamesResponse.NameList, options?: $protobuf.IConversionOptions): { [k: string]: any }; + /** GetTabletsRequest keyspace */ + keyspace?: (string|null); - /** - * Converts this NameList to JSON. - * @returns JSON object - */ - public toJSON(): { [k: string]: any }; + /** GetTabletsRequest shard */ + shard?: (string|null); - /** - * Gets the default type url for NameList - * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns The default type url - */ - public static getTypeUrl(typeUrlPrefix?: string): string; - } - } + /** GetTabletsRequest cells */ + cells?: (string[]|null); - /** Properties of a GetSrvKeyspacesRequest. */ - interface IGetSrvKeyspacesRequest { + /** GetTabletsRequest strict */ + strict?: (boolean|null); - /** GetSrvKeyspacesRequest keyspace */ - keyspace?: (string|null); + /** GetTabletsRequest tablet_aliases */ + tablet_aliases?: (topodata.ITabletAlias[]|null); - /** GetSrvKeyspacesRequest cells */ - cells?: (string[]|null); + /** GetTabletsRequest tablet_type */ + tablet_type?: (topodata.TabletType|null); } - /** Represents a GetSrvKeyspacesRequest. */ - class GetSrvKeyspacesRequest implements IGetSrvKeyspacesRequest { + /** Represents a GetTabletsRequest. */ + class GetTabletsRequest implements IGetTabletsRequest { /** - * Constructs a new GetSrvKeyspacesRequest. + * Constructs a new GetTabletsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvKeyspacesRequest); + constructor(properties?: vtctldata.IGetTabletsRequest); - /** GetSrvKeyspacesRequest keyspace. */ + /** GetTabletsRequest keyspace. */ public keyspace: string; - /** GetSrvKeyspacesRequest cells. */ + /** GetTabletsRequest shard. */ + public shard: string; + + /** GetTabletsRequest cells. */ public cells: string[]; + /** GetTabletsRequest strict. */ + public strict: boolean; + + /** GetTabletsRequest tablet_aliases. */ + public tablet_aliases: topodata.ITabletAlias[]; + + /** GetTabletsRequest tablet_type. */ + public tablet_type: topodata.TabletType; + /** - * Creates a new GetSrvKeyspacesRequest instance using the specified properties. + * Creates a new GetTabletsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvKeyspacesRequest instance + * @returns GetTabletsRequest instance */ - public static create(properties?: vtctldata.IGetSrvKeyspacesRequest): vtctldata.GetSrvKeyspacesRequest; + public static create(properties?: vtctldata.IGetTabletsRequest): vtctldata.GetTabletsRequest; /** - * Encodes the specified GetSrvKeyspacesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. - * @param message GetSrvKeyspacesRequest message or plain object to encode + * Encodes the specified GetTabletsRequest message. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. + * @param message GetTabletsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. - * @param message GetSrvKeyspacesRequest message or plain object to encode + * Encodes the specified GetTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. + * @param message GetTabletsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvKeyspacesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer. + * Decodes a GetTabletsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvKeyspacesRequest + * @returns GetTabletsRequest * @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.GetSrvKeyspacesRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTabletsRequest; /** - * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTabletsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvKeyspacesRequest + * @returns GetTabletsRequest * @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.GetSrvKeyspacesRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTabletsRequest; /** - * Verifies a GetSrvKeyspacesRequest message. + * Verifies a GetTabletsRequest 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 GetSrvKeyspacesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvKeyspacesRequest + * @returns GetTabletsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspacesRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletsRequest; /** - * Creates a plain object from a GetSrvKeyspacesRequest message. Also converts values to other types if specified. - * @param message GetSrvKeyspacesRequest + * Creates a plain object from a GetTabletsRequest message. Also converts values to other types if specified. + * @param message GetTabletsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvKeyspacesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTabletsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvKeyspacesRequest to JSON. + * Converts this GetTabletsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvKeyspacesRequest + * Gets the default type url for GetTabletsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvKeyspacesResponse. */ - interface IGetSrvKeyspacesResponse { + /** Properties of a GetTabletsResponse. */ + interface IGetTabletsResponse { - /** GetSrvKeyspacesResponse srv_keyspaces */ - srv_keyspaces?: ({ [k: string]: topodata.ISrvKeyspace }|null); + /** GetTabletsResponse tablets */ + tablets?: (topodata.ITablet[]|null); } - /** Represents a GetSrvKeyspacesResponse. */ - class GetSrvKeyspacesResponse implements IGetSrvKeyspacesResponse { + /** Represents a GetTabletsResponse. */ + class GetTabletsResponse implements IGetTabletsResponse { /** - * Constructs a new GetSrvKeyspacesResponse. + * Constructs a new GetTabletsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvKeyspacesResponse); + constructor(properties?: vtctldata.IGetTabletsResponse); - /** GetSrvKeyspacesResponse srv_keyspaces. */ - public srv_keyspaces: { [k: string]: topodata.ISrvKeyspace }; + /** GetTabletsResponse tablets. */ + public tablets: topodata.ITablet[]; /** - * Creates a new GetSrvKeyspacesResponse instance using the specified properties. + * Creates a new GetTabletsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvKeyspacesResponse instance + * @returns GetTabletsResponse instance */ - public static create(properties?: vtctldata.IGetSrvKeyspacesResponse): vtctldata.GetSrvKeyspacesResponse; + public static create(properties?: vtctldata.IGetTabletsResponse): vtctldata.GetTabletsResponse; /** - * Encodes the specified GetSrvKeyspacesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. - * @param message GetSrvKeyspacesResponse message or plain object to encode + * Encodes the specified GetTabletsResponse message. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. + * @param message GetTabletsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. - * @param message GetSrvKeyspacesResponse message or plain object to encode + * Encodes the specified GetTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. + * @param message GetTabletsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvKeyspacesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer. + * Decodes a GetTabletsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvKeyspacesResponse + * @returns GetTabletsResponse * @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.GetSrvKeyspacesResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTabletsResponse; /** - * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTabletsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvKeyspacesResponse + * @returns GetTabletsResponse * @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.GetSrvKeyspacesResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTabletsResponse; /** - * Verifies a GetSrvKeyspacesResponse message. + * Verifies a GetTabletsResponse 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 GetSrvKeyspacesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvKeyspacesResponse + * @returns GetTabletsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvKeyspacesResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletsResponse; /** - * Creates a plain object from a GetSrvKeyspacesResponse message. Also converts values to other types if specified. - * @param message GetSrvKeyspacesResponse + * Creates a plain object from a GetTabletsResponse message. Also converts values to other types if specified. + * @param message GetTabletsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvKeyspacesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTabletsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvKeyspacesResponse to JSON. + * Converts this GetTabletsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvKeyspacesResponse + * Gets the default type url for GetTabletsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateThrottlerConfigRequest. */ - interface IUpdateThrottlerConfigRequest { - - /** UpdateThrottlerConfigRequest keyspace */ - keyspace?: (string|null); - - /** UpdateThrottlerConfigRequest enable */ - enable?: (boolean|null); - - /** UpdateThrottlerConfigRequest disable */ - disable?: (boolean|null); - - /** UpdateThrottlerConfigRequest threshold */ - threshold?: (number|null); - - /** UpdateThrottlerConfigRequest custom_query */ - custom_query?: (string|null); - - /** UpdateThrottlerConfigRequest custom_query_set */ - custom_query_set?: (boolean|null); - - /** UpdateThrottlerConfigRequest check_as_check_self */ - check_as_check_self?: (boolean|null); - - /** UpdateThrottlerConfigRequest check_as_check_shard */ - check_as_check_shard?: (boolean|null); + /** Properties of a GetTopologyPathRequest. */ + interface IGetTopologyPathRequest { - /** UpdateThrottlerConfigRequest throttled_app */ - throttled_app?: (topodata.IThrottledAppRule|null); + /** GetTopologyPathRequest path */ + path?: (string|null); } - /** Represents an UpdateThrottlerConfigRequest. */ - class UpdateThrottlerConfigRequest implements IUpdateThrottlerConfigRequest { + /** Represents a GetTopologyPathRequest. */ + class GetTopologyPathRequest implements IGetTopologyPathRequest { /** - * Constructs a new UpdateThrottlerConfigRequest. + * Constructs a new GetTopologyPathRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateThrottlerConfigRequest); - - /** UpdateThrottlerConfigRequest keyspace. */ - public keyspace: string; - - /** UpdateThrottlerConfigRequest enable. */ - public enable: boolean; - - /** UpdateThrottlerConfigRequest disable. */ - public disable: boolean; - - /** UpdateThrottlerConfigRequest threshold. */ - public threshold: number; - - /** UpdateThrottlerConfigRequest custom_query. */ - public custom_query: string; - - /** UpdateThrottlerConfigRequest custom_query_set. */ - public custom_query_set: boolean; - - /** UpdateThrottlerConfigRequest check_as_check_self. */ - public check_as_check_self: boolean; - - /** UpdateThrottlerConfigRequest check_as_check_shard. */ - public check_as_check_shard: boolean; + constructor(properties?: vtctldata.IGetTopologyPathRequest); - /** UpdateThrottlerConfigRequest throttled_app. */ - public throttled_app?: (topodata.IThrottledAppRule|null); + /** GetTopologyPathRequest path. */ + public path: string; /** - * Creates a new UpdateThrottlerConfigRequest instance using the specified properties. + * Creates a new GetTopologyPathRequest instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateThrottlerConfigRequest instance + * @returns GetTopologyPathRequest instance */ - public static create(properties?: vtctldata.IUpdateThrottlerConfigRequest): vtctldata.UpdateThrottlerConfigRequest; + public static create(properties?: vtctldata.IGetTopologyPathRequest): vtctldata.GetTopologyPathRequest; /** - * Encodes the specified UpdateThrottlerConfigRequest message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. - * @param message UpdateThrottlerConfigRequest message or plain object to encode + * Encodes the specified GetTopologyPathRequest message. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. + * @param message GetTopologyPathRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateThrottlerConfigRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTopologyPathRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateThrottlerConfigRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. - * @param message UpdateThrottlerConfigRequest message or plain object to encode + * Encodes the specified GetTopologyPathRequest message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. + * @param message GetTopologyPathRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateThrottlerConfigRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTopologyPathRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer. + * Decodes a GetTopologyPathRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateThrottlerConfigRequest + * @returns GetTopologyPathRequest * @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.UpdateThrottlerConfigRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTopologyPathRequest; /** - * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTopologyPathRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateThrottlerConfigRequest + * @returns GetTopologyPathRequest * @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.UpdateThrottlerConfigRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTopologyPathRequest; /** - * Verifies an UpdateThrottlerConfigRequest message. + * Verifies a GetTopologyPathRequest 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 UpdateThrottlerConfigRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTopologyPathRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateThrottlerConfigRequest + * @returns GetTopologyPathRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateThrottlerConfigRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTopologyPathRequest; /** - * Creates a plain object from an UpdateThrottlerConfigRequest message. Also converts values to other types if specified. - * @param message UpdateThrottlerConfigRequest + * Creates a plain object from a GetTopologyPathRequest message. Also converts values to other types if specified. + * @param message GetTopologyPathRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateThrottlerConfigRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTopologyPathRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateThrottlerConfigRequest to JSON. + * Converts this GetTopologyPathRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateThrottlerConfigRequest + * Gets the default type url for GetTopologyPathRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateThrottlerConfigResponse. */ - interface IUpdateThrottlerConfigResponse { + /** Properties of a GetTopologyPathResponse. */ + interface IGetTopologyPathResponse { + + /** GetTopologyPathResponse cell */ + cell?: (vtctldata.ITopologyCell|null); } - /** Represents an UpdateThrottlerConfigResponse. */ - class UpdateThrottlerConfigResponse implements IUpdateThrottlerConfigResponse { + /** Represents a GetTopologyPathResponse. */ + class GetTopologyPathResponse implements IGetTopologyPathResponse { /** - * Constructs a new UpdateThrottlerConfigResponse. + * Constructs a new GetTopologyPathResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateThrottlerConfigResponse); + constructor(properties?: vtctldata.IGetTopologyPathResponse); + + /** GetTopologyPathResponse cell. */ + public cell?: (vtctldata.ITopologyCell|null); /** - * Creates a new UpdateThrottlerConfigResponse instance using the specified properties. + * Creates a new GetTopologyPathResponse instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateThrottlerConfigResponse instance + * @returns GetTopologyPathResponse instance */ - public static create(properties?: vtctldata.IUpdateThrottlerConfigResponse): vtctldata.UpdateThrottlerConfigResponse; + public static create(properties?: vtctldata.IGetTopologyPathResponse): vtctldata.GetTopologyPathResponse; /** - * Encodes the specified UpdateThrottlerConfigResponse message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. - * @param message UpdateThrottlerConfigResponse message or plain object to encode + * Encodes the specified GetTopologyPathResponse message. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. + * @param message GetTopologyPathResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateThrottlerConfigResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetTopologyPathResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateThrottlerConfigResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. - * @param message UpdateThrottlerConfigResponse message or plain object to encode + * Encodes the specified GetTopologyPathResponse message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. + * @param message GetTopologyPathResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateThrottlerConfigResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetTopologyPathResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer. + * Decodes a GetTopologyPathResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateThrottlerConfigResponse + * @returns GetTopologyPathResponse * @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.UpdateThrottlerConfigResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetTopologyPathResponse; /** - * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTopologyPathResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateThrottlerConfigResponse + * @returns GetTopologyPathResponse * @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.UpdateThrottlerConfigResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetTopologyPathResponse; /** - * Verifies an UpdateThrottlerConfigResponse message. + * Verifies a GetTopologyPathResponse 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 UpdateThrottlerConfigResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTopologyPathResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateThrottlerConfigResponse + * @returns GetTopologyPathResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateThrottlerConfigResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetTopologyPathResponse; /** - * Creates a plain object from an UpdateThrottlerConfigResponse message. Also converts values to other types if specified. - * @param message UpdateThrottlerConfigResponse + * Creates a plain object from a GetTopologyPathResponse message. Also converts values to other types if specified. + * @param message GetTopologyPathResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateThrottlerConfigResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetTopologyPathResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateThrottlerConfigResponse to JSON. + * Converts this GetTopologyPathResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateThrottlerConfigResponse + * Gets the default type url for GetTopologyPathResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvVSchemaRequest. */ - interface IGetSrvVSchemaRequest { + /** Properties of a TopologyCell. */ + interface ITopologyCell { - /** GetSrvVSchemaRequest cell */ - cell?: (string|null); + /** TopologyCell name */ + name?: (string|null); + + /** TopologyCell path */ + path?: (string|null); + + /** TopologyCell data */ + data?: (string|null); + + /** TopologyCell children */ + children?: (string[]|null); } - /** Represents a GetSrvVSchemaRequest. */ - class GetSrvVSchemaRequest implements IGetSrvVSchemaRequest { + /** Represents a TopologyCell. */ + class TopologyCell implements ITopologyCell { /** - * Constructs a new GetSrvVSchemaRequest. + * Constructs a new TopologyCell. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvVSchemaRequest); + constructor(properties?: vtctldata.ITopologyCell); - /** GetSrvVSchemaRequest cell. */ - public cell: string; + /** TopologyCell name. */ + public name: string; + + /** TopologyCell path. */ + public path: string; + + /** TopologyCell data. */ + public data: string; + + /** TopologyCell children. */ + public children: string[]; /** - * Creates a new GetSrvVSchemaRequest instance using the specified properties. + * Creates a new TopologyCell instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvVSchemaRequest instance + * @returns TopologyCell instance */ - public static create(properties?: vtctldata.IGetSrvVSchemaRequest): vtctldata.GetSrvVSchemaRequest; + public static create(properties?: vtctldata.ITopologyCell): vtctldata.TopologyCell; /** - * Encodes the specified GetSrvVSchemaRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. - * @param message GetSrvVSchemaRequest message or plain object to encode + * Encodes the specified TopologyCell message. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. + * @param message TopologyCell message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ITopologyCell, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. - * @param message GetSrvVSchemaRequest message or plain object to encode + * Encodes the specified TopologyCell message, length delimited. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. + * @param message TopologyCell message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ITopologyCell, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer. + * Decodes a TopologyCell message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvVSchemaRequest + * @returns TopologyCell * @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.GetSrvVSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.TopologyCell; /** - * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a TopologyCell message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvVSchemaRequest + * @returns TopologyCell * @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.GetSrvVSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.TopologyCell; /** - * Verifies a GetSrvVSchemaRequest message. + * Verifies a TopologyCell 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 GetSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a TopologyCell message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvVSchemaRequest + * @returns TopologyCell */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.TopologyCell; /** - * Creates a plain object from a GetSrvVSchemaRequest message. Also converts values to other types if specified. - * @param message GetSrvVSchemaRequest + * Creates a plain object from a TopologyCell message. Also converts values to other types if specified. + * @param message TopologyCell * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.TopologyCell, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvVSchemaRequest to JSON. + * Converts this TopologyCell to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvVSchemaRequest + * Gets the default type url for TopologyCell * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvVSchemaResponse. */ - interface IGetSrvVSchemaResponse { + /** Properties of a GetVSchemaRequest. */ + interface IGetVSchemaRequest { - /** GetSrvVSchemaResponse srv_v_schema */ - srv_v_schema?: (vschema.ISrvVSchema|null); + /** GetVSchemaRequest keyspace */ + keyspace?: (string|null); } - /** Represents a GetSrvVSchemaResponse. */ - class GetSrvVSchemaResponse implements IGetSrvVSchemaResponse { + /** Represents a GetVSchemaRequest. */ + class GetVSchemaRequest implements IGetVSchemaRequest { /** - * Constructs a new GetSrvVSchemaResponse. + * Constructs a new GetVSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvVSchemaResponse); + constructor(properties?: vtctldata.IGetVSchemaRequest); - /** GetSrvVSchemaResponse srv_v_schema. */ - public srv_v_schema?: (vschema.ISrvVSchema|null); + /** GetVSchemaRequest keyspace. */ + public keyspace: string; /** - * Creates a new GetSrvVSchemaResponse instance using the specified properties. + * Creates a new GetVSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvVSchemaResponse instance + * @returns GetVSchemaRequest instance */ - public static create(properties?: vtctldata.IGetSrvVSchemaResponse): vtctldata.GetSrvVSchemaResponse; + public static create(properties?: vtctldata.IGetVSchemaRequest): vtctldata.GetVSchemaRequest; /** - * Encodes the specified GetSrvVSchemaResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. - * @param message GetSrvVSchemaResponse message or plain object to encode + * Encodes the specified GetVSchemaRequest message. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. + * @param message GetVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. - * @param message GetSrvVSchemaResponse message or plain object to encode + * Encodes the specified GetVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. + * @param message GetVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer. + * Decodes a GetVSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvVSchemaResponse + * @returns GetVSchemaRequest * @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.GetSrvVSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetVSchemaRequest; /** - * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a GetVSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvVSchemaResponse + * @returns GetVSchemaRequest * @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.GetSrvVSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetVSchemaRequest; /** - * Verifies a GetSrvVSchemaResponse message. + * Verifies a GetVSchemaRequest 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 GetSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvVSchemaResponse + * @returns GetVSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetVSchemaRequest; /** - * Creates a plain object from a GetSrvVSchemaResponse message. Also converts values to other types if specified. - * @param message GetSrvVSchemaResponse + * Creates a plain object from a GetVSchemaRequest message. Also converts values to other types if specified. + * @param message GetVSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvVSchemaResponse to JSON. + * Converts this GetVSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvVSchemaResponse + * Gets the default type url for GetVSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvVSchemasRequest. */ - interface IGetSrvVSchemasRequest { + /** Properties of a GetVersionRequest. */ + interface IGetVersionRequest { - /** GetSrvVSchemasRequest cells */ - cells?: (string[]|null); + /** GetVersionRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a GetSrvVSchemasRequest. */ - class GetSrvVSchemasRequest implements IGetSrvVSchemasRequest { + /** Represents a GetVersionRequest. */ + class GetVersionRequest implements IGetVersionRequest { /** - * Constructs a new GetSrvVSchemasRequest. + * Constructs a new GetVersionRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvVSchemasRequest); + constructor(properties?: vtctldata.IGetVersionRequest); - /** GetSrvVSchemasRequest cells. */ - public cells: string[]; + /** GetVersionRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new GetSrvVSchemasRequest instance using the specified properties. + * Creates a new GetVersionRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvVSchemasRequest instance + * @returns GetVersionRequest instance */ - public static create(properties?: vtctldata.IGetSrvVSchemasRequest): vtctldata.GetSrvVSchemasRequest; + public static create(properties?: vtctldata.IGetVersionRequest): vtctldata.GetVersionRequest; /** - * Encodes the specified GetSrvVSchemasRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. - * @param message GetSrvVSchemasRequest message or plain object to encode + * Encodes the specified GetVersionRequest message. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. + * @param message GetVersionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvVSchemasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetVersionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvVSchemasRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. - * @param message GetSrvVSchemasRequest message or plain object to encode + * Encodes the specified GetVersionRequest message, length delimited. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. + * @param message GetVersionRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvVSchemasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetVersionRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer. + * Decodes a GetVersionRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvVSchemasRequest + * @returns GetVersionRequest * @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.GetSrvVSchemasRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetVersionRequest; /** - * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer, length delimited. + * Decodes a GetVersionRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvVSchemasRequest + * @returns GetVersionRequest * @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.GetSrvVSchemasRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetVersionRequest; /** - * Verifies a GetSrvVSchemasRequest message. + * Verifies a GetVersionRequest 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 GetSrvVSchemasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetVersionRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvVSchemasRequest + * @returns GetVersionRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemasRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetVersionRequest; /** - * Creates a plain object from a GetSrvVSchemasRequest message. Also converts values to other types if specified. - * @param message GetSrvVSchemasRequest + * Creates a plain object from a GetVersionRequest message. Also converts values to other types if specified. + * @param message GetVersionRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvVSchemasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetVersionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvVSchemasRequest to JSON. + * Converts this GetVersionRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvVSchemasRequest + * Gets the default type url for GetVersionRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetSrvVSchemasResponse. */ - interface IGetSrvVSchemasResponse { + /** Properties of a GetVersionResponse. */ + interface IGetVersionResponse { - /** GetSrvVSchemasResponse srv_v_schemas */ - srv_v_schemas?: ({ [k: string]: vschema.ISrvVSchema }|null); + /** GetVersionResponse version */ + version?: (string|null); } - /** Represents a GetSrvVSchemasResponse. */ - class GetSrvVSchemasResponse implements IGetSrvVSchemasResponse { + /** Represents a GetVersionResponse. */ + class GetVersionResponse implements IGetVersionResponse { /** - * Constructs a new GetSrvVSchemasResponse. + * Constructs a new GetVersionResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetSrvVSchemasResponse); + constructor(properties?: vtctldata.IGetVersionResponse); - /** GetSrvVSchemasResponse srv_v_schemas. */ - public srv_v_schemas: { [k: string]: vschema.ISrvVSchema }; + /** GetVersionResponse version. */ + public version: string; /** - * Creates a new GetSrvVSchemasResponse instance using the specified properties. + * Creates a new GetVersionResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetSrvVSchemasResponse instance + * @returns GetVersionResponse instance */ - public static create(properties?: vtctldata.IGetSrvVSchemasResponse): vtctldata.GetSrvVSchemasResponse; + public static create(properties?: vtctldata.IGetVersionResponse): vtctldata.GetVersionResponse; /** - * Encodes the specified GetSrvVSchemasResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. - * @param message GetSrvVSchemasResponse message or plain object to encode + * Encodes the specified GetVersionResponse message. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. + * @param message GetVersionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetSrvVSchemasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetVersionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetSrvVSchemasResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. - * @param message GetSrvVSchemasResponse message or plain object to encode + * Encodes the specified GetVersionResponse message, length delimited. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. + * @param message GetVersionResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetSrvVSchemasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetVersionResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer. + * Decodes a GetVersionResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetSrvVSchemasResponse + * @returns GetVersionResponse * @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.GetSrvVSchemasResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetVersionResponse; /** - * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer, length delimited. + * Decodes a GetVersionResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetSrvVSchemasResponse + * @returns GetVersionResponse * @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.GetSrvVSchemasResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetVersionResponse; /** - * Verifies a GetSrvVSchemasResponse message. + * Verifies a GetVersionResponse 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 GetSrvVSchemasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetVersionResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetSrvVSchemasResponse + * @returns GetVersionResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetSrvVSchemasResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetVersionResponse; /** - * Creates a plain object from a GetSrvVSchemasResponse message. Also converts values to other types if specified. - * @param message GetSrvVSchemasResponse + * Creates a plain object from a GetVersionResponse message. Also converts values to other types if specified. + * @param message GetVersionResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetSrvVSchemasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetVersionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetSrvVSchemasResponse to JSON. + * Converts this GetVersionResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetSrvVSchemasResponse + * Gets the default type url for GetVersionResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTabletRequest. */ - interface IGetTabletRequest { + /** Properties of a GetVSchemaResponse. */ + interface IGetVSchemaResponse { - /** GetTabletRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** GetVSchemaResponse v_schema */ + v_schema?: (vschema.IKeyspace|null); } - /** Represents a GetTabletRequest. */ - class GetTabletRequest implements IGetTabletRequest { + /** Represents a GetVSchemaResponse. */ + class GetVSchemaResponse implements IGetVSchemaResponse { /** - * Constructs a new GetTabletRequest. + * Constructs a new GetVSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTabletRequest); + constructor(properties?: vtctldata.IGetVSchemaResponse); - /** GetTabletRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** GetVSchemaResponse v_schema. */ + public v_schema?: (vschema.IKeyspace|null); /** - * Creates a new GetTabletRequest instance using the specified properties. + * Creates a new GetVSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetTabletRequest instance + * @returns GetVSchemaResponse instance */ - public static create(properties?: vtctldata.IGetTabletRequest): vtctldata.GetTabletRequest; + public static create(properties?: vtctldata.IGetVSchemaResponse): vtctldata.GetVSchemaResponse; /** - * Encodes the specified GetTabletRequest message. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. - * @param message GetTabletRequest message or plain object to encode + * Encodes the specified GetVSchemaResponse message. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. + * @param message GetVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTabletRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. - * @param message GetTabletRequest message or plain object to encode + * Encodes the specified GetVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. + * @param message GetVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTabletRequest message from the specified reader or buffer. + * Decodes a GetVSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTabletRequest + * @returns GetVSchemaResponse * @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.GetTabletRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetVSchemaResponse; /** - * Decodes a GetTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a GetVSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTabletRequest + * @returns GetVSchemaResponse * @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.GetTabletRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetVSchemaResponse; /** - * Verifies a GetTabletRequest message. + * Verifies a GetVSchemaResponse 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 GetTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTabletRequest + * @returns GetVSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetVSchemaResponse; /** - * Creates a plain object from a GetTabletRequest message. Also converts values to other types if specified. - * @param message GetTabletRequest + * Creates a plain object from a GetVSchemaResponse message. Also converts values to other types if specified. + * @param message GetVSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTabletRequest to JSON. + * Converts this GetVSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTabletRequest + * Gets the default type url for GetVSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTabletResponse. */ - interface IGetTabletResponse { + /** Properties of a GetWorkflowsRequest. */ + interface IGetWorkflowsRequest { - /** GetTabletResponse tablet */ - tablet?: (topodata.ITablet|null); + /** GetWorkflowsRequest keyspace */ + keyspace?: (string|null); + + /** GetWorkflowsRequest active_only */ + active_only?: (boolean|null); + + /** GetWorkflowsRequest name_only */ + name_only?: (boolean|null); + + /** GetWorkflowsRequest workflow */ + workflow?: (string|null); } - /** Represents a GetTabletResponse. */ - class GetTabletResponse implements IGetTabletResponse { + /** Represents a GetWorkflowsRequest. */ + class GetWorkflowsRequest implements IGetWorkflowsRequest { /** - * Constructs a new GetTabletResponse. + * Constructs a new GetWorkflowsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTabletResponse); + constructor(properties?: vtctldata.IGetWorkflowsRequest); - /** GetTabletResponse tablet. */ - public tablet?: (topodata.ITablet|null); + /** GetWorkflowsRequest keyspace. */ + public keyspace: string; + + /** GetWorkflowsRequest active_only. */ + public active_only: boolean; + + /** GetWorkflowsRequest name_only. */ + public name_only: boolean; + + /** GetWorkflowsRequest workflow. */ + public workflow: string; /** - * Creates a new GetTabletResponse instance using the specified properties. + * Creates a new GetWorkflowsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetTabletResponse instance + * @returns GetWorkflowsRequest instance */ - public static create(properties?: vtctldata.IGetTabletResponse): vtctldata.GetTabletResponse; + public static create(properties?: vtctldata.IGetWorkflowsRequest): vtctldata.GetWorkflowsRequest; /** - * Encodes the specified GetTabletResponse message. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. - * @param message GetTabletResponse message or plain object to encode + * Encodes the specified GetWorkflowsRequest message. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. + * @param message GetWorkflowsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTabletResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. - * @param message GetTabletResponse message or plain object to encode + * Encodes the specified GetWorkflowsRequest message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. + * @param message GetWorkflowsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTabletResponse message from the specified reader or buffer. + * Decodes a GetWorkflowsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTabletResponse + * @returns GetWorkflowsRequest * @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.GetTabletResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetWorkflowsRequest; /** - * Decodes a GetTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a GetWorkflowsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTabletResponse + * @returns GetWorkflowsRequest * @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.GetTabletResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetWorkflowsRequest; /** - * Verifies a GetTabletResponse message. + * Verifies a GetWorkflowsRequest 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 GetTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetWorkflowsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTabletResponse + * @returns GetWorkflowsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.GetWorkflowsRequest; /** - * Creates a plain object from a GetTabletResponse message. Also converts values to other types if specified. - * @param message GetTabletResponse + * Creates a plain object from a GetWorkflowsRequest message. Also converts values to other types if specified. + * @param message GetWorkflowsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetWorkflowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTabletResponse to JSON. + * Converts this GetWorkflowsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTabletResponse + * Gets the default type url for GetWorkflowsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTabletsRequest. */ - interface IGetTabletsRequest { - - /** GetTabletsRequest keyspace */ - keyspace?: (string|null); - - /** GetTabletsRequest shard */ - shard?: (string|null); - - /** GetTabletsRequest cells */ - cells?: (string[]|null); - - /** GetTabletsRequest strict */ - strict?: (boolean|null); - - /** GetTabletsRequest tablet_aliases */ - tablet_aliases?: (topodata.ITabletAlias[]|null); + /** Properties of a GetWorkflowsResponse. */ + interface IGetWorkflowsResponse { - /** GetTabletsRequest tablet_type */ - tablet_type?: (topodata.TabletType|null); + /** GetWorkflowsResponse workflows */ + workflows?: (vtctldata.IWorkflow[]|null); } - /** Represents a GetTabletsRequest. */ - class GetTabletsRequest implements IGetTabletsRequest { + /** Represents a GetWorkflowsResponse. */ + class GetWorkflowsResponse implements IGetWorkflowsResponse { /** - * Constructs a new GetTabletsRequest. + * Constructs a new GetWorkflowsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTabletsRequest); - - /** GetTabletsRequest keyspace. */ - public keyspace: string; - - /** GetTabletsRequest shard. */ - public shard: string; - - /** GetTabletsRequest cells. */ - public cells: string[]; - - /** GetTabletsRequest strict. */ - public strict: boolean; - - /** GetTabletsRequest tablet_aliases. */ - public tablet_aliases: topodata.ITabletAlias[]; + constructor(properties?: vtctldata.IGetWorkflowsResponse); - /** GetTabletsRequest tablet_type. */ - public tablet_type: topodata.TabletType; + /** GetWorkflowsResponse workflows. */ + public workflows: vtctldata.IWorkflow[]; /** - * Creates a new GetTabletsRequest instance using the specified properties. + * Creates a new GetWorkflowsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetTabletsRequest instance + * @returns GetWorkflowsResponse instance */ - public static create(properties?: vtctldata.IGetTabletsRequest): vtctldata.GetTabletsRequest; + public static create(properties?: vtctldata.IGetWorkflowsResponse): vtctldata.GetWorkflowsResponse; /** - * Encodes the specified GetTabletsRequest message. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. - * @param message GetTabletsRequest message or plain object to encode + * Encodes the specified GetWorkflowsResponse message. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. + * @param message GetWorkflowsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IGetWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. - * @param message GetTabletsRequest message or plain object to encode + * Encodes the specified GetWorkflowsResponse message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. + * @param message GetWorkflowsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTabletsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IGetWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTabletsRequest message from the specified reader or buffer. + * Decodes a GetWorkflowsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTabletsRequest + * @returns GetWorkflowsResponse * @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.GetTabletsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.GetWorkflowsResponse; /** - * Decodes a GetTabletsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetWorkflowsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTabletsRequest + * @returns GetWorkflowsResponse * @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.GetTabletsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.GetWorkflowsResponse; /** - * Verifies a GetTabletsRequest message. + * Verifies a GetWorkflowsResponse 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 GetTabletsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetWorkflowsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTabletsRequest + * @returns GetWorkflowsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.GetWorkflowsResponse; /** - * Creates a plain object from a GetTabletsRequest message. Also converts values to other types if specified. - * @param message GetTabletsRequest + * Creates a plain object from a GetWorkflowsResponse message. Also converts values to other types if specified. + * @param message GetWorkflowsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTabletsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.GetWorkflowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTabletsRequest to JSON. + * Converts this GetWorkflowsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTabletsRequest + * Gets the default type url for GetWorkflowsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTabletsResponse. */ - interface IGetTabletsResponse { + /** Properties of an InitShardPrimaryRequest. */ + interface IInitShardPrimaryRequest { + + /** InitShardPrimaryRequest keyspace */ + keyspace?: (string|null); + + /** InitShardPrimaryRequest shard */ + shard?: (string|null); + + /** InitShardPrimaryRequest primary_elect_tablet_alias */ + primary_elect_tablet_alias?: (topodata.ITabletAlias|null); + + /** InitShardPrimaryRequest force */ + force?: (boolean|null); - /** GetTabletsResponse tablets */ - tablets?: (topodata.ITablet[]|null); + /** InitShardPrimaryRequest wait_replicas_timeout */ + wait_replicas_timeout?: (vttime.IDuration|null); } - /** Represents a GetTabletsResponse. */ - class GetTabletsResponse implements IGetTabletsResponse { + /** Represents an InitShardPrimaryRequest. */ + class InitShardPrimaryRequest implements IInitShardPrimaryRequest { /** - * Constructs a new GetTabletsResponse. + * Constructs a new InitShardPrimaryRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTabletsResponse); + constructor(properties?: vtctldata.IInitShardPrimaryRequest); - /** GetTabletsResponse tablets. */ - public tablets: topodata.ITablet[]; + /** InitShardPrimaryRequest keyspace. */ + public keyspace: string; + + /** InitShardPrimaryRequest shard. */ + public shard: string; + + /** InitShardPrimaryRequest primary_elect_tablet_alias. */ + public primary_elect_tablet_alias?: (topodata.ITabletAlias|null); + + /** InitShardPrimaryRequest force. */ + public force: boolean; + + /** InitShardPrimaryRequest wait_replicas_timeout. */ + public wait_replicas_timeout?: (vttime.IDuration|null); /** - * Creates a new GetTabletsResponse instance using the specified properties. + * Creates a new InitShardPrimaryRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetTabletsResponse instance + * @returns InitShardPrimaryRequest instance */ - public static create(properties?: vtctldata.IGetTabletsResponse): vtctldata.GetTabletsResponse; + public static create(properties?: vtctldata.IInitShardPrimaryRequest): vtctldata.InitShardPrimaryRequest; /** - * Encodes the specified GetTabletsResponse message. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. - * @param message GetTabletsResponse message or plain object to encode + * Encodes the specified InitShardPrimaryRequest message. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. + * @param message InitShardPrimaryRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IInitShardPrimaryRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. - * @param message GetTabletsResponse message or plain object to encode + * Encodes the specified InitShardPrimaryRequest message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. + * @param message InitShardPrimaryRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTabletsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IInitShardPrimaryRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTabletsResponse message from the specified reader or buffer. + * Decodes an InitShardPrimaryRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTabletsResponse + * @returns InitShardPrimaryRequest * @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.GetTabletsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.InitShardPrimaryRequest; /** - * Decodes a GetTabletsResponse message from the specified reader or buffer, length delimited. + * Decodes an InitShardPrimaryRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTabletsResponse + * @returns InitShardPrimaryRequest * @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.GetTabletsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.InitShardPrimaryRequest; /** - * Verifies a GetTabletsResponse message. + * Verifies an InitShardPrimaryRequest 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 GetTabletsResponse message from a plain object. Also converts values to their respective internal types. + * Creates an InitShardPrimaryRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTabletsResponse + * @returns InitShardPrimaryRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTabletsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.InitShardPrimaryRequest; /** - * Creates a plain object from a GetTabletsResponse message. Also converts values to other types if specified. - * @param message GetTabletsResponse + * Creates a plain object from an InitShardPrimaryRequest message. Also converts values to other types if specified. + * @param message InitShardPrimaryRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTabletsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.InitShardPrimaryRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTabletsResponse to JSON. + * Converts this InitShardPrimaryRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTabletsResponse + * Gets the default type url for InitShardPrimaryRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTopologyPathRequest. */ - interface IGetTopologyPathRequest { + /** Properties of an InitShardPrimaryResponse. */ + interface IInitShardPrimaryResponse { - /** GetTopologyPathRequest path */ - path?: (string|null); + /** InitShardPrimaryResponse events */ + events?: (logutil.IEvent[]|null); } - /** Represents a GetTopologyPathRequest. */ - class GetTopologyPathRequest implements IGetTopologyPathRequest { + /** Represents an InitShardPrimaryResponse. */ + class InitShardPrimaryResponse implements IInitShardPrimaryResponse { /** - * Constructs a new GetTopologyPathRequest. + * Constructs a new InitShardPrimaryResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTopologyPathRequest); + constructor(properties?: vtctldata.IInitShardPrimaryResponse); - /** GetTopologyPathRequest path. */ - public path: string; + /** InitShardPrimaryResponse events. */ + public events: logutil.IEvent[]; /** - * Creates a new GetTopologyPathRequest instance using the specified properties. + * Creates a new InitShardPrimaryResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetTopologyPathRequest instance + * @returns InitShardPrimaryResponse instance */ - public static create(properties?: vtctldata.IGetTopologyPathRequest): vtctldata.GetTopologyPathRequest; + public static create(properties?: vtctldata.IInitShardPrimaryResponse): vtctldata.InitShardPrimaryResponse; /** - * Encodes the specified GetTopologyPathRequest message. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. - * @param message GetTopologyPathRequest message or plain object to encode + * Encodes the specified InitShardPrimaryResponse message. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. + * @param message InitShardPrimaryResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTopologyPathRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IInitShardPrimaryResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTopologyPathRequest message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. - * @param message GetTopologyPathRequest message or plain object to encode + * Encodes the specified InitShardPrimaryResponse message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. + * @param message InitShardPrimaryResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTopologyPathRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IInitShardPrimaryResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTopologyPathRequest message from the specified reader or buffer. + * Decodes an InitShardPrimaryResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTopologyPathRequest + * @returns InitShardPrimaryResponse * @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.GetTopologyPathRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.InitShardPrimaryResponse; /** - * Decodes a GetTopologyPathRequest message from the specified reader or buffer, length delimited. + * Decodes an InitShardPrimaryResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTopologyPathRequest + * @returns InitShardPrimaryResponse * @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.GetTopologyPathRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.InitShardPrimaryResponse; /** - * Verifies a GetTopologyPathRequest message. + * Verifies an InitShardPrimaryResponse 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 GetTopologyPathRequest message from a plain object. Also converts values to their respective internal types. + * Creates an InitShardPrimaryResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTopologyPathRequest + * @returns InitShardPrimaryResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTopologyPathRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.InitShardPrimaryResponse; /** - * Creates a plain object from a GetTopologyPathRequest message. Also converts values to other types if specified. - * @param message GetTopologyPathRequest + * Creates a plain object from an InitShardPrimaryResponse message. Also converts values to other types if specified. + * @param message InitShardPrimaryResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTopologyPathRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.InitShardPrimaryResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTopologyPathRequest to JSON. + * Converts this InitShardPrimaryResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTopologyPathRequest + * Gets the default type url for InitShardPrimaryResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetTopologyPathResponse. */ - interface IGetTopologyPathResponse { + /** Properties of a MoveTablesCreateRequest. */ + interface IMoveTablesCreateRequest { - /** GetTopologyPathResponse cell */ - cell?: (vtctldata.ITopologyCell|null); + /** MoveTablesCreateRequest workflow */ + workflow?: (string|null); + + /** MoveTablesCreateRequest source_keyspace */ + source_keyspace?: (string|null); + + /** MoveTablesCreateRequest target_keyspace */ + target_keyspace?: (string|null); + + /** MoveTablesCreateRequest cells */ + cells?: (string[]|null); + + /** MoveTablesCreateRequest tablet_types */ + tablet_types?: (topodata.TabletType[]|null); + + /** MoveTablesCreateRequest tablet_selection_preference */ + tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); + + /** MoveTablesCreateRequest source_shards */ + source_shards?: (string[]|null); + + /** MoveTablesCreateRequest all_tables */ + all_tables?: (boolean|null); + + /** MoveTablesCreateRequest include_tables */ + include_tables?: (string[]|null); + + /** MoveTablesCreateRequest exclude_tables */ + exclude_tables?: (string[]|null); + + /** MoveTablesCreateRequest external_cluster_name */ + external_cluster_name?: (string|null); + + /** MoveTablesCreateRequest source_time_zone */ + source_time_zone?: (string|null); + + /** MoveTablesCreateRequest on_ddl */ + on_ddl?: (string|null); + + /** MoveTablesCreateRequest stop_after_copy */ + stop_after_copy?: (boolean|null); + + /** MoveTablesCreateRequest drop_foreign_keys */ + drop_foreign_keys?: (boolean|null); + + /** MoveTablesCreateRequest defer_secondary_keys */ + defer_secondary_keys?: (boolean|null); + + /** MoveTablesCreateRequest auto_start */ + auto_start?: (boolean|null); } - /** Represents a GetTopologyPathResponse. */ - class GetTopologyPathResponse implements IGetTopologyPathResponse { + /** Represents a MoveTablesCreateRequest. */ + class MoveTablesCreateRequest implements IMoveTablesCreateRequest { /** - * Constructs a new GetTopologyPathResponse. + * Constructs a new MoveTablesCreateRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetTopologyPathResponse); + constructor(properties?: vtctldata.IMoveTablesCreateRequest); - /** GetTopologyPathResponse cell. */ - public cell?: (vtctldata.ITopologyCell|null); + /** MoveTablesCreateRequest workflow. */ + public workflow: string; + + /** MoveTablesCreateRequest source_keyspace. */ + public source_keyspace: string; + + /** MoveTablesCreateRequest target_keyspace. */ + public target_keyspace: string; + + /** MoveTablesCreateRequest cells. */ + public cells: string[]; + + /** MoveTablesCreateRequest tablet_types. */ + public tablet_types: topodata.TabletType[]; + + /** MoveTablesCreateRequest tablet_selection_preference. */ + public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; + + /** MoveTablesCreateRequest source_shards. */ + public source_shards: string[]; + + /** MoveTablesCreateRequest all_tables. */ + public all_tables: boolean; + + /** MoveTablesCreateRequest include_tables. */ + public include_tables: string[]; + + /** MoveTablesCreateRequest exclude_tables. */ + public exclude_tables: string[]; + + /** MoveTablesCreateRequest external_cluster_name. */ + public external_cluster_name: string; + + /** MoveTablesCreateRequest source_time_zone. */ + public source_time_zone: string; + + /** MoveTablesCreateRequest on_ddl. */ + public on_ddl: string; + + /** MoveTablesCreateRequest stop_after_copy. */ + public stop_after_copy: boolean; + + /** MoveTablesCreateRequest drop_foreign_keys. */ + public drop_foreign_keys: boolean; + + /** MoveTablesCreateRequest defer_secondary_keys. */ + public defer_secondary_keys: boolean; + + /** MoveTablesCreateRequest auto_start. */ + public auto_start: boolean; /** - * Creates a new GetTopologyPathResponse instance using the specified properties. + * Creates a new MoveTablesCreateRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetTopologyPathResponse instance + * @returns MoveTablesCreateRequest instance */ - public static create(properties?: vtctldata.IGetTopologyPathResponse): vtctldata.GetTopologyPathResponse; + public static create(properties?: vtctldata.IMoveTablesCreateRequest): vtctldata.MoveTablesCreateRequest; /** - * Encodes the specified GetTopologyPathResponse message. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. - * @param message GetTopologyPathResponse message or plain object to encode + * Encodes the specified MoveTablesCreateRequest message. Does not implicitly {@link vtctldata.MoveTablesCreateRequest.verify|verify} messages. + * @param message MoveTablesCreateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetTopologyPathResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IMoveTablesCreateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetTopologyPathResponse message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. - * @param message GetTopologyPathResponse message or plain object to encode + * Encodes the specified MoveTablesCreateRequest message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateRequest.verify|verify} messages. + * @param message MoveTablesCreateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetTopologyPathResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IMoveTablesCreateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetTopologyPathResponse message from the specified reader or buffer. + * Decodes a MoveTablesCreateRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetTopologyPathResponse + * @returns MoveTablesCreateRequest * @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.GetTopologyPathResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.MoveTablesCreateRequest; /** - * Decodes a GetTopologyPathResponse message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCreateRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetTopologyPathResponse + * @returns MoveTablesCreateRequest * @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.GetTopologyPathResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.MoveTablesCreateRequest; /** - * Verifies a GetTopologyPathResponse message. + * Verifies a MoveTablesCreateRequest 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 GetTopologyPathResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCreateRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetTopologyPathResponse + * @returns MoveTablesCreateRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetTopologyPathResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.MoveTablesCreateRequest; /** - * Creates a plain object from a GetTopologyPathResponse message. Also converts values to other types if specified. - * @param message GetTopologyPathResponse + * Creates a plain object from a MoveTablesCreateRequest message. Also converts values to other types if specified. + * @param message MoveTablesCreateRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetTopologyPathResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.MoveTablesCreateRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetTopologyPathResponse to JSON. + * Converts this MoveTablesCreateRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetTopologyPathResponse + * Gets the default type url for MoveTablesCreateRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a TopologyCell. */ - interface ITopologyCell { - - /** TopologyCell name */ - name?: (string|null); - - /** TopologyCell path */ - path?: (string|null); + /** Properties of a MoveTablesCreateResponse. */ + interface IMoveTablesCreateResponse { - /** TopologyCell data */ - data?: (string|null); + /** MoveTablesCreateResponse summary */ + summary?: (string|null); - /** TopologyCell children */ - children?: (string[]|null); + /** MoveTablesCreateResponse details */ + details?: (vtctldata.MoveTablesCreateResponse.ITabletInfo[]|null); } - /** Represents a TopologyCell. */ - class TopologyCell implements ITopologyCell { + /** Represents a MoveTablesCreateResponse. */ + class MoveTablesCreateResponse implements IMoveTablesCreateResponse { /** - * Constructs a new TopologyCell. + * Constructs a new MoveTablesCreateResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ITopologyCell); + constructor(properties?: vtctldata.IMoveTablesCreateResponse); - /** TopologyCell name. */ - public name: string; - - /** TopologyCell path. */ - public path: string; - - /** TopologyCell data. */ - public data: string; + /** MoveTablesCreateResponse summary. */ + public summary: string; - /** TopologyCell children. */ - public children: string[]; + /** MoveTablesCreateResponse details. */ + public details: vtctldata.MoveTablesCreateResponse.ITabletInfo[]; /** - * Creates a new TopologyCell instance using the specified properties. + * Creates a new MoveTablesCreateResponse instance using the specified properties. * @param [properties] Properties to set - * @returns TopologyCell instance + * @returns MoveTablesCreateResponse instance */ - public static create(properties?: vtctldata.ITopologyCell): vtctldata.TopologyCell; + public static create(properties?: vtctldata.IMoveTablesCreateResponse): vtctldata.MoveTablesCreateResponse; /** - * Encodes the specified TopologyCell message. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. - * @param message TopologyCell message or plain object to encode + * Encodes the specified MoveTablesCreateResponse message. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.verify|verify} messages. + * @param message MoveTablesCreateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ITopologyCell, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IMoveTablesCreateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TopologyCell message, length delimited. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. - * @param message TopologyCell message or plain object to encode + * Encodes the specified MoveTablesCreateResponse message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.verify|verify} messages. + * @param message MoveTablesCreateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ITopologyCell, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IMoveTablesCreateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TopologyCell message from the specified reader or buffer. + * Decodes a MoveTablesCreateResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TopologyCell + * @returns MoveTablesCreateResponse * @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.TopologyCell; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.MoveTablesCreateResponse; /** - * Decodes a TopologyCell message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCreateResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TopologyCell + * @returns MoveTablesCreateResponse * @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.TopologyCell; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.MoveTablesCreateResponse; /** - * Verifies a TopologyCell message. + * Verifies a MoveTablesCreateResponse 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 TopologyCell message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCreateResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TopologyCell + * @returns MoveTablesCreateResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.TopologyCell; + public static fromObject(object: { [k: string]: any }): vtctldata.MoveTablesCreateResponse; /** - * Creates a plain object from a TopologyCell message. Also converts values to other types if specified. - * @param message TopologyCell + * Creates a plain object from a MoveTablesCreateResponse message. Also converts values to other types if specified. + * @param message MoveTablesCreateResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.TopologyCell, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.MoveTablesCreateResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TopologyCell to JSON. + * Converts this MoveTablesCreateResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TopologyCell + * Gets the default type url for MoveTablesCreateResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetVSchemaRequest. */ - interface IGetVSchemaRequest { + namespace MoveTablesCreateResponse { - /** GetVSchemaRequest keyspace */ - keyspace?: (string|null); + /** Properties of a TabletInfo. */ + interface ITabletInfo { + + /** TabletInfo tablet */ + tablet?: (topodata.ITabletAlias|null); + + /** TabletInfo created */ + created?: (boolean|null); + } + + /** Represents a TabletInfo. */ + class TabletInfo implements ITabletInfo { + + /** + * Constructs a new TabletInfo. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.MoveTablesCreateResponse.ITabletInfo); + + /** TabletInfo tablet. */ + public tablet?: (topodata.ITabletAlias|null); + + /** TabletInfo created. */ + public created: boolean; + + /** + * Creates a new TabletInfo instance using the specified properties. + * @param [properties] Properties to set + * @returns TabletInfo instance + */ + public static create(properties?: vtctldata.MoveTablesCreateResponse.ITabletInfo): vtctldata.MoveTablesCreateResponse.TabletInfo; + + /** + * Encodes the specified TabletInfo message. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.TabletInfo.verify|verify} messages. + * @param message TabletInfo message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.MoveTablesCreateResponse.ITabletInfo, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified TabletInfo message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.TabletInfo.verify|verify} messages. + * @param message TabletInfo message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.MoveTablesCreateResponse.ITabletInfo, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a TabletInfo message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns TabletInfo + * @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.MoveTablesCreateResponse.TabletInfo; + + /** + * Decodes a TabletInfo message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns TabletInfo + * @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.MoveTablesCreateResponse.TabletInfo; + + /** + * Verifies a TabletInfo 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 TabletInfo message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns TabletInfo + */ + public static fromObject(object: { [k: string]: any }): vtctldata.MoveTablesCreateResponse.TabletInfo; + + /** + * Creates a plain object from a TabletInfo message. Also converts values to other types if specified. + * @param message TabletInfo + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.MoveTablesCreateResponse.TabletInfo, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this TabletInfo to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for TabletInfo + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } } - /** Represents a GetVSchemaRequest. */ - class GetVSchemaRequest implements IGetVSchemaRequest { + /** Properties of a MoveTablesCompleteRequest. */ + interface IMoveTablesCompleteRequest { + + /** MoveTablesCompleteRequest workflow */ + workflow?: (string|null); + + /** MoveTablesCompleteRequest target_keyspace */ + target_keyspace?: (string|null); + + /** MoveTablesCompleteRequest keep_data */ + keep_data?: (boolean|null); + + /** MoveTablesCompleteRequest keep_routing_rules */ + keep_routing_rules?: (boolean|null); + + /** MoveTablesCompleteRequest rename_tables */ + rename_tables?: (boolean|null); + + /** MoveTablesCompleteRequest dry_run */ + dry_run?: (boolean|null); + } + + /** Represents a MoveTablesCompleteRequest. */ + class MoveTablesCompleteRequest implements IMoveTablesCompleteRequest { /** - * Constructs a new GetVSchemaRequest. + * Constructs a new MoveTablesCompleteRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetVSchemaRequest); + constructor(properties?: vtctldata.IMoveTablesCompleteRequest); - /** GetVSchemaRequest keyspace. */ - public keyspace: string; + /** MoveTablesCompleteRequest workflow. */ + public workflow: string; + + /** MoveTablesCompleteRequest target_keyspace. */ + public target_keyspace: string; + + /** MoveTablesCompleteRequest keep_data. */ + public keep_data: boolean; + + /** MoveTablesCompleteRequest keep_routing_rules. */ + public keep_routing_rules: boolean; + + /** MoveTablesCompleteRequest rename_tables. */ + public rename_tables: boolean; + + /** MoveTablesCompleteRequest dry_run. */ + public dry_run: boolean; /** - * Creates a new GetVSchemaRequest instance using the specified properties. + * Creates a new MoveTablesCompleteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetVSchemaRequest instance + * @returns MoveTablesCompleteRequest instance */ - public static create(properties?: vtctldata.IGetVSchemaRequest): vtctldata.GetVSchemaRequest; + public static create(properties?: vtctldata.IMoveTablesCompleteRequest): vtctldata.MoveTablesCompleteRequest; /** - * Encodes the specified GetVSchemaRequest message. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. - * @param message GetVSchemaRequest message or plain object to encode + * Encodes the specified MoveTablesCompleteRequest message. Does not implicitly {@link vtctldata.MoveTablesCompleteRequest.verify|verify} messages. + * @param message MoveTablesCompleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IMoveTablesCompleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. - * @param message GetVSchemaRequest message or plain object to encode + * Encodes the specified MoveTablesCompleteRequest message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCompleteRequest.verify|verify} messages. + * @param message MoveTablesCompleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IMoveTablesCompleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetVSchemaRequest message from the specified reader or buffer. + * Decodes a MoveTablesCompleteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetVSchemaRequest + * @returns MoveTablesCompleteRequest * @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.GetVSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.MoveTablesCompleteRequest; /** - * Decodes a GetVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCompleteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetVSchemaRequest + * @returns MoveTablesCompleteRequest * @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.GetVSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.MoveTablesCompleteRequest; /** - * Verifies a GetVSchemaRequest message. + * Verifies a MoveTablesCompleteRequest 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 GetVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCompleteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetVSchemaRequest + * @returns MoveTablesCompleteRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetVSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.MoveTablesCompleteRequest; /** - * Creates a plain object from a GetVSchemaRequest message. Also converts values to other types if specified. - * @param message GetVSchemaRequest + * Creates a plain object from a MoveTablesCompleteRequest message. Also converts values to other types if specified. + * @param message MoveTablesCompleteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.MoveTablesCompleteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetVSchemaRequest to JSON. + * Converts this MoveTablesCompleteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetVSchemaRequest + * Gets the default type url for MoveTablesCompleteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetVersionRequest. */ - interface IGetVersionRequest { + /** Properties of a MoveTablesCompleteResponse. */ + interface IMoveTablesCompleteResponse { - /** GetVersionRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** MoveTablesCompleteResponse summary */ + summary?: (string|null); + + /** MoveTablesCompleteResponse dry_run_results */ + dry_run_results?: (string[]|null); } - /** Represents a GetVersionRequest. */ - class GetVersionRequest implements IGetVersionRequest { + /** Represents a MoveTablesCompleteResponse. */ + class MoveTablesCompleteResponse implements IMoveTablesCompleteResponse { /** - * Constructs a new GetVersionRequest. + * Constructs a new MoveTablesCompleteResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetVersionRequest); + constructor(properties?: vtctldata.IMoveTablesCompleteResponse); - /** GetVersionRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** MoveTablesCompleteResponse summary. */ + public summary: string; + + /** MoveTablesCompleteResponse dry_run_results. */ + public dry_run_results: string[]; /** - * Creates a new GetVersionRequest instance using the specified properties. + * Creates a new MoveTablesCompleteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetVersionRequest instance + * @returns MoveTablesCompleteResponse instance */ - public static create(properties?: vtctldata.IGetVersionRequest): vtctldata.GetVersionRequest; + public static create(properties?: vtctldata.IMoveTablesCompleteResponse): vtctldata.MoveTablesCompleteResponse; /** - * Encodes the specified GetVersionRequest message. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. - * @param message GetVersionRequest message or plain object to encode + * Encodes the specified MoveTablesCompleteResponse message. Does not implicitly {@link vtctldata.MoveTablesCompleteResponse.verify|verify} messages. + * @param message MoveTablesCompleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetVersionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IMoveTablesCompleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetVersionRequest message, length delimited. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. - * @param message GetVersionRequest message or plain object to encode + * Encodes the specified MoveTablesCompleteResponse message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCompleteResponse.verify|verify} messages. + * @param message MoveTablesCompleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetVersionRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IMoveTablesCompleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetVersionRequest message from the specified reader or buffer. + * Decodes a MoveTablesCompleteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetVersionRequest + * @returns MoveTablesCompleteResponse * @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.GetVersionRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.MoveTablesCompleteResponse; /** - * Decodes a GetVersionRequest message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCompleteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetVersionRequest + * @returns MoveTablesCompleteResponse * @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.GetVersionRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.MoveTablesCompleteResponse; /** - * Verifies a GetVersionRequest message. + * Verifies a MoveTablesCompleteResponse 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 GetVersionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCompleteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetVersionRequest + * @returns MoveTablesCompleteResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetVersionRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.MoveTablesCompleteResponse; /** - * Creates a plain object from a GetVersionRequest message. Also converts values to other types if specified. - * @param message GetVersionRequest + * Creates a plain object from a MoveTablesCompleteResponse message. Also converts values to other types if specified. + * @param message MoveTablesCompleteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetVersionRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.MoveTablesCompleteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetVersionRequest to JSON. + * Converts this MoveTablesCompleteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetVersionRequest + * Gets the default type url for MoveTablesCompleteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetVersionResponse. */ - interface IGetVersionResponse { + /** Properties of a PingTabletRequest. */ + interface IPingTabletRequest { - /** GetVersionResponse version */ - version?: (string|null); + /** PingTabletRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a GetVersionResponse. */ - class GetVersionResponse implements IGetVersionResponse { + /** Represents a PingTabletRequest. */ + class PingTabletRequest implements IPingTabletRequest { /** - * Constructs a new GetVersionResponse. + * Constructs a new PingTabletRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetVersionResponse); + constructor(properties?: vtctldata.IPingTabletRequest); - /** GetVersionResponse version. */ - public version: string; + /** PingTabletRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new GetVersionResponse instance using the specified properties. + * Creates a new PingTabletRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetVersionResponse instance + * @returns PingTabletRequest instance */ - public static create(properties?: vtctldata.IGetVersionResponse): vtctldata.GetVersionResponse; + public static create(properties?: vtctldata.IPingTabletRequest): vtctldata.PingTabletRequest; /** - * Encodes the specified GetVersionResponse message. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. - * @param message GetVersionResponse message or plain object to encode + * Encodes the specified PingTabletRequest message. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. + * @param message PingTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetVersionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IPingTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetVersionResponse message, length delimited. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. - * @param message GetVersionResponse message or plain object to encode + * Encodes the specified PingTabletRequest message, length delimited. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. + * @param message PingTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetVersionResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IPingTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetVersionResponse message from the specified reader or buffer. + * Decodes a PingTabletRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetVersionResponse + * @returns PingTabletRequest * @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.GetVersionResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.PingTabletRequest; /** - * Decodes a GetVersionResponse message from the specified reader or buffer, length delimited. + * Decodes a PingTabletRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetVersionResponse + * @returns PingTabletRequest * @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.GetVersionResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.PingTabletRequest; /** - * Verifies a GetVersionResponse message. + * Verifies a PingTabletRequest 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 GetVersionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PingTabletRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetVersionResponse + * @returns PingTabletRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetVersionResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.PingTabletRequest; /** - * Creates a plain object from a GetVersionResponse message. Also converts values to other types if specified. - * @param message GetVersionResponse + * Creates a plain object from a PingTabletRequest message. Also converts values to other types if specified. + * @param message PingTabletRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetVersionResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.PingTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetVersionResponse to JSON. + * Converts this PingTabletRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetVersionResponse + * Gets the default type url for PingTabletRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetVSchemaResponse. */ - interface IGetVSchemaResponse { - - /** GetVSchemaResponse v_schema */ - v_schema?: (vschema.IKeyspace|null); + /** Properties of a PingTabletResponse. */ + interface IPingTabletResponse { } - /** Represents a GetVSchemaResponse. */ - class GetVSchemaResponse implements IGetVSchemaResponse { + /** Represents a PingTabletResponse. */ + class PingTabletResponse implements IPingTabletResponse { /** - * Constructs a new GetVSchemaResponse. + * Constructs a new PingTabletResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetVSchemaResponse); - - /** GetVSchemaResponse v_schema. */ - public v_schema?: (vschema.IKeyspace|null); + constructor(properties?: vtctldata.IPingTabletResponse); /** - * Creates a new GetVSchemaResponse instance using the specified properties. + * Creates a new PingTabletResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetVSchemaResponse instance + * @returns PingTabletResponse instance */ - public static create(properties?: vtctldata.IGetVSchemaResponse): vtctldata.GetVSchemaResponse; + public static create(properties?: vtctldata.IPingTabletResponse): vtctldata.PingTabletResponse; /** - * Encodes the specified GetVSchemaResponse message. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. - * @param message GetVSchemaResponse message or plain object to encode + * Encodes the specified PingTabletResponse message. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. + * @param message PingTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IPingTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. - * @param message GetVSchemaResponse message or plain object to encode + * Encodes the specified PingTabletResponse message, length delimited. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. + * @param message PingTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IPingTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetVSchemaResponse message from the specified reader or buffer. + * Decodes a PingTabletResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetVSchemaResponse + * @returns PingTabletResponse * @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.GetVSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.PingTabletResponse; /** - * Decodes a GetVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a PingTabletResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetVSchemaResponse + * @returns PingTabletResponse * @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.GetVSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.PingTabletResponse; /** - * Verifies a GetVSchemaResponse message. + * Verifies a PingTabletResponse 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 GetVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PingTabletResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetVSchemaResponse + * @returns PingTabletResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetVSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.PingTabletResponse; /** - * Creates a plain object from a GetVSchemaResponse message. Also converts values to other types if specified. - * @param message GetVSchemaResponse + * Creates a plain object from a PingTabletResponse message. Also converts values to other types if specified. + * @param message PingTabletResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.PingTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetVSchemaResponse to JSON. + * Converts this PingTabletResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetVSchemaResponse + * Gets the default type url for PingTabletResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetWorkflowsRequest. */ - interface IGetWorkflowsRequest { + /** Properties of a PlannedReparentShardRequest. */ + interface IPlannedReparentShardRequest { - /** GetWorkflowsRequest keyspace */ + /** PlannedReparentShardRequest keyspace */ keyspace?: (string|null); - /** GetWorkflowsRequest active_only */ - active_only?: (boolean|null); + /** PlannedReparentShardRequest shard */ + shard?: (string|null); - /** GetWorkflowsRequest name_only */ - name_only?: (boolean|null); + /** PlannedReparentShardRequest new_primary */ + new_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardRequest avoid_primary */ + avoid_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardRequest wait_replicas_timeout */ + wait_replicas_timeout?: (vttime.IDuration|null); } - /** Represents a GetWorkflowsRequest. */ - class GetWorkflowsRequest implements IGetWorkflowsRequest { + /** Represents a PlannedReparentShardRequest. */ + class PlannedReparentShardRequest implements IPlannedReparentShardRequest { /** - * Constructs a new GetWorkflowsRequest. + * Constructs a new PlannedReparentShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetWorkflowsRequest); + constructor(properties?: vtctldata.IPlannedReparentShardRequest); - /** GetWorkflowsRequest keyspace. */ + /** PlannedReparentShardRequest keyspace. */ public keyspace: string; - /** GetWorkflowsRequest active_only. */ - public active_only: boolean; + /** PlannedReparentShardRequest shard. */ + public shard: string; - /** GetWorkflowsRequest name_only. */ - public name_only: boolean; + /** PlannedReparentShardRequest new_primary. */ + public new_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardRequest avoid_primary. */ + public avoid_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardRequest wait_replicas_timeout. */ + public wait_replicas_timeout?: (vttime.IDuration|null); /** - * Creates a new GetWorkflowsRequest instance using the specified properties. + * Creates a new PlannedReparentShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns GetWorkflowsRequest instance + * @returns PlannedReparentShardRequest instance */ - public static create(properties?: vtctldata.IGetWorkflowsRequest): vtctldata.GetWorkflowsRequest; + public static create(properties?: vtctldata.IPlannedReparentShardRequest): vtctldata.PlannedReparentShardRequest; /** - * Encodes the specified GetWorkflowsRequest message. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. - * @param message GetWorkflowsRequest message or plain object to encode + * Encodes the specified PlannedReparentShardRequest message. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. + * @param message PlannedReparentShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IPlannedReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetWorkflowsRequest message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. - * @param message GetWorkflowsRequest message or plain object to encode + * Encodes the specified PlannedReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. + * @param message PlannedReparentShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IPlannedReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetWorkflowsRequest message from the specified reader or buffer. + * Decodes a PlannedReparentShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetWorkflowsRequest + * @returns PlannedReparentShardRequest * @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.GetWorkflowsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.PlannedReparentShardRequest; /** - * Decodes a GetWorkflowsRequest message from the specified reader or buffer, length delimited. + * Decodes a PlannedReparentShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetWorkflowsRequest + * @returns PlannedReparentShardRequest * @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.GetWorkflowsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.PlannedReparentShardRequest; /** - * Verifies a GetWorkflowsRequest message. + * Verifies a PlannedReparentShardRequest 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 GetWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a PlannedReparentShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetWorkflowsRequest + * @returns PlannedReparentShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetWorkflowsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.PlannedReparentShardRequest; /** - * Creates a plain object from a GetWorkflowsRequest message. Also converts values to other types if specified. - * @param message GetWorkflowsRequest + * Creates a plain object from a PlannedReparentShardRequest message. Also converts values to other types if specified. + * @param message PlannedReparentShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetWorkflowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.PlannedReparentShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetWorkflowsRequest to JSON. + * Converts this PlannedReparentShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetWorkflowsRequest + * Gets the default type url for PlannedReparentShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a GetWorkflowsResponse. */ - interface IGetWorkflowsResponse { + /** Properties of a PlannedReparentShardResponse. */ + interface IPlannedReparentShardResponse { - /** GetWorkflowsResponse workflows */ - workflows?: (vtctldata.IWorkflow[]|null); + /** PlannedReparentShardResponse keyspace */ + keyspace?: (string|null); + + /** PlannedReparentShardResponse shard */ + shard?: (string|null); + + /** PlannedReparentShardResponse promoted_primary */ + promoted_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardResponse events */ + events?: (logutil.IEvent[]|null); } - /** Represents a GetWorkflowsResponse. */ - class GetWorkflowsResponse implements IGetWorkflowsResponse { + /** Represents a PlannedReparentShardResponse. */ + class PlannedReparentShardResponse implements IPlannedReparentShardResponse { /** - * Constructs a new GetWorkflowsResponse. + * Constructs a new PlannedReparentShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IGetWorkflowsResponse); + constructor(properties?: vtctldata.IPlannedReparentShardResponse); - /** GetWorkflowsResponse workflows. */ - public workflows: vtctldata.IWorkflow[]; + /** PlannedReparentShardResponse keyspace. */ + public keyspace: string; + + /** PlannedReparentShardResponse shard. */ + public shard: string; + + /** PlannedReparentShardResponse promoted_primary. */ + public promoted_primary?: (topodata.ITabletAlias|null); + + /** PlannedReparentShardResponse events. */ + public events: logutil.IEvent[]; /** - * Creates a new GetWorkflowsResponse instance using the specified properties. + * Creates a new PlannedReparentShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns GetWorkflowsResponse instance + * @returns PlannedReparentShardResponse instance */ - public static create(properties?: vtctldata.IGetWorkflowsResponse): vtctldata.GetWorkflowsResponse; + public static create(properties?: vtctldata.IPlannedReparentShardResponse): vtctldata.PlannedReparentShardResponse; /** - * Encodes the specified GetWorkflowsResponse message. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. - * @param message GetWorkflowsResponse message or plain object to encode + * Encodes the specified PlannedReparentShardResponse message. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. + * @param message PlannedReparentShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IGetWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IPlannedReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified GetWorkflowsResponse message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. - * @param message GetWorkflowsResponse message or plain object to encode + * Encodes the specified PlannedReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. + * @param message PlannedReparentShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IGetWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IPlannedReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a GetWorkflowsResponse message from the specified reader or buffer. + * Decodes a PlannedReparentShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns GetWorkflowsResponse + * @returns PlannedReparentShardResponse * @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.GetWorkflowsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.PlannedReparentShardResponse; /** - * Decodes a GetWorkflowsResponse message from the specified reader or buffer, length delimited. + * Decodes a PlannedReparentShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns GetWorkflowsResponse + * @returns PlannedReparentShardResponse * @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.GetWorkflowsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.PlannedReparentShardResponse; /** - * Verifies a GetWorkflowsResponse message. + * Verifies a PlannedReparentShardResponse 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 GetWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PlannedReparentShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns GetWorkflowsResponse + * @returns PlannedReparentShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.GetWorkflowsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.PlannedReparentShardResponse; /** - * Creates a plain object from a GetWorkflowsResponse message. Also converts values to other types if specified. - * @param message GetWorkflowsResponse + * Creates a plain object from a PlannedReparentShardResponse message. Also converts values to other types if specified. + * @param message PlannedReparentShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.GetWorkflowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.PlannedReparentShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this GetWorkflowsResponse to JSON. + * Converts this PlannedReparentShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for GetWorkflowsResponse + * Gets the default type url for PlannedReparentShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an InitShardPrimaryRequest. */ - interface IInitShardPrimaryRequest { + /** Properties of a RebuildKeyspaceGraphRequest. */ + interface IRebuildKeyspaceGraphRequest { - /** InitShardPrimaryRequest keyspace */ + /** RebuildKeyspaceGraphRequest keyspace */ keyspace?: (string|null); - /** InitShardPrimaryRequest shard */ - shard?: (string|null); - - /** InitShardPrimaryRequest primary_elect_tablet_alias */ - primary_elect_tablet_alias?: (topodata.ITabletAlias|null); - - /** InitShardPrimaryRequest force */ - force?: (boolean|null); + /** RebuildKeyspaceGraphRequest cells */ + cells?: (string[]|null); - /** InitShardPrimaryRequest wait_replicas_timeout */ - wait_replicas_timeout?: (vttime.IDuration|null); + /** RebuildKeyspaceGraphRequest allow_partial */ + allow_partial?: (boolean|null); } - /** Represents an InitShardPrimaryRequest. */ - class InitShardPrimaryRequest implements IInitShardPrimaryRequest { + /** Represents a RebuildKeyspaceGraphRequest. */ + class RebuildKeyspaceGraphRequest implements IRebuildKeyspaceGraphRequest { /** - * Constructs a new InitShardPrimaryRequest. + * Constructs a new RebuildKeyspaceGraphRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IInitShardPrimaryRequest); + constructor(properties?: vtctldata.IRebuildKeyspaceGraphRequest); - /** InitShardPrimaryRequest keyspace. */ + /** RebuildKeyspaceGraphRequest keyspace. */ public keyspace: string; - /** InitShardPrimaryRequest shard. */ - public shard: string; - - /** InitShardPrimaryRequest primary_elect_tablet_alias. */ - public primary_elect_tablet_alias?: (topodata.ITabletAlias|null); - - /** InitShardPrimaryRequest force. */ - public force: boolean; + /** RebuildKeyspaceGraphRequest cells. */ + public cells: string[]; - /** InitShardPrimaryRequest wait_replicas_timeout. */ - public wait_replicas_timeout?: (vttime.IDuration|null); + /** RebuildKeyspaceGraphRequest allow_partial. */ + public allow_partial: boolean; /** - * Creates a new InitShardPrimaryRequest instance using the specified properties. + * Creates a new RebuildKeyspaceGraphRequest instance using the specified properties. * @param [properties] Properties to set - * @returns InitShardPrimaryRequest instance + * @returns RebuildKeyspaceGraphRequest instance */ - public static create(properties?: vtctldata.IInitShardPrimaryRequest): vtctldata.InitShardPrimaryRequest; + public static create(properties?: vtctldata.IRebuildKeyspaceGraphRequest): vtctldata.RebuildKeyspaceGraphRequest; /** - * Encodes the specified InitShardPrimaryRequest message. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. - * @param message InitShardPrimaryRequest message or plain object to encode + * Encodes the specified RebuildKeyspaceGraphRequest message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. + * @param message RebuildKeyspaceGraphRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IInitShardPrimaryRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRebuildKeyspaceGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified InitShardPrimaryRequest message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. - * @param message InitShardPrimaryRequest message or plain object to encode + * Encodes the specified RebuildKeyspaceGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. + * @param message RebuildKeyspaceGraphRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IInitShardPrimaryRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRebuildKeyspaceGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an InitShardPrimaryRequest message from the specified reader or buffer. + * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns InitShardPrimaryRequest + * @returns RebuildKeyspaceGraphRequest * @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.InitShardPrimaryRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RebuildKeyspaceGraphRequest; /** - * Decodes an InitShardPrimaryRequest message from the specified reader or buffer, length delimited. + * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns InitShardPrimaryRequest + * @returns RebuildKeyspaceGraphRequest * @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.InitShardPrimaryRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RebuildKeyspaceGraphRequest; /** - * Verifies an InitShardPrimaryRequest message. + * Verifies a RebuildKeyspaceGraphRequest 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 InitShardPrimaryRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildKeyspaceGraphRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns InitShardPrimaryRequest + * @returns RebuildKeyspaceGraphRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.InitShardPrimaryRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RebuildKeyspaceGraphRequest; /** - * Creates a plain object from an InitShardPrimaryRequest message. Also converts values to other types if specified. - * @param message InitShardPrimaryRequest + * Creates a plain object from a RebuildKeyspaceGraphRequest message. Also converts values to other types if specified. + * @param message RebuildKeyspaceGraphRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.InitShardPrimaryRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RebuildKeyspaceGraphRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this InitShardPrimaryRequest to JSON. + * Converts this RebuildKeyspaceGraphRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for InitShardPrimaryRequest + * Gets the default type url for RebuildKeyspaceGraphRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an InitShardPrimaryResponse. */ - interface IInitShardPrimaryResponse { - - /** InitShardPrimaryResponse events */ - events?: (logutil.IEvent[]|null); + /** Properties of a RebuildKeyspaceGraphResponse. */ + interface IRebuildKeyspaceGraphResponse { } - /** Represents an InitShardPrimaryResponse. */ - class InitShardPrimaryResponse implements IInitShardPrimaryResponse { + /** Represents a RebuildKeyspaceGraphResponse. */ + class RebuildKeyspaceGraphResponse implements IRebuildKeyspaceGraphResponse { /** - * Constructs a new InitShardPrimaryResponse. + * Constructs a new RebuildKeyspaceGraphResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IInitShardPrimaryResponse); - - /** InitShardPrimaryResponse events. */ - public events: logutil.IEvent[]; + constructor(properties?: vtctldata.IRebuildKeyspaceGraphResponse); /** - * Creates a new InitShardPrimaryResponse instance using the specified properties. + * Creates a new RebuildKeyspaceGraphResponse instance using the specified properties. * @param [properties] Properties to set - * @returns InitShardPrimaryResponse instance + * @returns RebuildKeyspaceGraphResponse instance */ - public static create(properties?: vtctldata.IInitShardPrimaryResponse): vtctldata.InitShardPrimaryResponse; + public static create(properties?: vtctldata.IRebuildKeyspaceGraphResponse): vtctldata.RebuildKeyspaceGraphResponse; /** - * Encodes the specified InitShardPrimaryResponse message. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. - * @param message InitShardPrimaryResponse message or plain object to encode + * Encodes the specified RebuildKeyspaceGraphResponse message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. + * @param message RebuildKeyspaceGraphResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IInitShardPrimaryResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRebuildKeyspaceGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified InitShardPrimaryResponse message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. - * @param message InitShardPrimaryResponse message or plain object to encode + * Encodes the specified RebuildKeyspaceGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. + * @param message RebuildKeyspaceGraphResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IInitShardPrimaryResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRebuildKeyspaceGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an InitShardPrimaryResponse message from the specified reader or buffer. + * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns InitShardPrimaryResponse + * @returns RebuildKeyspaceGraphResponse * @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.InitShardPrimaryResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RebuildKeyspaceGraphResponse; /** - * Decodes an InitShardPrimaryResponse message from the specified reader or buffer, length delimited. + * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns InitShardPrimaryResponse + * @returns RebuildKeyspaceGraphResponse * @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.InitShardPrimaryResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RebuildKeyspaceGraphResponse; /** - * Verifies an InitShardPrimaryResponse message. + * Verifies a RebuildKeyspaceGraphResponse 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 InitShardPrimaryResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildKeyspaceGraphResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns InitShardPrimaryResponse + * @returns RebuildKeyspaceGraphResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.InitShardPrimaryResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RebuildKeyspaceGraphResponse; /** - * Creates a plain object from an InitShardPrimaryResponse message. Also converts values to other types if specified. - * @param message InitShardPrimaryResponse + * Creates a plain object from a RebuildKeyspaceGraphResponse message. Also converts values to other types if specified. + * @param message RebuildKeyspaceGraphResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.InitShardPrimaryResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RebuildKeyspaceGraphResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this InitShardPrimaryResponse to JSON. + * Converts this RebuildKeyspaceGraphResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for InitShardPrimaryResponse + * Gets the default type url for RebuildKeyspaceGraphResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PingTabletRequest. */ - interface IPingTabletRequest { + /** Properties of a RebuildVSchemaGraphRequest. */ + interface IRebuildVSchemaGraphRequest { - /** PingTabletRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** RebuildVSchemaGraphRequest cells */ + cells?: (string[]|null); } - /** Represents a PingTabletRequest. */ - class PingTabletRequest implements IPingTabletRequest { + /** Represents a RebuildVSchemaGraphRequest. */ + class RebuildVSchemaGraphRequest implements IRebuildVSchemaGraphRequest { /** - * Constructs a new PingTabletRequest. + * Constructs a new RebuildVSchemaGraphRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IPingTabletRequest); + constructor(properties?: vtctldata.IRebuildVSchemaGraphRequest); - /** PingTabletRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** RebuildVSchemaGraphRequest cells. */ + public cells: string[]; /** - * Creates a new PingTabletRequest instance using the specified properties. + * Creates a new RebuildVSchemaGraphRequest instance using the specified properties. * @param [properties] Properties to set - * @returns PingTabletRequest instance + * @returns RebuildVSchemaGraphRequest instance */ - public static create(properties?: vtctldata.IPingTabletRequest): vtctldata.PingTabletRequest; + public static create(properties?: vtctldata.IRebuildVSchemaGraphRequest): vtctldata.RebuildVSchemaGraphRequest; /** - * Encodes the specified PingTabletRequest message. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. - * @param message PingTabletRequest message or plain object to encode + * Encodes the specified RebuildVSchemaGraphRequest message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. + * @param message RebuildVSchemaGraphRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IPingTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRebuildVSchemaGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PingTabletRequest message, length delimited. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. - * @param message PingTabletRequest message or plain object to encode + * Encodes the specified RebuildVSchemaGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. + * @param message RebuildVSchemaGraphRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IPingTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRebuildVSchemaGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PingTabletRequest message from the specified reader or buffer. + * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PingTabletRequest + * @returns RebuildVSchemaGraphRequest * @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.PingTabletRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RebuildVSchemaGraphRequest; /** - * Decodes a PingTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PingTabletRequest + * @returns RebuildVSchemaGraphRequest * @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.PingTabletRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RebuildVSchemaGraphRequest; /** - * Verifies a PingTabletRequest message. + * Verifies a RebuildVSchemaGraphRequest 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 PingTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildVSchemaGraphRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PingTabletRequest + * @returns RebuildVSchemaGraphRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.PingTabletRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RebuildVSchemaGraphRequest; /** - * Creates a plain object from a PingTabletRequest message. Also converts values to other types if specified. - * @param message PingTabletRequest + * Creates a plain object from a RebuildVSchemaGraphRequest message. Also converts values to other types if specified. + * @param message RebuildVSchemaGraphRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.PingTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RebuildVSchemaGraphRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PingTabletRequest to JSON. + * Converts this RebuildVSchemaGraphRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PingTabletRequest + * Gets the default type url for RebuildVSchemaGraphRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PingTabletResponse. */ - interface IPingTabletResponse { + /** Properties of a RebuildVSchemaGraphResponse. */ + interface IRebuildVSchemaGraphResponse { } - /** Represents a PingTabletResponse. */ - class PingTabletResponse implements IPingTabletResponse { + /** Represents a RebuildVSchemaGraphResponse. */ + class RebuildVSchemaGraphResponse implements IRebuildVSchemaGraphResponse { /** - * Constructs a new PingTabletResponse. + * Constructs a new RebuildVSchemaGraphResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IPingTabletResponse); + constructor(properties?: vtctldata.IRebuildVSchemaGraphResponse); /** - * Creates a new PingTabletResponse instance using the specified properties. + * Creates a new RebuildVSchemaGraphResponse instance using the specified properties. * @param [properties] Properties to set - * @returns PingTabletResponse instance + * @returns RebuildVSchemaGraphResponse instance */ - public static create(properties?: vtctldata.IPingTabletResponse): vtctldata.PingTabletResponse; + public static create(properties?: vtctldata.IRebuildVSchemaGraphResponse): vtctldata.RebuildVSchemaGraphResponse; /** - * Encodes the specified PingTabletResponse message. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. - * @param message PingTabletResponse message or plain object to encode + * Encodes the specified RebuildVSchemaGraphResponse message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. + * @param message RebuildVSchemaGraphResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IPingTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRebuildVSchemaGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PingTabletResponse message, length delimited. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. - * @param message PingTabletResponse message or plain object to encode + * Encodes the specified RebuildVSchemaGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. + * @param message RebuildVSchemaGraphResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IPingTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRebuildVSchemaGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PingTabletResponse message from the specified reader or buffer. + * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PingTabletResponse + * @returns RebuildVSchemaGraphResponse * @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.PingTabletResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RebuildVSchemaGraphResponse; /** - * Decodes a PingTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PingTabletResponse + * @returns RebuildVSchemaGraphResponse * @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.PingTabletResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RebuildVSchemaGraphResponse; /** - * Verifies a PingTabletResponse message. + * Verifies a RebuildVSchemaGraphResponse 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 PingTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildVSchemaGraphResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PingTabletResponse + * @returns RebuildVSchemaGraphResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.PingTabletResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RebuildVSchemaGraphResponse; /** - * Creates a plain object from a PingTabletResponse message. Also converts values to other types if specified. - * @param message PingTabletResponse + * Creates a plain object from a RebuildVSchemaGraphResponse message. Also converts values to other types if specified. + * @param message RebuildVSchemaGraphResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.PingTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RebuildVSchemaGraphResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PingTabletResponse to JSON. + * Converts this RebuildVSchemaGraphResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PingTabletResponse + * Gets the default type url for RebuildVSchemaGraphResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PlannedReparentShardRequest. */ - interface IPlannedReparentShardRequest { - - /** PlannedReparentShardRequest keyspace */ - keyspace?: (string|null); - - /** PlannedReparentShardRequest shard */ - shard?: (string|null); - - /** PlannedReparentShardRequest new_primary */ - new_primary?: (topodata.ITabletAlias|null); - - /** PlannedReparentShardRequest avoid_primary */ - avoid_primary?: (topodata.ITabletAlias|null); + /** Properties of a RefreshStateRequest. */ + interface IRefreshStateRequest { - /** PlannedReparentShardRequest wait_replicas_timeout */ - wait_replicas_timeout?: (vttime.IDuration|null); + /** RefreshStateRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a PlannedReparentShardRequest. */ - class PlannedReparentShardRequest implements IPlannedReparentShardRequest { + /** Represents a RefreshStateRequest. */ + class RefreshStateRequest implements IRefreshStateRequest { /** - * Constructs a new PlannedReparentShardRequest. + * Constructs a new RefreshStateRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IPlannedReparentShardRequest); - - /** PlannedReparentShardRequest keyspace. */ - public keyspace: string; - - /** PlannedReparentShardRequest shard. */ - public shard: string; - - /** PlannedReparentShardRequest new_primary. */ - public new_primary?: (topodata.ITabletAlias|null); - - /** PlannedReparentShardRequest avoid_primary. */ - public avoid_primary?: (topodata.ITabletAlias|null); + constructor(properties?: vtctldata.IRefreshStateRequest); - /** PlannedReparentShardRequest wait_replicas_timeout. */ - public wait_replicas_timeout?: (vttime.IDuration|null); + /** RefreshStateRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new PlannedReparentShardRequest instance using the specified properties. + * Creates a new RefreshStateRequest instance using the specified properties. * @param [properties] Properties to set - * @returns PlannedReparentShardRequest instance + * @returns RefreshStateRequest instance */ - public static create(properties?: vtctldata.IPlannedReparentShardRequest): vtctldata.PlannedReparentShardRequest; + public static create(properties?: vtctldata.IRefreshStateRequest): vtctldata.RefreshStateRequest; /** - * Encodes the specified PlannedReparentShardRequest message. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. - * @param message PlannedReparentShardRequest message or plain object to encode + * Encodes the specified RefreshStateRequest message. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. + * @param message RefreshStateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IPlannedReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRefreshStateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PlannedReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. - * @param message PlannedReparentShardRequest message or plain object to encode + * Encodes the specified RefreshStateRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. + * @param message RefreshStateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IPlannedReparentShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRefreshStateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PlannedReparentShardRequest message from the specified reader or buffer. + * Decodes a RefreshStateRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PlannedReparentShardRequest + * @returns RefreshStateRequest * @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.PlannedReparentShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RefreshStateRequest; /** - * Decodes a PlannedReparentShardRequest message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PlannedReparentShardRequest + * @returns RefreshStateRequest * @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.PlannedReparentShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RefreshStateRequest; /** - * Verifies a PlannedReparentShardRequest message. + * Verifies a RefreshStateRequest 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 PlannedReparentShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PlannedReparentShardRequest + * @returns RefreshStateRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.PlannedReparentShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateRequest; /** - * Creates a plain object from a PlannedReparentShardRequest message. Also converts values to other types if specified. - * @param message PlannedReparentShardRequest + * Creates a plain object from a RefreshStateRequest message. Also converts values to other types if specified. + * @param message RefreshStateRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.PlannedReparentShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RefreshStateRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PlannedReparentShardRequest to JSON. + * Converts this RefreshStateRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PlannedReparentShardRequest + * Gets the default type url for RefreshStateRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a PlannedReparentShardResponse. */ - interface IPlannedReparentShardResponse { - - /** PlannedReparentShardResponse keyspace */ - keyspace?: (string|null); - - /** PlannedReparentShardResponse shard */ - shard?: (string|null); - - /** PlannedReparentShardResponse promoted_primary */ - promoted_primary?: (topodata.ITabletAlias|null); - - /** PlannedReparentShardResponse events */ - events?: (logutil.IEvent[]|null); + /** Properties of a RefreshStateResponse. */ + interface IRefreshStateResponse { } - /** Represents a PlannedReparentShardResponse. */ - class PlannedReparentShardResponse implements IPlannedReparentShardResponse { + /** Represents a RefreshStateResponse. */ + class RefreshStateResponse implements IRefreshStateResponse { /** - * Constructs a new PlannedReparentShardResponse. + * Constructs a new RefreshStateResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IPlannedReparentShardResponse); - - /** PlannedReparentShardResponse keyspace. */ - public keyspace: string; - - /** PlannedReparentShardResponse shard. */ - public shard: string; - - /** PlannedReparentShardResponse promoted_primary. */ - public promoted_primary?: (topodata.ITabletAlias|null); - - /** PlannedReparentShardResponse events. */ - public events: logutil.IEvent[]; + constructor(properties?: vtctldata.IRefreshStateResponse); /** - * Creates a new PlannedReparentShardResponse instance using the specified properties. + * Creates a new RefreshStateResponse instance using the specified properties. * @param [properties] Properties to set - * @returns PlannedReparentShardResponse instance + * @returns RefreshStateResponse instance */ - public static create(properties?: vtctldata.IPlannedReparentShardResponse): vtctldata.PlannedReparentShardResponse; + public static create(properties?: vtctldata.IRefreshStateResponse): vtctldata.RefreshStateResponse; /** - * Encodes the specified PlannedReparentShardResponse message. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. - * @param message PlannedReparentShardResponse message or plain object to encode + * Encodes the specified RefreshStateResponse message. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. + * @param message RefreshStateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IPlannedReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRefreshStateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified PlannedReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. - * @param message PlannedReparentShardResponse message or plain object to encode + * Encodes the specified RefreshStateResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. + * @param message RefreshStateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IPlannedReparentShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRefreshStateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a PlannedReparentShardResponse message from the specified reader or buffer. + * Decodes a RefreshStateResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns PlannedReparentShardResponse + * @returns RefreshStateResponse * @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.PlannedReparentShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RefreshStateResponse; /** - * Decodes a PlannedReparentShardResponse message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns PlannedReparentShardResponse + * @returns RefreshStateResponse * @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.PlannedReparentShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RefreshStateResponse; /** - * Verifies a PlannedReparentShardResponse message. + * Verifies a RefreshStateResponse 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 PlannedReparentShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns PlannedReparentShardResponse + * @returns RefreshStateResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.PlannedReparentShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateResponse; /** - * Creates a plain object from a PlannedReparentShardResponse message. Also converts values to other types if specified. - * @param message PlannedReparentShardResponse + * Creates a plain object from a RefreshStateResponse message. Also converts values to other types if specified. + * @param message RefreshStateResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.PlannedReparentShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RefreshStateResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this PlannedReparentShardResponse to JSON. + * Converts this RefreshStateResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for PlannedReparentShardResponse + * Gets the default type url for RefreshStateResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RebuildKeyspaceGraphRequest. */ - interface IRebuildKeyspaceGraphRequest { + /** Properties of a RefreshStateByShardRequest. */ + interface IRefreshStateByShardRequest { - /** RebuildKeyspaceGraphRequest keyspace */ + /** RefreshStateByShardRequest keyspace */ keyspace?: (string|null); - /** RebuildKeyspaceGraphRequest cells */ - cells?: (string[]|null); + /** RefreshStateByShardRequest shard */ + shard?: (string|null); - /** RebuildKeyspaceGraphRequest allow_partial */ - allow_partial?: (boolean|null); + /** RefreshStateByShardRequest cells */ + cells?: (string[]|null); } - /** Represents a RebuildKeyspaceGraphRequest. */ - class RebuildKeyspaceGraphRequest implements IRebuildKeyspaceGraphRequest { + /** Represents a RefreshStateByShardRequest. */ + class RefreshStateByShardRequest implements IRefreshStateByShardRequest { /** - * Constructs a new RebuildKeyspaceGraphRequest. + * Constructs a new RefreshStateByShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRebuildKeyspaceGraphRequest); + constructor(properties?: vtctldata.IRefreshStateByShardRequest); - /** RebuildKeyspaceGraphRequest keyspace. */ + /** RefreshStateByShardRequest keyspace. */ public keyspace: string; - /** RebuildKeyspaceGraphRequest cells. */ - public cells: string[]; + /** RefreshStateByShardRequest shard. */ + public shard: string; - /** RebuildKeyspaceGraphRequest allow_partial. */ - public allow_partial: boolean; + /** RefreshStateByShardRequest cells. */ + public cells: string[]; /** - * Creates a new RebuildKeyspaceGraphRequest instance using the specified properties. + * Creates a new RefreshStateByShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RebuildKeyspaceGraphRequest instance + * @returns RefreshStateByShardRequest instance */ - public static create(properties?: vtctldata.IRebuildKeyspaceGraphRequest): vtctldata.RebuildKeyspaceGraphRequest; + public static create(properties?: vtctldata.IRefreshStateByShardRequest): vtctldata.RefreshStateByShardRequest; /** - * Encodes the specified RebuildKeyspaceGraphRequest message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. - * @param message RebuildKeyspaceGraphRequest message or plain object to encode + * Encodes the specified RefreshStateByShardRequest message. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. + * @param message RefreshStateByShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRebuildKeyspaceGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRefreshStateByShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RebuildKeyspaceGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. - * @param message RebuildKeyspaceGraphRequest message or plain object to encode + * Encodes the specified RefreshStateByShardRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. + * @param message RefreshStateByShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRebuildKeyspaceGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRefreshStateByShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer. + * Decodes a RefreshStateByShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RebuildKeyspaceGraphRequest + * @returns RefreshStateByShardRequest * @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.RebuildKeyspaceGraphRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RefreshStateByShardRequest; /** - * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateByShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RebuildKeyspaceGraphRequest + * @returns RefreshStateByShardRequest * @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.RebuildKeyspaceGraphRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RefreshStateByShardRequest; /** - * Verifies a RebuildKeyspaceGraphRequest message. + * Verifies a RefreshStateByShardRequest 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 RebuildKeyspaceGraphRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateByShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RebuildKeyspaceGraphRequest + * @returns RefreshStateByShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RebuildKeyspaceGraphRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateByShardRequest; /** - * Creates a plain object from a RebuildKeyspaceGraphRequest message. Also converts values to other types if specified. - * @param message RebuildKeyspaceGraphRequest + * Creates a plain object from a RefreshStateByShardRequest message. Also converts values to other types if specified. + * @param message RefreshStateByShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RebuildKeyspaceGraphRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RefreshStateByShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RebuildKeyspaceGraphRequest to JSON. + * Converts this RefreshStateByShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RebuildKeyspaceGraphRequest + * Gets the default type url for RefreshStateByShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RebuildKeyspaceGraphResponse. */ - interface IRebuildKeyspaceGraphResponse { + /** Properties of a RefreshStateByShardResponse. */ + interface IRefreshStateByShardResponse { + + /** RefreshStateByShardResponse is_partial_refresh */ + is_partial_refresh?: (boolean|null); + + /** RefreshStateByShardResponse partial_refresh_details */ + partial_refresh_details?: (string|null); } - /** Represents a RebuildKeyspaceGraphResponse. */ - class RebuildKeyspaceGraphResponse implements IRebuildKeyspaceGraphResponse { + /** Represents a RefreshStateByShardResponse. */ + class RefreshStateByShardResponse implements IRefreshStateByShardResponse { /** - * Constructs a new RebuildKeyspaceGraphResponse. + * Constructs a new RefreshStateByShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRebuildKeyspaceGraphResponse); + constructor(properties?: vtctldata.IRefreshStateByShardResponse); + + /** RefreshStateByShardResponse is_partial_refresh. */ + public is_partial_refresh: boolean; + + /** RefreshStateByShardResponse partial_refresh_details. */ + public partial_refresh_details: string; /** - * Creates a new RebuildKeyspaceGraphResponse instance using the specified properties. + * Creates a new RefreshStateByShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RebuildKeyspaceGraphResponse instance + * @returns RefreshStateByShardResponse instance */ - public static create(properties?: vtctldata.IRebuildKeyspaceGraphResponse): vtctldata.RebuildKeyspaceGraphResponse; + public static create(properties?: vtctldata.IRefreshStateByShardResponse): vtctldata.RefreshStateByShardResponse; /** - * Encodes the specified RebuildKeyspaceGraphResponse message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. - * @param message RebuildKeyspaceGraphResponse message or plain object to encode + * Encodes the specified RefreshStateByShardResponse message. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. + * @param message RefreshStateByShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRebuildKeyspaceGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRefreshStateByShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RebuildKeyspaceGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. - * @param message RebuildKeyspaceGraphResponse message or plain object to encode + * Encodes the specified RefreshStateByShardResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. + * @param message RefreshStateByShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRebuildKeyspaceGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRefreshStateByShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer. + * Decodes a RefreshStateByShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RebuildKeyspaceGraphResponse + * @returns RefreshStateByShardResponse * @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.RebuildKeyspaceGraphResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RefreshStateByShardResponse; /** - * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateByShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RebuildKeyspaceGraphResponse + * @returns RefreshStateByShardResponse * @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.RebuildKeyspaceGraphResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RefreshStateByShardResponse; /** - * Verifies a RebuildKeyspaceGraphResponse message. + * Verifies a RefreshStateByShardResponse 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 RebuildKeyspaceGraphResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateByShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RebuildKeyspaceGraphResponse + * @returns RefreshStateByShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RebuildKeyspaceGraphResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateByShardResponse; /** - * Creates a plain object from a RebuildKeyspaceGraphResponse message. Also converts values to other types if specified. - * @param message RebuildKeyspaceGraphResponse + * Creates a plain object from a RefreshStateByShardResponse message. Also converts values to other types if specified. + * @param message RefreshStateByShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RebuildKeyspaceGraphResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RefreshStateByShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RebuildKeyspaceGraphResponse to JSON. + * Converts this RefreshStateByShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RebuildKeyspaceGraphResponse + * Gets the default type url for RefreshStateByShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RebuildVSchemaGraphRequest. */ - interface IRebuildVSchemaGraphRequest { + /** Properties of a ReloadSchemaRequest. */ + interface IReloadSchemaRequest { - /** RebuildVSchemaGraphRequest cells */ - cells?: (string[]|null); + /** ReloadSchemaRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a RebuildVSchemaGraphRequest. */ - class RebuildVSchemaGraphRequest implements IRebuildVSchemaGraphRequest { + /** Represents a ReloadSchemaRequest. */ + class ReloadSchemaRequest implements IReloadSchemaRequest { /** - * Constructs a new RebuildVSchemaGraphRequest. + * Constructs a new ReloadSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRebuildVSchemaGraphRequest); + constructor(properties?: vtctldata.IReloadSchemaRequest); - /** RebuildVSchemaGraphRequest cells. */ - public cells: string[]; + /** ReloadSchemaRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new RebuildVSchemaGraphRequest instance using the specified properties. + * Creates a new ReloadSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RebuildVSchemaGraphRequest instance + * @returns ReloadSchemaRequest instance */ - public static create(properties?: vtctldata.IRebuildVSchemaGraphRequest): vtctldata.RebuildVSchemaGraphRequest; + public static create(properties?: vtctldata.IReloadSchemaRequest): vtctldata.ReloadSchemaRequest; /** - * Encodes the specified RebuildVSchemaGraphRequest message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. - * @param message RebuildVSchemaGraphRequest message or plain object to encode + * Encodes the specified ReloadSchemaRequest message. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. + * @param message ReloadSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRebuildVSchemaGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RebuildVSchemaGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. - * @param message RebuildVSchemaGraphRequest message or plain object to encode + * Encodes the specified ReloadSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. + * @param message ReloadSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRebuildVSchemaGraphRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RebuildVSchemaGraphRequest + * @returns ReloadSchemaRequest * @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.RebuildVSchemaGraphRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaRequest; /** - * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RebuildVSchemaGraphRequest + * @returns ReloadSchemaRequest * @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.RebuildVSchemaGraphRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaRequest; /** - * Verifies a RebuildVSchemaGraphRequest message. + * Verifies a ReloadSchemaRequest 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 RebuildVSchemaGraphRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RebuildVSchemaGraphRequest + * @returns ReloadSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RebuildVSchemaGraphRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaRequest; /** - * Creates a plain object from a RebuildVSchemaGraphRequest message. Also converts values to other types if specified. - * @param message RebuildVSchemaGraphRequest + * Creates a plain object from a ReloadSchemaRequest message. Also converts values to other types if specified. + * @param message ReloadSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RebuildVSchemaGraphRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RebuildVSchemaGraphRequest to JSON. + * Converts this ReloadSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RebuildVSchemaGraphRequest + * Gets the default type url for ReloadSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RebuildVSchemaGraphResponse. */ - interface IRebuildVSchemaGraphResponse { + /** Properties of a ReloadSchemaResponse. */ + interface IReloadSchemaResponse { } - /** Represents a RebuildVSchemaGraphResponse. */ - class RebuildVSchemaGraphResponse implements IRebuildVSchemaGraphResponse { + /** Represents a ReloadSchemaResponse. */ + class ReloadSchemaResponse implements IReloadSchemaResponse { /** - * Constructs a new RebuildVSchemaGraphResponse. + * Constructs a new ReloadSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRebuildVSchemaGraphResponse); + constructor(properties?: vtctldata.IReloadSchemaResponse); /** - * Creates a new RebuildVSchemaGraphResponse instance using the specified properties. + * Creates a new ReloadSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RebuildVSchemaGraphResponse instance + * @returns ReloadSchemaResponse instance */ - public static create(properties?: vtctldata.IRebuildVSchemaGraphResponse): vtctldata.RebuildVSchemaGraphResponse; + public static create(properties?: vtctldata.IReloadSchemaResponse): vtctldata.ReloadSchemaResponse; /** - * Encodes the specified RebuildVSchemaGraphResponse message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. - * @param message RebuildVSchemaGraphResponse message or plain object to encode + * Encodes the specified ReloadSchemaResponse message. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. + * @param message ReloadSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRebuildVSchemaGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RebuildVSchemaGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. - * @param message RebuildVSchemaGraphResponse message or plain object to encode + * Encodes the specified ReloadSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. + * @param message ReloadSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRebuildVSchemaGraphResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RebuildVSchemaGraphResponse + * @returns ReloadSchemaResponse * @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.RebuildVSchemaGraphResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaResponse; /** - * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RebuildVSchemaGraphResponse + * @returns ReloadSchemaResponse * @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.RebuildVSchemaGraphResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaResponse; /** - * Verifies a RebuildVSchemaGraphResponse message. + * Verifies a ReloadSchemaResponse 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 RebuildVSchemaGraphResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RebuildVSchemaGraphResponse + * @returns ReloadSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RebuildVSchemaGraphResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaResponse; /** - * Creates a plain object from a RebuildVSchemaGraphResponse message. Also converts values to other types if specified. - * @param message RebuildVSchemaGraphResponse + * Creates a plain object from a ReloadSchemaResponse message. Also converts values to other types if specified. + * @param message ReloadSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RebuildVSchemaGraphResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RebuildVSchemaGraphResponse to JSON. + * Converts this ReloadSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RebuildVSchemaGraphResponse + * Gets the default type url for ReloadSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RefreshStateRequest. */ - interface IRefreshStateRequest { + /** Properties of a ReloadSchemaKeyspaceRequest. */ + interface IReloadSchemaKeyspaceRequest { - /** RefreshStateRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** ReloadSchemaKeyspaceRequest keyspace */ + keyspace?: (string|null); + + /** ReloadSchemaKeyspaceRequest wait_position */ + wait_position?: (string|null); + + /** ReloadSchemaKeyspaceRequest include_primary */ + include_primary?: (boolean|null); + + /** ReloadSchemaKeyspaceRequest concurrency */ + concurrency?: (number|null); } - /** Represents a RefreshStateRequest. */ - class RefreshStateRequest implements IRefreshStateRequest { + /** Represents a ReloadSchemaKeyspaceRequest. */ + class ReloadSchemaKeyspaceRequest implements IReloadSchemaKeyspaceRequest { /** - * Constructs a new RefreshStateRequest. + * Constructs a new ReloadSchemaKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRefreshStateRequest); + constructor(properties?: vtctldata.IReloadSchemaKeyspaceRequest); - /** RefreshStateRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** ReloadSchemaKeyspaceRequest keyspace. */ + public keyspace: string; + + /** ReloadSchemaKeyspaceRequest wait_position. */ + public wait_position: string; + + /** ReloadSchemaKeyspaceRequest include_primary. */ + public include_primary: boolean; + + /** ReloadSchemaKeyspaceRequest concurrency. */ + public concurrency: number; /** - * Creates a new RefreshStateRequest instance using the specified properties. + * Creates a new ReloadSchemaKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RefreshStateRequest instance + * @returns ReloadSchemaKeyspaceRequest instance */ - public static create(properties?: vtctldata.IRefreshStateRequest): vtctldata.RefreshStateRequest; + public static create(properties?: vtctldata.IReloadSchemaKeyspaceRequest): vtctldata.ReloadSchemaKeyspaceRequest; /** - * Encodes the specified RefreshStateRequest message. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. - * @param message RefreshStateRequest message or plain object to encode + * Encodes the specified ReloadSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. + * @param message ReloadSchemaKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRefreshStateRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RefreshStateRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. - * @param message RefreshStateRequest message or plain object to encode + * Encodes the specified ReloadSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. + * @param message ReloadSchemaKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRefreshStateRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RefreshStateRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RefreshStateRequest + * @returns ReloadSchemaKeyspaceRequest * @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.RefreshStateRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaKeyspaceRequest; /** - * Decodes a RefreshStateRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RefreshStateRequest + * @returns ReloadSchemaKeyspaceRequest * @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.RefreshStateRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaKeyspaceRequest; /** - * Verifies a RefreshStateRequest message. + * Verifies a ReloadSchemaKeyspaceRequest 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 RefreshStateRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RefreshStateRequest + * @returns ReloadSchemaKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaKeyspaceRequest; /** - * Creates a plain object from a RefreshStateRequest message. Also converts values to other types if specified. - * @param message RefreshStateRequest + * Creates a plain object from a ReloadSchemaKeyspaceRequest message. Also converts values to other types if specified. + * @param message ReloadSchemaKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RefreshStateRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RefreshStateRequest to JSON. + * Converts this ReloadSchemaKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RefreshStateRequest + * Gets the default type url for ReloadSchemaKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RefreshStateResponse. */ - interface IRefreshStateResponse { + /** Properties of a ReloadSchemaKeyspaceResponse. */ + interface IReloadSchemaKeyspaceResponse { + + /** ReloadSchemaKeyspaceResponse events */ + events?: (logutil.IEvent[]|null); } - /** Represents a RefreshStateResponse. */ - class RefreshStateResponse implements IRefreshStateResponse { + /** Represents a ReloadSchemaKeyspaceResponse. */ + class ReloadSchemaKeyspaceResponse implements IReloadSchemaKeyspaceResponse { /** - * Constructs a new RefreshStateResponse. + * Constructs a new ReloadSchemaKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRefreshStateResponse); + constructor(properties?: vtctldata.IReloadSchemaKeyspaceResponse); + + /** ReloadSchemaKeyspaceResponse events. */ + public events: logutil.IEvent[]; /** - * Creates a new RefreshStateResponse instance using the specified properties. + * Creates a new ReloadSchemaKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RefreshStateResponse instance + * @returns ReloadSchemaKeyspaceResponse instance */ - public static create(properties?: vtctldata.IRefreshStateResponse): vtctldata.RefreshStateResponse; + public static create(properties?: vtctldata.IReloadSchemaKeyspaceResponse): vtctldata.ReloadSchemaKeyspaceResponse; /** - * Encodes the specified RefreshStateResponse message. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. - * @param message RefreshStateResponse message or plain object to encode + * Encodes the specified ReloadSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. + * @param message ReloadSchemaKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRefreshStateResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RefreshStateResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. - * @param message RefreshStateResponse message or plain object to encode + * Encodes the specified ReloadSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. + * @param message ReloadSchemaKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRefreshStateResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RefreshStateResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RefreshStateResponse + * @returns ReloadSchemaKeyspaceResponse * @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.RefreshStateResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaKeyspaceResponse; /** - * Decodes a RefreshStateResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RefreshStateResponse + * @returns ReloadSchemaKeyspaceResponse * @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.RefreshStateResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaKeyspaceResponse; /** - * Verifies a RefreshStateResponse message. + * Verifies a ReloadSchemaKeyspaceResponse 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 RefreshStateResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RefreshStateResponse + * @returns ReloadSchemaKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaKeyspaceResponse; /** - * Creates a plain object from a RefreshStateResponse message. Also converts values to other types if specified. - * @param message RefreshStateResponse + * Creates a plain object from a ReloadSchemaKeyspaceResponse message. Also converts values to other types if specified. + * @param message ReloadSchemaKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RefreshStateResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RefreshStateResponse to JSON. + * Converts this ReloadSchemaKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RefreshStateResponse + * Gets the default type url for ReloadSchemaKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RefreshStateByShardRequest. */ - interface IRefreshStateByShardRequest { + /** Properties of a ReloadSchemaShardRequest. */ + interface IReloadSchemaShardRequest { - /** RefreshStateByShardRequest keyspace */ + /** ReloadSchemaShardRequest keyspace */ keyspace?: (string|null); - /** RefreshStateByShardRequest shard */ + /** ReloadSchemaShardRequest shard */ shard?: (string|null); - /** RefreshStateByShardRequest cells */ - cells?: (string[]|null); + /** ReloadSchemaShardRequest wait_position */ + wait_position?: (string|null); + + /** ReloadSchemaShardRequest include_primary */ + include_primary?: (boolean|null); + + /** ReloadSchemaShardRequest concurrency */ + concurrency?: (number|null); } - /** Represents a RefreshStateByShardRequest. */ - class RefreshStateByShardRequest implements IRefreshStateByShardRequest { + /** Represents a ReloadSchemaShardRequest. */ + class ReloadSchemaShardRequest implements IReloadSchemaShardRequest { /** - * Constructs a new RefreshStateByShardRequest. + * Constructs a new ReloadSchemaShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRefreshStateByShardRequest); + constructor(properties?: vtctldata.IReloadSchemaShardRequest); - /** RefreshStateByShardRequest keyspace. */ + /** ReloadSchemaShardRequest keyspace. */ public keyspace: string; - /** RefreshStateByShardRequest shard. */ + /** ReloadSchemaShardRequest shard. */ public shard: string; - /** RefreshStateByShardRequest cells. */ - public cells: string[]; + /** ReloadSchemaShardRequest wait_position. */ + public wait_position: string; + + /** ReloadSchemaShardRequest include_primary. */ + public include_primary: boolean; + + /** ReloadSchemaShardRequest concurrency. */ + public concurrency: number; /** - * Creates a new RefreshStateByShardRequest instance using the specified properties. + * Creates a new ReloadSchemaShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RefreshStateByShardRequest instance + * @returns ReloadSchemaShardRequest instance */ - public static create(properties?: vtctldata.IRefreshStateByShardRequest): vtctldata.RefreshStateByShardRequest; + public static create(properties?: vtctldata.IReloadSchemaShardRequest): vtctldata.ReloadSchemaShardRequest; /** - * Encodes the specified RefreshStateByShardRequest message. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. - * @param message RefreshStateByShardRequest message or plain object to encode + * Encodes the specified ReloadSchemaShardRequest message. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. + * @param message ReloadSchemaShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRefreshStateByShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RefreshStateByShardRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. - * @param message RefreshStateByShardRequest message or plain object to encode + * Encodes the specified ReloadSchemaShardRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. + * @param message ReloadSchemaShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRefreshStateByShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RefreshStateByShardRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RefreshStateByShardRequest + * @returns ReloadSchemaShardRequest * @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.RefreshStateByShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaShardRequest; /** - * Decodes a RefreshStateByShardRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RefreshStateByShardRequest + * @returns ReloadSchemaShardRequest * @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.RefreshStateByShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaShardRequest; /** - * Verifies a RefreshStateByShardRequest message. + * Verifies a ReloadSchemaShardRequest 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 RefreshStateByShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RefreshStateByShardRequest + * @returns ReloadSchemaShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateByShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaShardRequest; /** - * Creates a plain object from a RefreshStateByShardRequest message. Also converts values to other types if specified. - * @param message RefreshStateByShardRequest + * Creates a plain object from a ReloadSchemaShardRequest message. Also converts values to other types if specified. + * @param message ReloadSchemaShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RefreshStateByShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RefreshStateByShardRequest to JSON. + * Converts this ReloadSchemaShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RefreshStateByShardRequest + * Gets the default type url for ReloadSchemaShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RefreshStateByShardResponse. */ - interface IRefreshStateByShardResponse { - - /** RefreshStateByShardResponse is_partial_refresh */ - is_partial_refresh?: (boolean|null); + /** Properties of a ReloadSchemaShardResponse. */ + interface IReloadSchemaShardResponse { - /** RefreshStateByShardResponse partial_refresh_details */ - partial_refresh_details?: (string|null); + /** ReloadSchemaShardResponse events */ + events?: (logutil.IEvent[]|null); } - /** Represents a RefreshStateByShardResponse. */ - class RefreshStateByShardResponse implements IRefreshStateByShardResponse { + /** Represents a ReloadSchemaShardResponse. */ + class ReloadSchemaShardResponse implements IReloadSchemaShardResponse { /** - * Constructs a new RefreshStateByShardResponse. + * Constructs a new ReloadSchemaShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRefreshStateByShardResponse); - - /** RefreshStateByShardResponse is_partial_refresh. */ - public is_partial_refresh: boolean; + constructor(properties?: vtctldata.IReloadSchemaShardResponse); - /** RefreshStateByShardResponse partial_refresh_details. */ - public partial_refresh_details: string; + /** ReloadSchemaShardResponse events. */ + public events: logutil.IEvent[]; /** - * Creates a new RefreshStateByShardResponse instance using the specified properties. + * Creates a new ReloadSchemaShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RefreshStateByShardResponse instance + * @returns ReloadSchemaShardResponse instance */ - public static create(properties?: vtctldata.IRefreshStateByShardResponse): vtctldata.RefreshStateByShardResponse; + public static create(properties?: vtctldata.IReloadSchemaShardResponse): vtctldata.ReloadSchemaShardResponse; /** - * Encodes the specified RefreshStateByShardResponse message. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. - * @param message RefreshStateByShardResponse message or plain object to encode + * Encodes the specified ReloadSchemaShardResponse message. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. + * @param message ReloadSchemaShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRefreshStateByShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReloadSchemaShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RefreshStateByShardResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. - * @param message RefreshStateByShardResponse message or plain object to encode + * Encodes the specified ReloadSchemaShardResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. + * @param message ReloadSchemaShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRefreshStateByShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReloadSchemaShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RefreshStateByShardResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RefreshStateByShardResponse + * @returns ReloadSchemaShardResponse * @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.RefreshStateByShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReloadSchemaShardResponse; /** - * Decodes a RefreshStateByShardResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RefreshStateByShardResponse + * @returns ReloadSchemaShardResponse * @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.RefreshStateByShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReloadSchemaShardResponse; /** - * Verifies a RefreshStateByShardResponse message. + * Verifies a ReloadSchemaShardResponse 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 RefreshStateByShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RefreshStateByShardResponse + * @returns ReloadSchemaShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RefreshStateByShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaShardResponse; /** - * Creates a plain object from a RefreshStateByShardResponse message. Also converts values to other types if specified. - * @param message RefreshStateByShardResponse + * Creates a plain object from a ReloadSchemaShardResponse message. Also converts values to other types if specified. + * @param message ReloadSchemaShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RefreshStateByShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReloadSchemaShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RefreshStateByShardResponse to JSON. + * Converts this ReloadSchemaShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RefreshStateByShardResponse + * Gets the default type url for ReloadSchemaShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaRequest. */ - interface IReloadSchemaRequest { + /** Properties of a RemoveBackupRequest. */ + interface IRemoveBackupRequest { - /** ReloadSchemaRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** RemoveBackupRequest keyspace */ + keyspace?: (string|null); + + /** RemoveBackupRequest shard */ + shard?: (string|null); + + /** RemoveBackupRequest name */ + name?: (string|null); } - /** Represents a ReloadSchemaRequest. */ - class ReloadSchemaRequest implements IReloadSchemaRequest { + /** Represents a RemoveBackupRequest. */ + class RemoveBackupRequest implements IRemoveBackupRequest { /** - * Constructs a new ReloadSchemaRequest. + * Constructs a new RemoveBackupRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaRequest); + constructor(properties?: vtctldata.IRemoveBackupRequest); - /** ReloadSchemaRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** RemoveBackupRequest keyspace. */ + public keyspace: string; + + /** RemoveBackupRequest shard. */ + public shard: string; + + /** RemoveBackupRequest name. */ + public name: string; /** - * Creates a new ReloadSchemaRequest instance using the specified properties. + * Creates a new RemoveBackupRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaRequest instance + * @returns RemoveBackupRequest instance */ - public static create(properties?: vtctldata.IReloadSchemaRequest): vtctldata.ReloadSchemaRequest; + public static create(properties?: vtctldata.IRemoveBackupRequest): vtctldata.RemoveBackupRequest; /** - * Encodes the specified ReloadSchemaRequest message. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. - * @param message ReloadSchemaRequest message or plain object to encode + * Encodes the specified RemoveBackupRequest message. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. + * @param message RemoveBackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. - * @param message ReloadSchemaRequest message or plain object to encode + * Encodes the specified RemoveBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. + * @param message RemoveBackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaRequest message from the specified reader or buffer. + * Decodes a RemoveBackupRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaRequest + * @returns RemoveBackupRequest * @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.ReloadSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveBackupRequest; /** - * Decodes a ReloadSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveBackupRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaRequest + * @returns RemoveBackupRequest * @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.ReloadSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveBackupRequest; /** - * Verifies a ReloadSchemaRequest message. + * Verifies a RemoveBackupRequest 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 ReloadSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveBackupRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaRequest + * @returns RemoveBackupRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveBackupRequest; /** - * Creates a plain object from a ReloadSchemaRequest message. Also converts values to other types if specified. - * @param message ReloadSchemaRequest + * Creates a plain object from a RemoveBackupRequest message. Also converts values to other types if specified. + * @param message RemoveBackupRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveBackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaRequest to JSON. + * Converts this RemoveBackupRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaRequest + * Gets the default type url for RemoveBackupRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaResponse. */ - interface IReloadSchemaResponse { + /** Properties of a RemoveBackupResponse. */ + interface IRemoveBackupResponse { } - /** Represents a ReloadSchemaResponse. */ - class ReloadSchemaResponse implements IReloadSchemaResponse { + /** Represents a RemoveBackupResponse. */ + class RemoveBackupResponse implements IRemoveBackupResponse { /** - * Constructs a new ReloadSchemaResponse. + * Constructs a new RemoveBackupResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaResponse); + constructor(properties?: vtctldata.IRemoveBackupResponse); /** - * Creates a new ReloadSchemaResponse instance using the specified properties. + * Creates a new RemoveBackupResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaResponse instance + * @returns RemoveBackupResponse instance */ - public static create(properties?: vtctldata.IReloadSchemaResponse): vtctldata.ReloadSchemaResponse; + public static create(properties?: vtctldata.IRemoveBackupResponse): vtctldata.RemoveBackupResponse; /** - * Encodes the specified ReloadSchemaResponse message. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. - * @param message ReloadSchemaResponse message or plain object to encode + * Encodes the specified RemoveBackupResponse message. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. + * @param message RemoveBackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. - * @param message ReloadSchemaResponse message or plain object to encode + * Encodes the specified RemoveBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. + * @param message RemoveBackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaResponse message from the specified reader or buffer. + * Decodes a RemoveBackupResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaResponse + * @returns RemoveBackupResponse * @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.ReloadSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveBackupResponse; /** - * Decodes a ReloadSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveBackupResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaResponse + * @returns RemoveBackupResponse * @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.ReloadSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveBackupResponse; /** - * Verifies a ReloadSchemaResponse message. + * Verifies a RemoveBackupResponse 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 ReloadSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveBackupResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaResponse + * @returns RemoveBackupResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveBackupResponse; /** - * Creates a plain object from a ReloadSchemaResponse message. Also converts values to other types if specified. - * @param message ReloadSchemaResponse + * Creates a plain object from a RemoveBackupResponse message. Also converts values to other types if specified. + * @param message RemoveBackupResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveBackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaResponse to JSON. + * Converts this RemoveBackupResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaResponse + * Gets the default type url for RemoveBackupResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaKeyspaceRequest. */ - interface IReloadSchemaKeyspaceRequest { + /** Properties of a RemoveKeyspaceCellRequest. */ + interface IRemoveKeyspaceCellRequest { - /** ReloadSchemaKeyspaceRequest keyspace */ + /** RemoveKeyspaceCellRequest keyspace */ keyspace?: (string|null); - /** ReloadSchemaKeyspaceRequest wait_position */ - wait_position?: (string|null); + /** RemoveKeyspaceCellRequest cell */ + cell?: (string|null); - /** ReloadSchemaKeyspaceRequest include_primary */ - include_primary?: (boolean|null); + /** RemoveKeyspaceCellRequest force */ + force?: (boolean|null); - /** ReloadSchemaKeyspaceRequest concurrency */ - concurrency?: (number|null); + /** RemoveKeyspaceCellRequest recursive */ + recursive?: (boolean|null); } - /** Represents a ReloadSchemaKeyspaceRequest. */ - class ReloadSchemaKeyspaceRequest implements IReloadSchemaKeyspaceRequest { + /** Represents a RemoveKeyspaceCellRequest. */ + class RemoveKeyspaceCellRequest implements IRemoveKeyspaceCellRequest { /** - * Constructs a new ReloadSchemaKeyspaceRequest. + * Constructs a new RemoveKeyspaceCellRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaKeyspaceRequest); + constructor(properties?: vtctldata.IRemoveKeyspaceCellRequest); - /** ReloadSchemaKeyspaceRequest keyspace. */ + /** RemoveKeyspaceCellRequest keyspace. */ public keyspace: string; - /** ReloadSchemaKeyspaceRequest wait_position. */ - public wait_position: string; + /** RemoveKeyspaceCellRequest cell. */ + public cell: string; - /** ReloadSchemaKeyspaceRequest include_primary. */ - public include_primary: boolean; + /** RemoveKeyspaceCellRequest force. */ + public force: boolean; - /** ReloadSchemaKeyspaceRequest concurrency. */ - public concurrency: number; + /** RemoveKeyspaceCellRequest recursive. */ + public recursive: boolean; /** - * Creates a new ReloadSchemaKeyspaceRequest instance using the specified properties. + * Creates a new RemoveKeyspaceCellRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaKeyspaceRequest instance + * @returns RemoveKeyspaceCellRequest instance */ - public static create(properties?: vtctldata.IReloadSchemaKeyspaceRequest): vtctldata.ReloadSchemaKeyspaceRequest; + public static create(properties?: vtctldata.IRemoveKeyspaceCellRequest): vtctldata.RemoveKeyspaceCellRequest; /** - * Encodes the specified ReloadSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. - * @param message ReloadSchemaKeyspaceRequest message or plain object to encode + * Encodes the specified RemoveKeyspaceCellRequest message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. + * @param message RemoveKeyspaceCellRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveKeyspaceCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. - * @param message ReloadSchemaKeyspaceRequest message or plain object to encode + * Encodes the specified RemoveKeyspaceCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. + * @param message RemoveKeyspaceCellRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveKeyspaceCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer. + * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaKeyspaceRequest + * @returns RemoveKeyspaceCellRequest * @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.ReloadSchemaKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveKeyspaceCellRequest; /** - * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaKeyspaceRequest + * @returns RemoveKeyspaceCellRequest * @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.ReloadSchemaKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveKeyspaceCellRequest; /** - * Verifies a ReloadSchemaKeyspaceRequest message. + * Verifies a RemoveKeyspaceCellRequest 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 ReloadSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveKeyspaceCellRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaKeyspaceRequest + * @returns RemoveKeyspaceCellRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveKeyspaceCellRequest; /** - * Creates a plain object from a ReloadSchemaKeyspaceRequest message. Also converts values to other types if specified. - * @param message ReloadSchemaKeyspaceRequest + * Creates a plain object from a RemoveKeyspaceCellRequest message. Also converts values to other types if specified. + * @param message RemoveKeyspaceCellRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveKeyspaceCellRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaKeyspaceRequest to JSON. + * Converts this RemoveKeyspaceCellRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaKeyspaceRequest + * Gets the default type url for RemoveKeyspaceCellRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaKeyspaceResponse. */ - interface IReloadSchemaKeyspaceResponse { - - /** ReloadSchemaKeyspaceResponse events */ - events?: (logutil.IEvent[]|null); + /** Properties of a RemoveKeyspaceCellResponse. */ + interface IRemoveKeyspaceCellResponse { } - /** Represents a ReloadSchemaKeyspaceResponse. */ - class ReloadSchemaKeyspaceResponse implements IReloadSchemaKeyspaceResponse { + /** Represents a RemoveKeyspaceCellResponse. */ + class RemoveKeyspaceCellResponse implements IRemoveKeyspaceCellResponse { /** - * Constructs a new ReloadSchemaKeyspaceResponse. + * Constructs a new RemoveKeyspaceCellResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaKeyspaceResponse); - - /** ReloadSchemaKeyspaceResponse events. */ - public events: logutil.IEvent[]; + constructor(properties?: vtctldata.IRemoveKeyspaceCellResponse); /** - * Creates a new ReloadSchemaKeyspaceResponse instance using the specified properties. + * Creates a new RemoveKeyspaceCellResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaKeyspaceResponse instance + * @returns RemoveKeyspaceCellResponse instance */ - public static create(properties?: vtctldata.IReloadSchemaKeyspaceResponse): vtctldata.ReloadSchemaKeyspaceResponse; + public static create(properties?: vtctldata.IRemoveKeyspaceCellResponse): vtctldata.RemoveKeyspaceCellResponse; /** - * Encodes the specified ReloadSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. - * @param message ReloadSchemaKeyspaceResponse message or plain object to encode + * Encodes the specified RemoveKeyspaceCellResponse message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. + * @param message RemoveKeyspaceCellResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveKeyspaceCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. - * @param message ReloadSchemaKeyspaceResponse message or plain object to encode + * Encodes the specified RemoveKeyspaceCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. + * @param message RemoveKeyspaceCellResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveKeyspaceCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer. + * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaKeyspaceResponse + * @returns RemoveKeyspaceCellResponse * @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.ReloadSchemaKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveKeyspaceCellResponse; /** - * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaKeyspaceResponse + * @returns RemoveKeyspaceCellResponse * @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.ReloadSchemaKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveKeyspaceCellResponse; /** - * Verifies a ReloadSchemaKeyspaceResponse message. + * Verifies a RemoveKeyspaceCellResponse 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 ReloadSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveKeyspaceCellResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaKeyspaceResponse + * @returns RemoveKeyspaceCellResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveKeyspaceCellResponse; /** - * Creates a plain object from a ReloadSchemaKeyspaceResponse message. Also converts values to other types if specified. - * @param message ReloadSchemaKeyspaceResponse + * Creates a plain object from a RemoveKeyspaceCellResponse message. Also converts values to other types if specified. + * @param message RemoveKeyspaceCellResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveKeyspaceCellResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaKeyspaceResponse to JSON. + * Converts this RemoveKeyspaceCellResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaKeyspaceResponse + * Gets the default type url for RemoveKeyspaceCellResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaShardRequest. */ - interface IReloadSchemaShardRequest { + /** Properties of a RemoveShardCellRequest. */ + interface IRemoveShardCellRequest { - /** ReloadSchemaShardRequest keyspace */ + /** RemoveShardCellRequest keyspace */ keyspace?: (string|null); - /** ReloadSchemaShardRequest shard */ - shard?: (string|null); + /** RemoveShardCellRequest shard_name */ + shard_name?: (string|null); - /** ReloadSchemaShardRequest wait_position */ - wait_position?: (string|null); + /** RemoveShardCellRequest cell */ + cell?: (string|null); - /** ReloadSchemaShardRequest include_primary */ - include_primary?: (boolean|null); + /** RemoveShardCellRequest force */ + force?: (boolean|null); - /** ReloadSchemaShardRequest concurrency */ - concurrency?: (number|null); + /** RemoveShardCellRequest recursive */ + recursive?: (boolean|null); } - /** Represents a ReloadSchemaShardRequest. */ - class ReloadSchemaShardRequest implements IReloadSchemaShardRequest { + /** Represents a RemoveShardCellRequest. */ + class RemoveShardCellRequest implements IRemoveShardCellRequest { /** - * Constructs a new ReloadSchemaShardRequest. + * Constructs a new RemoveShardCellRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaShardRequest); + constructor(properties?: vtctldata.IRemoveShardCellRequest); - /** ReloadSchemaShardRequest keyspace. */ + /** RemoveShardCellRequest keyspace. */ public keyspace: string; - /** ReloadSchemaShardRequest shard. */ - public shard: string; + /** RemoveShardCellRequest shard_name. */ + public shard_name: string; - /** ReloadSchemaShardRequest wait_position. */ - public wait_position: string; + /** RemoveShardCellRequest cell. */ + public cell: string; - /** ReloadSchemaShardRequest include_primary. */ - public include_primary: boolean; + /** RemoveShardCellRequest force. */ + public force: boolean; - /** ReloadSchemaShardRequest concurrency. */ - public concurrency: number; + /** RemoveShardCellRequest recursive. */ + public recursive: boolean; /** - * Creates a new ReloadSchemaShardRequest instance using the specified properties. + * Creates a new RemoveShardCellRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaShardRequest instance + * @returns RemoveShardCellRequest instance */ - public static create(properties?: vtctldata.IReloadSchemaShardRequest): vtctldata.ReloadSchemaShardRequest; + public static create(properties?: vtctldata.IRemoveShardCellRequest): vtctldata.RemoveShardCellRequest; /** - * Encodes the specified ReloadSchemaShardRequest message. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. - * @param message ReloadSchemaShardRequest message or plain object to encode + * Encodes the specified RemoveShardCellRequest message. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. + * @param message RemoveShardCellRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveShardCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaShardRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. - * @param message ReloadSchemaShardRequest message or plain object to encode + * Encodes the specified RemoveShardCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. + * @param message RemoveShardCellRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveShardCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer. + * Decodes a RemoveShardCellRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaShardRequest + * @returns RemoveShardCellRequest * @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.ReloadSchemaShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveShardCellRequest; /** - * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveShardCellRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaShardRequest + * @returns RemoveShardCellRequest * @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.ReloadSchemaShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveShardCellRequest; /** - * Verifies a ReloadSchemaShardRequest message. + * Verifies a RemoveShardCellRequest 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 ReloadSchemaShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveShardCellRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaShardRequest + * @returns RemoveShardCellRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveShardCellRequest; /** - * Creates a plain object from a ReloadSchemaShardRequest message. Also converts values to other types if specified. - * @param message ReloadSchemaShardRequest + * Creates a plain object from a RemoveShardCellRequest message. Also converts values to other types if specified. + * @param message RemoveShardCellRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveShardCellRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaShardRequest to JSON. + * Converts this RemoveShardCellRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaShardRequest + * Gets the default type url for RemoveShardCellRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReloadSchemaShardResponse. */ - interface IReloadSchemaShardResponse { - - /** ReloadSchemaShardResponse events */ - events?: (logutil.IEvent[]|null); + /** Properties of a RemoveShardCellResponse. */ + interface IRemoveShardCellResponse { } - /** Represents a ReloadSchemaShardResponse. */ - class ReloadSchemaShardResponse implements IReloadSchemaShardResponse { + /** Represents a RemoveShardCellResponse. */ + class RemoveShardCellResponse implements IRemoveShardCellResponse { /** - * Constructs a new ReloadSchemaShardResponse. + * Constructs a new RemoveShardCellResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReloadSchemaShardResponse); - - /** ReloadSchemaShardResponse events. */ - public events: logutil.IEvent[]; + constructor(properties?: vtctldata.IRemoveShardCellResponse); /** - * Creates a new ReloadSchemaShardResponse instance using the specified properties. + * Creates a new RemoveShardCellResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReloadSchemaShardResponse instance + * @returns RemoveShardCellResponse instance */ - public static create(properties?: vtctldata.IReloadSchemaShardResponse): vtctldata.ReloadSchemaShardResponse; + public static create(properties?: vtctldata.IRemoveShardCellResponse): vtctldata.RemoveShardCellResponse; /** - * Encodes the specified ReloadSchemaShardResponse message. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. - * @param message ReloadSchemaShardResponse message or plain object to encode + * Encodes the specified RemoveShardCellResponse message. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. + * @param message RemoveShardCellResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReloadSchemaShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRemoveShardCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReloadSchemaShardResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. - * @param message ReloadSchemaShardResponse message or plain object to encode + * Encodes the specified RemoveShardCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. + * @param message RemoveShardCellResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReloadSchemaShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRemoveShardCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer. + * Decodes a RemoveShardCellResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReloadSchemaShardResponse + * @returns RemoveShardCellResponse * @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.ReloadSchemaShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RemoveShardCellResponse; /** - * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveShardCellResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReloadSchemaShardResponse + * @returns RemoveShardCellResponse * @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.ReloadSchemaShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RemoveShardCellResponse; /** - * Verifies a ReloadSchemaShardResponse message. + * Verifies a RemoveShardCellResponse 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 ReloadSchemaShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveShardCellResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReloadSchemaShardResponse + * @returns RemoveShardCellResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReloadSchemaShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RemoveShardCellResponse; /** - * Creates a plain object from a ReloadSchemaShardResponse message. Also converts values to other types if specified. - * @param message ReloadSchemaShardResponse + * Creates a plain object from a RemoveShardCellResponse message. Also converts values to other types if specified. + * @param message RemoveShardCellResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReloadSchemaShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RemoveShardCellResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReloadSchemaShardResponse to JSON. + * Converts this RemoveShardCellResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReloadSchemaShardResponse + * Gets the default type url for RemoveShardCellResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveBackupRequest. */ - interface IRemoveBackupRequest { - - /** RemoveBackupRequest keyspace */ - keyspace?: (string|null); - - /** RemoveBackupRequest shard */ - shard?: (string|null); + /** Properties of a ReparentTabletRequest. */ + interface IReparentTabletRequest { - /** RemoveBackupRequest name */ - name?: (string|null); + /** ReparentTabletRequest tablet */ + tablet?: (topodata.ITabletAlias|null); } - /** Represents a RemoveBackupRequest. */ - class RemoveBackupRequest implements IRemoveBackupRequest { + /** Represents a ReparentTabletRequest. */ + class ReparentTabletRequest implements IReparentTabletRequest { /** - * Constructs a new RemoveBackupRequest. + * Constructs a new ReparentTabletRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveBackupRequest); - - /** RemoveBackupRequest keyspace. */ - public keyspace: string; - - /** RemoveBackupRequest shard. */ - public shard: string; + constructor(properties?: vtctldata.IReparentTabletRequest); - /** RemoveBackupRequest name. */ - public name: string; + /** ReparentTabletRequest tablet. */ + public tablet?: (topodata.ITabletAlias|null); /** - * Creates a new RemoveBackupRequest instance using the specified properties. + * Creates a new ReparentTabletRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveBackupRequest instance + * @returns ReparentTabletRequest instance */ - public static create(properties?: vtctldata.IRemoveBackupRequest): vtctldata.RemoveBackupRequest; + public static create(properties?: vtctldata.IReparentTabletRequest): vtctldata.ReparentTabletRequest; /** - * Encodes the specified RemoveBackupRequest message. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. - * @param message RemoveBackupRequest message or plain object to encode + * Encodes the specified ReparentTabletRequest message. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. + * @param message ReparentTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReparentTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. - * @param message RemoveBackupRequest message or plain object to encode + * Encodes the specified ReparentTabletRequest message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. + * @param message ReparentTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReparentTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveBackupRequest message from the specified reader or buffer. + * Decodes a ReparentTabletRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveBackupRequest + * @returns ReparentTabletRequest * @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.RemoveBackupRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReparentTabletRequest; /** - * Decodes a RemoveBackupRequest message from the specified reader or buffer, length delimited. + * Decodes a ReparentTabletRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveBackupRequest + * @returns ReparentTabletRequest * @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.RemoveBackupRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReparentTabletRequest; /** - * Verifies a RemoveBackupRequest message. + * Verifies a ReparentTabletRequest 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 RemoveBackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReparentTabletRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveBackupRequest + * @returns ReparentTabletRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveBackupRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ReparentTabletRequest; /** - * Creates a plain object from a RemoveBackupRequest message. Also converts values to other types if specified. - * @param message RemoveBackupRequest + * Creates a plain object from a ReparentTabletRequest message. Also converts values to other types if specified. + * @param message ReparentTabletRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveBackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReparentTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveBackupRequest to JSON. + * Converts this ReparentTabletRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveBackupRequest + * Gets the default type url for ReparentTabletRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveBackupResponse. */ - interface IRemoveBackupResponse { + /** Properties of a ReparentTabletResponse. */ + interface IReparentTabletResponse { + + /** ReparentTabletResponse keyspace */ + keyspace?: (string|null); + + /** ReparentTabletResponse shard */ + shard?: (string|null); + + /** ReparentTabletResponse primary */ + primary?: (topodata.ITabletAlias|null); } - /** Represents a RemoveBackupResponse. */ - class RemoveBackupResponse implements IRemoveBackupResponse { + /** Represents a ReparentTabletResponse. */ + class ReparentTabletResponse implements IReparentTabletResponse { /** - * Constructs a new RemoveBackupResponse. + * Constructs a new ReparentTabletResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveBackupResponse); + constructor(properties?: vtctldata.IReparentTabletResponse); + + /** ReparentTabletResponse keyspace. */ + public keyspace: string; + + /** ReparentTabletResponse shard. */ + public shard: string; + + /** ReparentTabletResponse primary. */ + public primary?: (topodata.ITabletAlias|null); /** - * Creates a new RemoveBackupResponse instance using the specified properties. + * Creates a new ReparentTabletResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveBackupResponse instance + * @returns ReparentTabletResponse instance */ - public static create(properties?: vtctldata.IRemoveBackupResponse): vtctldata.RemoveBackupResponse; + public static create(properties?: vtctldata.IReparentTabletResponse): vtctldata.ReparentTabletResponse; /** - * Encodes the specified RemoveBackupResponse message. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. - * @param message RemoveBackupResponse message or plain object to encode + * Encodes the specified ReparentTabletResponse message. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. + * @param message ReparentTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IReparentTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. - * @param message RemoveBackupResponse message or plain object to encode + * Encodes the specified ReparentTabletResponse message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. + * @param message ReparentTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IReparentTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveBackupResponse message from the specified reader or buffer. + * Decodes a ReparentTabletResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveBackupResponse + * @returns ReparentTabletResponse * @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.RemoveBackupResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ReparentTabletResponse; /** - * Decodes a RemoveBackupResponse message from the specified reader or buffer, length delimited. + * Decodes a ReparentTabletResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveBackupResponse + * @returns ReparentTabletResponse * @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.RemoveBackupResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ReparentTabletResponse; /** - * Verifies a RemoveBackupResponse message. + * Verifies a ReparentTabletResponse 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 RemoveBackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReparentTabletResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveBackupResponse + * @returns ReparentTabletResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveBackupResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ReparentTabletResponse; /** - * Creates a plain object from a RemoveBackupResponse message. Also converts values to other types if specified. - * @param message RemoveBackupResponse + * Creates a plain object from a ReparentTabletResponse message. Also converts values to other types if specified. + * @param message ReparentTabletResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveBackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ReparentTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveBackupResponse to JSON. + * Converts this ReparentTabletResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveBackupResponse + * Gets the default type url for ReparentTabletResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveKeyspaceCellRequest. */ - interface IRemoveKeyspaceCellRequest { + /** Properties of a RestoreFromBackupRequest. */ + interface IRestoreFromBackupRequest { - /** RemoveKeyspaceCellRequest keyspace */ - keyspace?: (string|null); + /** RestoreFromBackupRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** RemoveKeyspaceCellRequest cell */ - cell?: (string|null); + /** RestoreFromBackupRequest backup_time */ + backup_time?: (vttime.ITime|null); - /** RemoveKeyspaceCellRequest force */ - force?: (boolean|null); + /** RestoreFromBackupRequest restore_to_pos */ + restore_to_pos?: (string|null); - /** RemoveKeyspaceCellRequest recursive */ - recursive?: (boolean|null); + /** RestoreFromBackupRequest dry_run */ + dry_run?: (boolean|null); + + /** RestoreFromBackupRequest restore_to_timestamp */ + restore_to_timestamp?: (vttime.ITime|null); } - /** Represents a RemoveKeyspaceCellRequest. */ - class RemoveKeyspaceCellRequest implements IRemoveKeyspaceCellRequest { + /** Represents a RestoreFromBackupRequest. */ + class RestoreFromBackupRequest implements IRestoreFromBackupRequest { /** - * Constructs a new RemoveKeyspaceCellRequest. + * Constructs a new RestoreFromBackupRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveKeyspaceCellRequest); + constructor(properties?: vtctldata.IRestoreFromBackupRequest); - /** RemoveKeyspaceCellRequest keyspace. */ - public keyspace: string; + /** RestoreFromBackupRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** RemoveKeyspaceCellRequest cell. */ - public cell: string; + /** RestoreFromBackupRequest backup_time. */ + public backup_time?: (vttime.ITime|null); - /** RemoveKeyspaceCellRequest force. */ - public force: boolean; + /** RestoreFromBackupRequest restore_to_pos. */ + public restore_to_pos: string; - /** RemoveKeyspaceCellRequest recursive. */ - public recursive: boolean; + /** RestoreFromBackupRequest dry_run. */ + public dry_run: boolean; + + /** RestoreFromBackupRequest restore_to_timestamp. */ + public restore_to_timestamp?: (vttime.ITime|null); /** - * Creates a new RemoveKeyspaceCellRequest instance using the specified properties. + * Creates a new RestoreFromBackupRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveKeyspaceCellRequest instance + * @returns RestoreFromBackupRequest instance */ - public static create(properties?: vtctldata.IRemoveKeyspaceCellRequest): vtctldata.RemoveKeyspaceCellRequest; + public static create(properties?: vtctldata.IRestoreFromBackupRequest): vtctldata.RestoreFromBackupRequest; /** - * Encodes the specified RemoveKeyspaceCellRequest message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. - * @param message RemoveKeyspaceCellRequest message or plain object to encode + * Encodes the specified RestoreFromBackupRequest message. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. + * @param message RestoreFromBackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveKeyspaceCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRestoreFromBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveKeyspaceCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. - * @param message RemoveKeyspaceCellRequest message or plain object to encode + * Encodes the specified RestoreFromBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. + * @param message RestoreFromBackupRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveKeyspaceCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRestoreFromBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer. + * Decodes a RestoreFromBackupRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveKeyspaceCellRequest + * @returns RestoreFromBackupRequest * @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.RemoveKeyspaceCellRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RestoreFromBackupRequest; /** - * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer, length delimited. + * Decodes a RestoreFromBackupRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveKeyspaceCellRequest + * @returns RestoreFromBackupRequest * @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.RemoveKeyspaceCellRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RestoreFromBackupRequest; /** - * Verifies a RemoveKeyspaceCellRequest message. + * Verifies a RestoreFromBackupRequest 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 RemoveKeyspaceCellRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RestoreFromBackupRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveKeyspaceCellRequest + * @returns RestoreFromBackupRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveKeyspaceCellRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RestoreFromBackupRequest; /** - * Creates a plain object from a RemoveKeyspaceCellRequest message. Also converts values to other types if specified. - * @param message RemoveKeyspaceCellRequest + * Creates a plain object from a RestoreFromBackupRequest message. Also converts values to other types if specified. + * @param message RestoreFromBackupRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveKeyspaceCellRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RestoreFromBackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveKeyspaceCellRequest to JSON. + * Converts this RestoreFromBackupRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveKeyspaceCellRequest + * Gets the default type url for RestoreFromBackupRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveKeyspaceCellResponse. */ - interface IRemoveKeyspaceCellResponse { + /** Properties of a RestoreFromBackupResponse. */ + interface IRestoreFromBackupResponse { + + /** RestoreFromBackupResponse tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); + + /** RestoreFromBackupResponse keyspace */ + keyspace?: (string|null); + + /** RestoreFromBackupResponse shard */ + shard?: (string|null); + + /** RestoreFromBackupResponse event */ + event?: (logutil.IEvent|null); } - /** Represents a RemoveKeyspaceCellResponse. */ - class RemoveKeyspaceCellResponse implements IRemoveKeyspaceCellResponse { + /** Represents a RestoreFromBackupResponse. */ + class RestoreFromBackupResponse implements IRestoreFromBackupResponse { /** - * Constructs a new RemoveKeyspaceCellResponse. + * Constructs a new RestoreFromBackupResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveKeyspaceCellResponse); + constructor(properties?: vtctldata.IRestoreFromBackupResponse); + + /** RestoreFromBackupResponse tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); + + /** RestoreFromBackupResponse keyspace. */ + public keyspace: string; + + /** RestoreFromBackupResponse shard. */ + public shard: string; + + /** RestoreFromBackupResponse event. */ + public event?: (logutil.IEvent|null); /** - * Creates a new RemoveKeyspaceCellResponse instance using the specified properties. + * Creates a new RestoreFromBackupResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveKeyspaceCellResponse instance + * @returns RestoreFromBackupResponse instance */ - public static create(properties?: vtctldata.IRemoveKeyspaceCellResponse): vtctldata.RemoveKeyspaceCellResponse; + public static create(properties?: vtctldata.IRestoreFromBackupResponse): vtctldata.RestoreFromBackupResponse; /** - * Encodes the specified RemoveKeyspaceCellResponse message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. - * @param message RemoveKeyspaceCellResponse message or plain object to encode + * Encodes the specified RestoreFromBackupResponse message. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. + * @param message RestoreFromBackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveKeyspaceCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRestoreFromBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveKeyspaceCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. - * @param message RemoveKeyspaceCellResponse message or plain object to encode + * Encodes the specified RestoreFromBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. + * @param message RestoreFromBackupResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveKeyspaceCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRestoreFromBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer. + * Decodes a RestoreFromBackupResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveKeyspaceCellResponse + * @returns RestoreFromBackupResponse * @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.RemoveKeyspaceCellResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RestoreFromBackupResponse; /** - * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer, length delimited. + * Decodes a RestoreFromBackupResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveKeyspaceCellResponse + * @returns RestoreFromBackupResponse * @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.RemoveKeyspaceCellResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RestoreFromBackupResponse; /** - * Verifies a RemoveKeyspaceCellResponse message. + * Verifies a RestoreFromBackupResponse 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 RemoveKeyspaceCellResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RestoreFromBackupResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveKeyspaceCellResponse + * @returns RestoreFromBackupResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveKeyspaceCellResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RestoreFromBackupResponse; /** - * Creates a plain object from a RemoveKeyspaceCellResponse message. Also converts values to other types if specified. - * @param message RemoveKeyspaceCellResponse + * Creates a plain object from a RestoreFromBackupResponse message. Also converts values to other types if specified. + * @param message RestoreFromBackupResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveKeyspaceCellResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RestoreFromBackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveKeyspaceCellResponse to JSON. + * Converts this RestoreFromBackupResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveKeyspaceCellResponse + * Gets the default type url for RestoreFromBackupResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveShardCellRequest. */ - interface IRemoveShardCellRequest { - - /** RemoveShardCellRequest keyspace */ - keyspace?: (string|null); - - /** RemoveShardCellRequest shard_name */ - shard_name?: (string|null); - - /** RemoveShardCellRequest cell */ - cell?: (string|null); - - /** RemoveShardCellRequest force */ - force?: (boolean|null); + /** Properties of a RunHealthCheckRequest. */ + interface IRunHealthCheckRequest { - /** RemoveShardCellRequest recursive */ - recursive?: (boolean|null); + /** RunHealthCheckRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a RemoveShardCellRequest. */ - class RemoveShardCellRequest implements IRemoveShardCellRequest { + /** Represents a RunHealthCheckRequest. */ + class RunHealthCheckRequest implements IRunHealthCheckRequest { /** - * Constructs a new RemoveShardCellRequest. + * Constructs a new RunHealthCheckRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveShardCellRequest); - - /** RemoveShardCellRequest keyspace. */ - public keyspace: string; - - /** RemoveShardCellRequest shard_name. */ - public shard_name: string; - - /** RemoveShardCellRequest cell. */ - public cell: string; - - /** RemoveShardCellRequest force. */ - public force: boolean; + constructor(properties?: vtctldata.IRunHealthCheckRequest); - /** RemoveShardCellRequest recursive. */ - public recursive: boolean; + /** RunHealthCheckRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new RemoveShardCellRequest instance using the specified properties. + * Creates a new RunHealthCheckRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveShardCellRequest instance + * @returns RunHealthCheckRequest instance */ - public static create(properties?: vtctldata.IRemoveShardCellRequest): vtctldata.RemoveShardCellRequest; + public static create(properties?: vtctldata.IRunHealthCheckRequest): vtctldata.RunHealthCheckRequest; /** - * Encodes the specified RemoveShardCellRequest message. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. - * @param message RemoveShardCellRequest message or plain object to encode + * Encodes the specified RunHealthCheckRequest message. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. + * @param message RunHealthCheckRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveShardCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRunHealthCheckRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveShardCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. - * @param message RemoveShardCellRequest message or plain object to encode + * Encodes the specified RunHealthCheckRequest message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. + * @param message RunHealthCheckRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveShardCellRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRunHealthCheckRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveShardCellRequest message from the specified reader or buffer. + * Decodes a RunHealthCheckRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveShardCellRequest + * @returns RunHealthCheckRequest * @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.RemoveShardCellRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RunHealthCheckRequest; /** - * Decodes a RemoveShardCellRequest message from the specified reader or buffer, length delimited. + * Decodes a RunHealthCheckRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveShardCellRequest + * @returns RunHealthCheckRequest * @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.RemoveShardCellRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RunHealthCheckRequest; /** - * Verifies a RemoveShardCellRequest message. + * Verifies a RunHealthCheckRequest 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 RemoveShardCellRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RunHealthCheckRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveShardCellRequest + * @returns RunHealthCheckRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveShardCellRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.RunHealthCheckRequest; /** - * Creates a plain object from a RemoveShardCellRequest message. Also converts values to other types if specified. - * @param message RemoveShardCellRequest + * Creates a plain object from a RunHealthCheckRequest message. Also converts values to other types if specified. + * @param message RunHealthCheckRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveShardCellRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RunHealthCheckRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveShardCellRequest to JSON. + * Converts this RunHealthCheckRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveShardCellRequest + * Gets the default type url for RunHealthCheckRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RemoveShardCellResponse. */ - interface IRemoveShardCellResponse { + /** Properties of a RunHealthCheckResponse. */ + interface IRunHealthCheckResponse { } - /** Represents a RemoveShardCellResponse. */ - class RemoveShardCellResponse implements IRemoveShardCellResponse { + /** Represents a RunHealthCheckResponse. */ + class RunHealthCheckResponse implements IRunHealthCheckResponse { /** - * Constructs a new RemoveShardCellResponse. + * Constructs a new RunHealthCheckResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRemoveShardCellResponse); + constructor(properties?: vtctldata.IRunHealthCheckResponse); /** - * Creates a new RemoveShardCellResponse instance using the specified properties. + * Creates a new RunHealthCheckResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RemoveShardCellResponse instance + * @returns RunHealthCheckResponse instance */ - public static create(properties?: vtctldata.IRemoveShardCellResponse): vtctldata.RemoveShardCellResponse; + public static create(properties?: vtctldata.IRunHealthCheckResponse): vtctldata.RunHealthCheckResponse; /** - * Encodes the specified RemoveShardCellResponse message. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. - * @param message RemoveShardCellResponse message or plain object to encode + * Encodes the specified RunHealthCheckResponse message. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. + * @param message RunHealthCheckResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRemoveShardCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IRunHealthCheckResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RemoveShardCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. - * @param message RemoveShardCellResponse message or plain object to encode + * Encodes the specified RunHealthCheckResponse message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. + * @param message RunHealthCheckResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRemoveShardCellResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IRunHealthCheckResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RemoveShardCellResponse message from the specified reader or buffer. + * Decodes a RunHealthCheckResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RemoveShardCellResponse + * @returns RunHealthCheckResponse * @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.RemoveShardCellResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.RunHealthCheckResponse; /** - * Decodes a RemoveShardCellResponse message from the specified reader or buffer, length delimited. + * Decodes a RunHealthCheckResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RemoveShardCellResponse + * @returns RunHealthCheckResponse * @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.RemoveShardCellResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.RunHealthCheckResponse; /** - * Verifies a RemoveShardCellResponse message. + * Verifies a RunHealthCheckResponse 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 RemoveShardCellResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RunHealthCheckResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RemoveShardCellResponse + * @returns RunHealthCheckResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RemoveShardCellResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.RunHealthCheckResponse; /** - * Creates a plain object from a RemoveShardCellResponse message. Also converts values to other types if specified. - * @param message RemoveShardCellResponse + * Creates a plain object from a RunHealthCheckResponse message. Also converts values to other types if specified. + * @param message RunHealthCheckResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RemoveShardCellResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.RunHealthCheckResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RemoveShardCellResponse to JSON. + * Converts this RunHealthCheckResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RemoveShardCellResponse + * Gets the default type url for RunHealthCheckResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReparentTabletRequest. */ - interface IReparentTabletRequest { + /** Properties of a SetKeyspaceDurabilityPolicyRequest. */ + interface ISetKeyspaceDurabilityPolicyRequest { - /** ReparentTabletRequest tablet */ - tablet?: (topodata.ITabletAlias|null); + /** SetKeyspaceDurabilityPolicyRequest keyspace */ + keyspace?: (string|null); + + /** SetKeyspaceDurabilityPolicyRequest durability_policy */ + durability_policy?: (string|null); } - /** Represents a ReparentTabletRequest. */ - class ReparentTabletRequest implements IReparentTabletRequest { + /** Represents a SetKeyspaceDurabilityPolicyRequest. */ + class SetKeyspaceDurabilityPolicyRequest implements ISetKeyspaceDurabilityPolicyRequest { /** - * Constructs a new ReparentTabletRequest. + * Constructs a new SetKeyspaceDurabilityPolicyRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReparentTabletRequest); + constructor(properties?: vtctldata.ISetKeyspaceDurabilityPolicyRequest); - /** ReparentTabletRequest tablet. */ - public tablet?: (topodata.ITabletAlias|null); + /** SetKeyspaceDurabilityPolicyRequest keyspace. */ + public keyspace: string; + + /** SetKeyspaceDurabilityPolicyRequest durability_policy. */ + public durability_policy: string; /** - * Creates a new ReparentTabletRequest instance using the specified properties. + * Creates a new SetKeyspaceDurabilityPolicyRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ReparentTabletRequest instance + * @returns SetKeyspaceDurabilityPolicyRequest instance */ - public static create(properties?: vtctldata.IReparentTabletRequest): vtctldata.ReparentTabletRequest; + public static create(properties?: vtctldata.ISetKeyspaceDurabilityPolicyRequest): vtctldata.SetKeyspaceDurabilityPolicyRequest; /** - * Encodes the specified ReparentTabletRequest message. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. - * @param message ReparentTabletRequest message or plain object to encode + * Encodes the specified SetKeyspaceDurabilityPolicyRequest message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. + * @param message SetKeyspaceDurabilityPolicyRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReparentTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceDurabilityPolicyRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReparentTabletRequest message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. - * @param message ReparentTabletRequest message or plain object to encode + * Encodes the specified SetKeyspaceDurabilityPolicyRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. + * @param message SetKeyspaceDurabilityPolicyRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReparentTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceDurabilityPolicyRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReparentTabletRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReparentTabletRequest + * @returns SetKeyspaceDurabilityPolicyRequest * @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.ReparentTabletRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceDurabilityPolicyRequest; /** - * Decodes a ReparentTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReparentTabletRequest + * @returns SetKeyspaceDurabilityPolicyRequest * @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.ReparentTabletRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceDurabilityPolicyRequest; /** - * Verifies a ReparentTabletRequest message. + * Verifies a SetKeyspaceDurabilityPolicyRequest 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 ReparentTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceDurabilityPolicyRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReparentTabletRequest + * @returns SetKeyspaceDurabilityPolicyRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReparentTabletRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceDurabilityPolicyRequest; /** - * Creates a plain object from a ReparentTabletRequest message. Also converts values to other types if specified. - * @param message ReparentTabletRequest + * Creates a plain object from a SetKeyspaceDurabilityPolicyRequest message. Also converts values to other types if specified. + * @param message SetKeyspaceDurabilityPolicyRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReparentTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceDurabilityPolicyRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReparentTabletRequest to JSON. + * Converts this SetKeyspaceDurabilityPolicyRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReparentTabletRequest + * Gets the default type url for SetKeyspaceDurabilityPolicyRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ReparentTabletResponse. */ - interface IReparentTabletResponse { - - /** ReparentTabletResponse keyspace */ - keyspace?: (string|null); - - /** ReparentTabletResponse shard */ - shard?: (string|null); + /** Properties of a SetKeyspaceDurabilityPolicyResponse. */ + interface ISetKeyspaceDurabilityPolicyResponse { - /** ReparentTabletResponse primary */ - primary?: (topodata.ITabletAlias|null); + /** SetKeyspaceDurabilityPolicyResponse keyspace */ + keyspace?: (topodata.IKeyspace|null); } - /** Represents a ReparentTabletResponse. */ - class ReparentTabletResponse implements IReparentTabletResponse { + /** Represents a SetKeyspaceDurabilityPolicyResponse. */ + class SetKeyspaceDurabilityPolicyResponse implements ISetKeyspaceDurabilityPolicyResponse { /** - * Constructs a new ReparentTabletResponse. + * Constructs a new SetKeyspaceDurabilityPolicyResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IReparentTabletResponse); - - /** ReparentTabletResponse keyspace. */ - public keyspace: string; - - /** ReparentTabletResponse shard. */ - public shard: string; + constructor(properties?: vtctldata.ISetKeyspaceDurabilityPolicyResponse); - /** ReparentTabletResponse primary. */ - public primary?: (topodata.ITabletAlias|null); + /** SetKeyspaceDurabilityPolicyResponse keyspace. */ + public keyspace?: (topodata.IKeyspace|null); /** - * Creates a new ReparentTabletResponse instance using the specified properties. + * Creates a new SetKeyspaceDurabilityPolicyResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ReparentTabletResponse instance + * @returns SetKeyspaceDurabilityPolicyResponse instance */ - public static create(properties?: vtctldata.IReparentTabletResponse): vtctldata.ReparentTabletResponse; + public static create(properties?: vtctldata.ISetKeyspaceDurabilityPolicyResponse): vtctldata.SetKeyspaceDurabilityPolicyResponse; /** - * Encodes the specified ReparentTabletResponse message. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. - * @param message ReparentTabletResponse message or plain object to encode + * Encodes the specified SetKeyspaceDurabilityPolicyResponse message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. + * @param message SetKeyspaceDurabilityPolicyResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IReparentTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceDurabilityPolicyResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ReparentTabletResponse message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. - * @param message ReparentTabletResponse message or plain object to encode + * Encodes the specified SetKeyspaceDurabilityPolicyResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. + * @param message SetKeyspaceDurabilityPolicyResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IReparentTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceDurabilityPolicyResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ReparentTabletResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ReparentTabletResponse + * @returns SetKeyspaceDurabilityPolicyResponse * @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.ReparentTabletResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceDurabilityPolicyResponse; /** - * Decodes a ReparentTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ReparentTabletResponse + * @returns SetKeyspaceDurabilityPolicyResponse * @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.ReparentTabletResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceDurabilityPolicyResponse; /** - * Verifies a ReparentTabletResponse message. + * Verifies a SetKeyspaceDurabilityPolicyResponse 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 ReparentTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceDurabilityPolicyResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ReparentTabletResponse + * @returns SetKeyspaceDurabilityPolicyResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ReparentTabletResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceDurabilityPolicyResponse; /** - * Creates a plain object from a ReparentTabletResponse message. Also converts values to other types if specified. - * @param message ReparentTabletResponse + * Creates a plain object from a SetKeyspaceDurabilityPolicyResponse message. Also converts values to other types if specified. + * @param message SetKeyspaceDurabilityPolicyResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ReparentTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceDurabilityPolicyResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ReparentTabletResponse to JSON. + * Converts this SetKeyspaceDurabilityPolicyResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ReparentTabletResponse + * Gets the default type url for SetKeyspaceDurabilityPolicyResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RestoreFromBackupRequest. */ - interface IRestoreFromBackupRequest { + /** Properties of a SetKeyspaceServedFromRequest. */ + interface ISetKeyspaceServedFromRequest { - /** RestoreFromBackupRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** SetKeyspaceServedFromRequest keyspace */ + keyspace?: (string|null); - /** RestoreFromBackupRequest backup_time */ - backup_time?: (vttime.ITime|null); + /** SetKeyspaceServedFromRequest tablet_type */ + tablet_type?: (topodata.TabletType|null); - /** RestoreFromBackupRequest restore_to_pos */ - restore_to_pos?: (string|null); + /** SetKeyspaceServedFromRequest cells */ + cells?: (string[]|null); - /** RestoreFromBackupRequest dry_run */ - dry_run?: (boolean|null); + /** SetKeyspaceServedFromRequest remove */ + remove?: (boolean|null); - /** RestoreFromBackupRequest restore_to_timestamp */ - restore_to_timestamp?: (vttime.ITime|null); + /** SetKeyspaceServedFromRequest source_keyspace */ + source_keyspace?: (string|null); } - /** Represents a RestoreFromBackupRequest. */ - class RestoreFromBackupRequest implements IRestoreFromBackupRequest { + /** Represents a SetKeyspaceServedFromRequest. */ + class SetKeyspaceServedFromRequest implements ISetKeyspaceServedFromRequest { /** - * Constructs a new RestoreFromBackupRequest. + * Constructs a new SetKeyspaceServedFromRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRestoreFromBackupRequest); + constructor(properties?: vtctldata.ISetKeyspaceServedFromRequest); - /** RestoreFromBackupRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** SetKeyspaceServedFromRequest keyspace. */ + public keyspace: string; - /** RestoreFromBackupRequest backup_time. */ - public backup_time?: (vttime.ITime|null); + /** SetKeyspaceServedFromRequest tablet_type. */ + public tablet_type: topodata.TabletType; - /** RestoreFromBackupRequest restore_to_pos. */ - public restore_to_pos: string; + /** SetKeyspaceServedFromRequest cells. */ + public cells: string[]; - /** RestoreFromBackupRequest dry_run. */ - public dry_run: boolean; + /** SetKeyspaceServedFromRequest remove. */ + public remove: boolean; - /** RestoreFromBackupRequest restore_to_timestamp. */ - public restore_to_timestamp?: (vttime.ITime|null); + /** SetKeyspaceServedFromRequest source_keyspace. */ + public source_keyspace: string; /** - * Creates a new RestoreFromBackupRequest instance using the specified properties. + * Creates a new SetKeyspaceServedFromRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RestoreFromBackupRequest instance + * @returns SetKeyspaceServedFromRequest instance */ - public static create(properties?: vtctldata.IRestoreFromBackupRequest): vtctldata.RestoreFromBackupRequest; + public static create(properties?: vtctldata.ISetKeyspaceServedFromRequest): vtctldata.SetKeyspaceServedFromRequest; /** - * Encodes the specified RestoreFromBackupRequest message. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. - * @param message RestoreFromBackupRequest message or plain object to encode + * Encodes the specified SetKeyspaceServedFromRequest message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. + * @param message SetKeyspaceServedFromRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRestoreFromBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceServedFromRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RestoreFromBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. - * @param message RestoreFromBackupRequest message or plain object to encode + * Encodes the specified SetKeyspaceServedFromRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. + * @param message SetKeyspaceServedFromRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRestoreFromBackupRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceServedFromRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RestoreFromBackupRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RestoreFromBackupRequest + * @returns SetKeyspaceServedFromRequest * @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.RestoreFromBackupRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceServedFromRequest; /** - * Decodes a RestoreFromBackupRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RestoreFromBackupRequest + * @returns SetKeyspaceServedFromRequest * @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.RestoreFromBackupRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceServedFromRequest; /** - * Verifies a RestoreFromBackupRequest message. + * Verifies a SetKeyspaceServedFromRequest 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 RestoreFromBackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceServedFromRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RestoreFromBackupRequest + * @returns SetKeyspaceServedFromRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RestoreFromBackupRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceServedFromRequest; /** - * Creates a plain object from a RestoreFromBackupRequest message. Also converts values to other types if specified. - * @param message RestoreFromBackupRequest + * Creates a plain object from a SetKeyspaceServedFromRequest message. Also converts values to other types if specified. + * @param message SetKeyspaceServedFromRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RestoreFromBackupRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceServedFromRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RestoreFromBackupRequest to JSON. + * Converts this SetKeyspaceServedFromRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RestoreFromBackupRequest + * Gets the default type url for SetKeyspaceServedFromRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RestoreFromBackupResponse. */ - interface IRestoreFromBackupResponse { - - /** RestoreFromBackupResponse tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); - - /** RestoreFromBackupResponse keyspace */ - keyspace?: (string|null); - - /** RestoreFromBackupResponse shard */ - shard?: (string|null); + /** Properties of a SetKeyspaceServedFromResponse. */ + interface ISetKeyspaceServedFromResponse { - /** RestoreFromBackupResponse event */ - event?: (logutil.IEvent|null); + /** SetKeyspaceServedFromResponse keyspace */ + keyspace?: (topodata.IKeyspace|null); } - /** Represents a RestoreFromBackupResponse. */ - class RestoreFromBackupResponse implements IRestoreFromBackupResponse { + /** Represents a SetKeyspaceServedFromResponse. */ + class SetKeyspaceServedFromResponse implements ISetKeyspaceServedFromResponse { /** - * Constructs a new RestoreFromBackupResponse. + * Constructs a new SetKeyspaceServedFromResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRestoreFromBackupResponse); - - /** RestoreFromBackupResponse tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); - - /** RestoreFromBackupResponse keyspace. */ - public keyspace: string; - - /** RestoreFromBackupResponse shard. */ - public shard: string; + constructor(properties?: vtctldata.ISetKeyspaceServedFromResponse); - /** RestoreFromBackupResponse event. */ - public event?: (logutil.IEvent|null); + /** SetKeyspaceServedFromResponse keyspace. */ + public keyspace?: (topodata.IKeyspace|null); /** - * Creates a new RestoreFromBackupResponse instance using the specified properties. + * Creates a new SetKeyspaceServedFromResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RestoreFromBackupResponse instance + * @returns SetKeyspaceServedFromResponse instance */ - public static create(properties?: vtctldata.IRestoreFromBackupResponse): vtctldata.RestoreFromBackupResponse; + public static create(properties?: vtctldata.ISetKeyspaceServedFromResponse): vtctldata.SetKeyspaceServedFromResponse; /** - * Encodes the specified RestoreFromBackupResponse message. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. - * @param message RestoreFromBackupResponse message or plain object to encode + * Encodes the specified SetKeyspaceServedFromResponse message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. + * @param message SetKeyspaceServedFromResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRestoreFromBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceServedFromResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RestoreFromBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. - * @param message RestoreFromBackupResponse message or plain object to encode + * Encodes the specified SetKeyspaceServedFromResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. + * @param message SetKeyspaceServedFromResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRestoreFromBackupResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceServedFromResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RestoreFromBackupResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RestoreFromBackupResponse + * @returns SetKeyspaceServedFromResponse * @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.RestoreFromBackupResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceServedFromResponse; /** - * Decodes a RestoreFromBackupResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RestoreFromBackupResponse + * @returns SetKeyspaceServedFromResponse * @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.RestoreFromBackupResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceServedFromResponse; /** - * Verifies a RestoreFromBackupResponse message. + * Verifies a SetKeyspaceServedFromResponse 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 RestoreFromBackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceServedFromResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RestoreFromBackupResponse + * @returns SetKeyspaceServedFromResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RestoreFromBackupResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceServedFromResponse; /** - * Creates a plain object from a RestoreFromBackupResponse message. Also converts values to other types if specified. - * @param message RestoreFromBackupResponse + * Creates a plain object from a SetKeyspaceServedFromResponse message. Also converts values to other types if specified. + * @param message SetKeyspaceServedFromResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RestoreFromBackupResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceServedFromResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RestoreFromBackupResponse to JSON. + * Converts this SetKeyspaceServedFromResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RestoreFromBackupResponse + * Gets the default type url for SetKeyspaceServedFromResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RunHealthCheckRequest. */ - interface IRunHealthCheckRequest { + /** Properties of a SetKeyspaceShardingInfoRequest. */ + interface ISetKeyspaceShardingInfoRequest { - /** RunHealthCheckRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** SetKeyspaceShardingInfoRequest keyspace */ + keyspace?: (string|null); + + /** SetKeyspaceShardingInfoRequest force */ + force?: (boolean|null); } - /** Represents a RunHealthCheckRequest. */ - class RunHealthCheckRequest implements IRunHealthCheckRequest { + /** Represents a SetKeyspaceShardingInfoRequest. */ + class SetKeyspaceShardingInfoRequest implements ISetKeyspaceShardingInfoRequest { /** - * Constructs a new RunHealthCheckRequest. + * Constructs a new SetKeyspaceShardingInfoRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRunHealthCheckRequest); + constructor(properties?: vtctldata.ISetKeyspaceShardingInfoRequest); - /** RunHealthCheckRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** SetKeyspaceShardingInfoRequest keyspace. */ + public keyspace: string; + + /** SetKeyspaceShardingInfoRequest force. */ + public force: boolean; /** - * Creates a new RunHealthCheckRequest instance using the specified properties. + * Creates a new SetKeyspaceShardingInfoRequest instance using the specified properties. * @param [properties] Properties to set - * @returns RunHealthCheckRequest instance + * @returns SetKeyspaceShardingInfoRequest instance */ - public static create(properties?: vtctldata.IRunHealthCheckRequest): vtctldata.RunHealthCheckRequest; + public static create(properties?: vtctldata.ISetKeyspaceShardingInfoRequest): vtctldata.SetKeyspaceShardingInfoRequest; /** - * Encodes the specified RunHealthCheckRequest message. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. - * @param message RunHealthCheckRequest message or plain object to encode + * Encodes the specified SetKeyspaceShardingInfoRequest message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. + * @param message SetKeyspaceShardingInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRunHealthCheckRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceShardingInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RunHealthCheckRequest message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. - * @param message RunHealthCheckRequest message or plain object to encode + * Encodes the specified SetKeyspaceShardingInfoRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. + * @param message SetKeyspaceShardingInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRunHealthCheckRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceShardingInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RunHealthCheckRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RunHealthCheckRequest + * @returns SetKeyspaceShardingInfoRequest * @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.RunHealthCheckRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceShardingInfoRequest; /** - * Decodes a RunHealthCheckRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RunHealthCheckRequest + * @returns SetKeyspaceShardingInfoRequest * @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.RunHealthCheckRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceShardingInfoRequest; /** - * Verifies a RunHealthCheckRequest message. + * Verifies a SetKeyspaceShardingInfoRequest 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 RunHealthCheckRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceShardingInfoRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RunHealthCheckRequest + * @returns SetKeyspaceShardingInfoRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.RunHealthCheckRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceShardingInfoRequest; /** - * Creates a plain object from a RunHealthCheckRequest message. Also converts values to other types if specified. - * @param message RunHealthCheckRequest + * Creates a plain object from a SetKeyspaceShardingInfoRequest message. Also converts values to other types if specified. + * @param message SetKeyspaceShardingInfoRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RunHealthCheckRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceShardingInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RunHealthCheckRequest to JSON. + * Converts this SetKeyspaceShardingInfoRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RunHealthCheckRequest + * Gets the default type url for SetKeyspaceShardingInfoRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a RunHealthCheckResponse. */ - interface IRunHealthCheckResponse { + /** Properties of a SetKeyspaceShardingInfoResponse. */ + interface ISetKeyspaceShardingInfoResponse { + + /** SetKeyspaceShardingInfoResponse keyspace */ + keyspace?: (topodata.IKeyspace|null); } - /** Represents a RunHealthCheckResponse. */ - class RunHealthCheckResponse implements IRunHealthCheckResponse { + /** Represents a SetKeyspaceShardingInfoResponse. */ + class SetKeyspaceShardingInfoResponse implements ISetKeyspaceShardingInfoResponse { /** - * Constructs a new RunHealthCheckResponse. + * Constructs a new SetKeyspaceShardingInfoResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IRunHealthCheckResponse); + constructor(properties?: vtctldata.ISetKeyspaceShardingInfoResponse); + + /** SetKeyspaceShardingInfoResponse keyspace. */ + public keyspace?: (topodata.IKeyspace|null); /** - * Creates a new RunHealthCheckResponse instance using the specified properties. + * Creates a new SetKeyspaceShardingInfoResponse instance using the specified properties. * @param [properties] Properties to set - * @returns RunHealthCheckResponse instance + * @returns SetKeyspaceShardingInfoResponse instance */ - public static create(properties?: vtctldata.IRunHealthCheckResponse): vtctldata.RunHealthCheckResponse; + public static create(properties?: vtctldata.ISetKeyspaceShardingInfoResponse): vtctldata.SetKeyspaceShardingInfoResponse; /** - * Encodes the specified RunHealthCheckResponse message. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. - * @param message RunHealthCheckResponse message or plain object to encode + * Encodes the specified SetKeyspaceShardingInfoResponse message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. + * @param message SetKeyspaceShardingInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IRunHealthCheckResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetKeyspaceShardingInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified RunHealthCheckResponse message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. - * @param message RunHealthCheckResponse message or plain object to encode + * Encodes the specified SetKeyspaceShardingInfoResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. + * @param message SetKeyspaceShardingInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IRunHealthCheckResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetKeyspaceShardingInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a RunHealthCheckResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns RunHealthCheckResponse + * @returns SetKeyspaceShardingInfoResponse * @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.RunHealthCheckResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetKeyspaceShardingInfoResponse; /** - * Decodes a RunHealthCheckResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns RunHealthCheckResponse + * @returns SetKeyspaceShardingInfoResponse * @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.RunHealthCheckResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetKeyspaceShardingInfoResponse; /** - * Verifies a RunHealthCheckResponse message. + * Verifies a SetKeyspaceShardingInfoResponse 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 RunHealthCheckResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceShardingInfoResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns RunHealthCheckResponse + * @returns SetKeyspaceShardingInfoResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.RunHealthCheckResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceShardingInfoResponse; /** - * Creates a plain object from a RunHealthCheckResponse message. Also converts values to other types if specified. - * @param message RunHealthCheckResponse + * Creates a plain object from a SetKeyspaceShardingInfoResponse message. Also converts values to other types if specified. + * @param message SetKeyspaceShardingInfoResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.RunHealthCheckResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetKeyspaceShardingInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this RunHealthCheckResponse to JSON. + * Converts this SetKeyspaceShardingInfoResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for RunHealthCheckResponse + * Gets the default type url for SetKeyspaceShardingInfoResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceDurabilityPolicyRequest. */ - interface ISetKeyspaceDurabilityPolicyRequest { + /** Properties of a SetShardIsPrimaryServingRequest. */ + interface ISetShardIsPrimaryServingRequest { - /** SetKeyspaceDurabilityPolicyRequest keyspace */ + /** SetShardIsPrimaryServingRequest keyspace */ keyspace?: (string|null); - /** SetKeyspaceDurabilityPolicyRequest durability_policy */ - durability_policy?: (string|null); + /** SetShardIsPrimaryServingRequest shard */ + shard?: (string|null); + + /** SetShardIsPrimaryServingRequest is_serving */ + is_serving?: (boolean|null); } - /** Represents a SetKeyspaceDurabilityPolicyRequest. */ - class SetKeyspaceDurabilityPolicyRequest implements ISetKeyspaceDurabilityPolicyRequest { + /** Represents a SetShardIsPrimaryServingRequest. */ + class SetShardIsPrimaryServingRequest implements ISetShardIsPrimaryServingRequest { /** - * Constructs a new SetKeyspaceDurabilityPolicyRequest. + * Constructs a new SetShardIsPrimaryServingRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceDurabilityPolicyRequest); + constructor(properties?: vtctldata.ISetShardIsPrimaryServingRequest); - /** SetKeyspaceDurabilityPolicyRequest keyspace. */ + /** SetShardIsPrimaryServingRequest keyspace. */ public keyspace: string; - /** SetKeyspaceDurabilityPolicyRequest durability_policy. */ - public durability_policy: string; + /** SetShardIsPrimaryServingRequest shard. */ + public shard: string; + + /** SetShardIsPrimaryServingRequest is_serving. */ + public is_serving: boolean; /** - * Creates a new SetKeyspaceDurabilityPolicyRequest instance using the specified properties. + * Creates a new SetShardIsPrimaryServingRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceDurabilityPolicyRequest instance + * @returns SetShardIsPrimaryServingRequest instance */ - public static create(properties?: vtctldata.ISetKeyspaceDurabilityPolicyRequest): vtctldata.SetKeyspaceDurabilityPolicyRequest; + public static create(properties?: vtctldata.ISetShardIsPrimaryServingRequest): vtctldata.SetShardIsPrimaryServingRequest; /** - * Encodes the specified SetKeyspaceDurabilityPolicyRequest message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. - * @param message SetKeyspaceDurabilityPolicyRequest message or plain object to encode + * Encodes the specified SetShardIsPrimaryServingRequest message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. + * @param message SetShardIsPrimaryServingRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceDurabilityPolicyRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetShardIsPrimaryServingRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceDurabilityPolicyRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. - * @param message SetKeyspaceDurabilityPolicyRequest message or plain object to encode + * Encodes the specified SetShardIsPrimaryServingRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. + * @param message SetShardIsPrimaryServingRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceDurabilityPolicyRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetShardIsPrimaryServingRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer. + * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceDurabilityPolicyRequest + * @returns SetShardIsPrimaryServingRequest * @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.SetKeyspaceDurabilityPolicyRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetShardIsPrimaryServingRequest; /** - * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer, length delimited. + * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceDurabilityPolicyRequest + * @returns SetShardIsPrimaryServingRequest * @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.SetKeyspaceDurabilityPolicyRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetShardIsPrimaryServingRequest; /** - * Verifies a SetKeyspaceDurabilityPolicyRequest message. + * Verifies a SetShardIsPrimaryServingRequest 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 SetKeyspaceDurabilityPolicyRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardIsPrimaryServingRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceDurabilityPolicyRequest + * @returns SetShardIsPrimaryServingRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceDurabilityPolicyRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetShardIsPrimaryServingRequest; /** - * Creates a plain object from a SetKeyspaceDurabilityPolicyRequest message. Also converts values to other types if specified. - * @param message SetKeyspaceDurabilityPolicyRequest + * Creates a plain object from a SetShardIsPrimaryServingRequest message. Also converts values to other types if specified. + * @param message SetShardIsPrimaryServingRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceDurabilityPolicyRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetShardIsPrimaryServingRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceDurabilityPolicyRequest to JSON. + * Converts this SetShardIsPrimaryServingRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceDurabilityPolicyRequest + * Gets the default type url for SetShardIsPrimaryServingRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceDurabilityPolicyResponse. */ - interface ISetKeyspaceDurabilityPolicyResponse { + /** Properties of a SetShardIsPrimaryServingResponse. */ + interface ISetShardIsPrimaryServingResponse { - /** SetKeyspaceDurabilityPolicyResponse keyspace */ - keyspace?: (topodata.IKeyspace|null); + /** SetShardIsPrimaryServingResponse shard */ + shard?: (topodata.IShard|null); } - /** Represents a SetKeyspaceDurabilityPolicyResponse. */ - class SetKeyspaceDurabilityPolicyResponse implements ISetKeyspaceDurabilityPolicyResponse { + /** Represents a SetShardIsPrimaryServingResponse. */ + class SetShardIsPrimaryServingResponse implements ISetShardIsPrimaryServingResponse { /** - * Constructs a new SetKeyspaceDurabilityPolicyResponse. + * Constructs a new SetShardIsPrimaryServingResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceDurabilityPolicyResponse); + constructor(properties?: vtctldata.ISetShardIsPrimaryServingResponse); - /** SetKeyspaceDurabilityPolicyResponse keyspace. */ - public keyspace?: (topodata.IKeyspace|null); + /** SetShardIsPrimaryServingResponse shard. */ + public shard?: (topodata.IShard|null); /** - * Creates a new SetKeyspaceDurabilityPolicyResponse instance using the specified properties. + * Creates a new SetShardIsPrimaryServingResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceDurabilityPolicyResponse instance + * @returns SetShardIsPrimaryServingResponse instance */ - public static create(properties?: vtctldata.ISetKeyspaceDurabilityPolicyResponse): vtctldata.SetKeyspaceDurabilityPolicyResponse; + public static create(properties?: vtctldata.ISetShardIsPrimaryServingResponse): vtctldata.SetShardIsPrimaryServingResponse; /** - * Encodes the specified SetKeyspaceDurabilityPolicyResponse message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. - * @param message SetKeyspaceDurabilityPolicyResponse message or plain object to encode + * Encodes the specified SetShardIsPrimaryServingResponse message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. + * @param message SetShardIsPrimaryServingResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceDurabilityPolicyResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetShardIsPrimaryServingResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceDurabilityPolicyResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. - * @param message SetKeyspaceDurabilityPolicyResponse message or plain object to encode + * Encodes the specified SetShardIsPrimaryServingResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. + * @param message SetShardIsPrimaryServingResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceDurabilityPolicyResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetShardIsPrimaryServingResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer. + * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceDurabilityPolicyResponse + * @returns SetShardIsPrimaryServingResponse * @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.SetKeyspaceDurabilityPolicyResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetShardIsPrimaryServingResponse; /** - * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer, length delimited. + * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceDurabilityPolicyResponse + * @returns SetShardIsPrimaryServingResponse * @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.SetKeyspaceDurabilityPolicyResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetShardIsPrimaryServingResponse; /** - * Verifies a SetKeyspaceDurabilityPolicyResponse message. + * Verifies a SetShardIsPrimaryServingResponse 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 SetKeyspaceDurabilityPolicyResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardIsPrimaryServingResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceDurabilityPolicyResponse + * @returns SetShardIsPrimaryServingResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceDurabilityPolicyResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetShardIsPrimaryServingResponse; /** - * Creates a plain object from a SetKeyspaceDurabilityPolicyResponse message. Also converts values to other types if specified. - * @param message SetKeyspaceDurabilityPolicyResponse + * Creates a plain object from a SetShardIsPrimaryServingResponse message. Also converts values to other types if specified. + * @param message SetShardIsPrimaryServingResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceDurabilityPolicyResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetShardIsPrimaryServingResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceDurabilityPolicyResponse to JSON. + * Converts this SetShardIsPrimaryServingResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceDurabilityPolicyResponse + * Gets the default type url for SetShardIsPrimaryServingResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceServedFromRequest. */ - interface ISetKeyspaceServedFromRequest { + /** Properties of a SetShardTabletControlRequest. */ + interface ISetShardTabletControlRequest { - /** SetKeyspaceServedFromRequest keyspace */ + /** SetShardTabletControlRequest keyspace */ keyspace?: (string|null); - /** SetKeyspaceServedFromRequest tablet_type */ + /** SetShardTabletControlRequest shard */ + shard?: (string|null); + + /** SetShardTabletControlRequest tablet_type */ tablet_type?: (topodata.TabletType|null); - /** SetKeyspaceServedFromRequest cells */ + /** SetShardTabletControlRequest cells */ cells?: (string[]|null); - /** SetKeyspaceServedFromRequest remove */ - remove?: (boolean|null); + /** SetShardTabletControlRequest denied_tables */ + denied_tables?: (string[]|null); - /** SetKeyspaceServedFromRequest source_keyspace */ - source_keyspace?: (string|null); + /** SetShardTabletControlRequest disable_query_service */ + disable_query_service?: (boolean|null); + + /** SetShardTabletControlRequest remove */ + remove?: (boolean|null); } - /** Represents a SetKeyspaceServedFromRequest. */ - class SetKeyspaceServedFromRequest implements ISetKeyspaceServedFromRequest { + /** Represents a SetShardTabletControlRequest. */ + class SetShardTabletControlRequest implements ISetShardTabletControlRequest { /** - * Constructs a new SetKeyspaceServedFromRequest. + * Constructs a new SetShardTabletControlRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceServedFromRequest); + constructor(properties?: vtctldata.ISetShardTabletControlRequest); - /** SetKeyspaceServedFromRequest keyspace. */ + /** SetShardTabletControlRequest keyspace. */ public keyspace: string; - /** SetKeyspaceServedFromRequest tablet_type. */ + /** SetShardTabletControlRequest shard. */ + public shard: string; + + /** SetShardTabletControlRequest tablet_type. */ public tablet_type: topodata.TabletType; - /** SetKeyspaceServedFromRequest cells. */ + /** SetShardTabletControlRequest cells. */ public cells: string[]; - /** SetKeyspaceServedFromRequest remove. */ - public remove: boolean; + /** SetShardTabletControlRequest denied_tables. */ + public denied_tables: string[]; - /** SetKeyspaceServedFromRequest source_keyspace. */ - public source_keyspace: string; + /** SetShardTabletControlRequest disable_query_service. */ + public disable_query_service: boolean; + + /** SetShardTabletControlRequest remove. */ + public remove: boolean; /** - * Creates a new SetKeyspaceServedFromRequest instance using the specified properties. + * Creates a new SetShardTabletControlRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceServedFromRequest instance + * @returns SetShardTabletControlRequest instance */ - public static create(properties?: vtctldata.ISetKeyspaceServedFromRequest): vtctldata.SetKeyspaceServedFromRequest; + public static create(properties?: vtctldata.ISetShardTabletControlRequest): vtctldata.SetShardTabletControlRequest; /** - * Encodes the specified SetKeyspaceServedFromRequest message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. - * @param message SetKeyspaceServedFromRequest message or plain object to encode + * Encodes the specified SetShardTabletControlRequest message. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. + * @param message SetShardTabletControlRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceServedFromRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetShardTabletControlRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceServedFromRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. - * @param message SetKeyspaceServedFromRequest message or plain object to encode + * Encodes the specified SetShardTabletControlRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. + * @param message SetShardTabletControlRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceServedFromRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetShardTabletControlRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer. + * Decodes a SetShardTabletControlRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceServedFromRequest + * @returns SetShardTabletControlRequest * @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.SetKeyspaceServedFromRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetShardTabletControlRequest; /** - * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer, length delimited. + * Decodes a SetShardTabletControlRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceServedFromRequest + * @returns SetShardTabletControlRequest * @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.SetKeyspaceServedFromRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetShardTabletControlRequest; /** - * Verifies a SetKeyspaceServedFromRequest message. + * Verifies a SetShardTabletControlRequest 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 SetKeyspaceServedFromRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardTabletControlRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceServedFromRequest + * @returns SetShardTabletControlRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceServedFromRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetShardTabletControlRequest; /** - * Creates a plain object from a SetKeyspaceServedFromRequest message. Also converts values to other types if specified. - * @param message SetKeyspaceServedFromRequest + * Creates a plain object from a SetShardTabletControlRequest message. Also converts values to other types if specified. + * @param message SetShardTabletControlRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceServedFromRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetShardTabletControlRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceServedFromRequest to JSON. + * Converts this SetShardTabletControlRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceServedFromRequest + * Gets the default type url for SetShardTabletControlRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceServedFromResponse. */ - interface ISetKeyspaceServedFromResponse { + /** Properties of a SetShardTabletControlResponse. */ + interface ISetShardTabletControlResponse { - /** SetKeyspaceServedFromResponse keyspace */ - keyspace?: (topodata.IKeyspace|null); + /** SetShardTabletControlResponse shard */ + shard?: (topodata.IShard|null); } - /** Represents a SetKeyspaceServedFromResponse. */ - class SetKeyspaceServedFromResponse implements ISetKeyspaceServedFromResponse { + /** Represents a SetShardTabletControlResponse. */ + class SetShardTabletControlResponse implements ISetShardTabletControlResponse { /** - * Constructs a new SetKeyspaceServedFromResponse. + * Constructs a new SetShardTabletControlResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceServedFromResponse); + constructor(properties?: vtctldata.ISetShardTabletControlResponse); - /** SetKeyspaceServedFromResponse keyspace. */ - public keyspace?: (topodata.IKeyspace|null); + /** SetShardTabletControlResponse shard. */ + public shard?: (topodata.IShard|null); /** - * Creates a new SetKeyspaceServedFromResponse instance using the specified properties. + * Creates a new SetShardTabletControlResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceServedFromResponse instance + * @returns SetShardTabletControlResponse instance */ - public static create(properties?: vtctldata.ISetKeyspaceServedFromResponse): vtctldata.SetKeyspaceServedFromResponse; + public static create(properties?: vtctldata.ISetShardTabletControlResponse): vtctldata.SetShardTabletControlResponse; /** - * Encodes the specified SetKeyspaceServedFromResponse message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. - * @param message SetKeyspaceServedFromResponse message or plain object to encode + * Encodes the specified SetShardTabletControlResponse message. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. + * @param message SetShardTabletControlResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceServedFromResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetShardTabletControlResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceServedFromResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. - * @param message SetKeyspaceServedFromResponse message or plain object to encode + * Encodes the specified SetShardTabletControlResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. + * @param message SetShardTabletControlResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceServedFromResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetShardTabletControlResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer. + * Decodes a SetShardTabletControlResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceServedFromResponse + * @returns SetShardTabletControlResponse * @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.SetKeyspaceServedFromResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetShardTabletControlResponse; /** - * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer, length delimited. + * Decodes a SetShardTabletControlResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceServedFromResponse + * @returns SetShardTabletControlResponse * @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.SetKeyspaceServedFromResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetShardTabletControlResponse; /** - * Verifies a SetKeyspaceServedFromResponse message. + * Verifies a SetShardTabletControlResponse 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 SetKeyspaceServedFromResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardTabletControlResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceServedFromResponse + * @returns SetShardTabletControlResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceServedFromResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetShardTabletControlResponse; /** - * Creates a plain object from a SetKeyspaceServedFromResponse message. Also converts values to other types if specified. - * @param message SetKeyspaceServedFromResponse + * Creates a plain object from a SetShardTabletControlResponse message. Also converts values to other types if specified. + * @param message SetShardTabletControlResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceServedFromResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetShardTabletControlResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceServedFromResponse to JSON. + * Converts this SetShardTabletControlResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceServedFromResponse + * Gets the default type url for SetShardTabletControlResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceShardingInfoRequest. */ - interface ISetKeyspaceShardingInfoRequest { + /** Properties of a SetWritableRequest. */ + interface ISetWritableRequest { - /** SetKeyspaceShardingInfoRequest keyspace */ - keyspace?: (string|null); + /** SetWritableRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** SetKeyspaceShardingInfoRequest force */ - force?: (boolean|null); + /** SetWritableRequest writable */ + writable?: (boolean|null); } - /** Represents a SetKeyspaceShardingInfoRequest. */ - class SetKeyspaceShardingInfoRequest implements ISetKeyspaceShardingInfoRequest { + /** Represents a SetWritableRequest. */ + class SetWritableRequest implements ISetWritableRequest { /** - * Constructs a new SetKeyspaceShardingInfoRequest. + * Constructs a new SetWritableRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceShardingInfoRequest); + constructor(properties?: vtctldata.ISetWritableRequest); - /** SetKeyspaceShardingInfoRequest keyspace. */ - public keyspace: string; + /** SetWritableRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** SetKeyspaceShardingInfoRequest force. */ - public force: boolean; + /** SetWritableRequest writable. */ + public writable: boolean; /** - * Creates a new SetKeyspaceShardingInfoRequest instance using the specified properties. + * Creates a new SetWritableRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceShardingInfoRequest instance + * @returns SetWritableRequest instance */ - public static create(properties?: vtctldata.ISetKeyspaceShardingInfoRequest): vtctldata.SetKeyspaceShardingInfoRequest; + public static create(properties?: vtctldata.ISetWritableRequest): vtctldata.SetWritableRequest; /** - * Encodes the specified SetKeyspaceShardingInfoRequest message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. - * @param message SetKeyspaceShardingInfoRequest message or plain object to encode + * Encodes the specified SetWritableRequest message. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. + * @param message SetWritableRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceShardingInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetWritableRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceShardingInfoRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. - * @param message SetKeyspaceShardingInfoRequest message or plain object to encode + * Encodes the specified SetWritableRequest message, length delimited. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. + * @param message SetWritableRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceShardingInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetWritableRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer. + * Decodes a SetWritableRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceShardingInfoRequest + * @returns SetWritableRequest * @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.SetKeyspaceShardingInfoRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetWritableRequest; /** - * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a SetWritableRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceShardingInfoRequest + * @returns SetWritableRequest * @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.SetKeyspaceShardingInfoRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetWritableRequest; /** - * Verifies a SetKeyspaceShardingInfoRequest message. + * Verifies a SetWritableRequest 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 SetKeyspaceShardingInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetWritableRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceShardingInfoRequest + * @returns SetWritableRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceShardingInfoRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SetWritableRequest; /** - * Creates a plain object from a SetKeyspaceShardingInfoRequest message. Also converts values to other types if specified. - * @param message SetKeyspaceShardingInfoRequest + * Creates a plain object from a SetWritableRequest message. Also converts values to other types if specified. + * @param message SetWritableRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceShardingInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetWritableRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceShardingInfoRequest to JSON. + * Converts this SetWritableRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceShardingInfoRequest + * Gets the default type url for SetWritableRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetKeyspaceShardingInfoResponse. */ - interface ISetKeyspaceShardingInfoResponse { - - /** SetKeyspaceShardingInfoResponse keyspace */ - keyspace?: (topodata.IKeyspace|null); + /** Properties of a SetWritableResponse. */ + interface ISetWritableResponse { } - /** Represents a SetKeyspaceShardingInfoResponse. */ - class SetKeyspaceShardingInfoResponse implements ISetKeyspaceShardingInfoResponse { + /** Represents a SetWritableResponse. */ + class SetWritableResponse implements ISetWritableResponse { /** - * Constructs a new SetKeyspaceShardingInfoResponse. + * Constructs a new SetWritableResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetKeyspaceShardingInfoResponse); - - /** SetKeyspaceShardingInfoResponse keyspace. */ - public keyspace?: (topodata.IKeyspace|null); + constructor(properties?: vtctldata.ISetWritableResponse); /** - * Creates a new SetKeyspaceShardingInfoResponse instance using the specified properties. + * Creates a new SetWritableResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetKeyspaceShardingInfoResponse instance + * @returns SetWritableResponse instance */ - public static create(properties?: vtctldata.ISetKeyspaceShardingInfoResponse): vtctldata.SetKeyspaceShardingInfoResponse; + public static create(properties?: vtctldata.ISetWritableResponse): vtctldata.SetWritableResponse; /** - * Encodes the specified SetKeyspaceShardingInfoResponse message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. - * @param message SetKeyspaceShardingInfoResponse message or plain object to encode + * Encodes the specified SetWritableResponse message. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. + * @param message SetWritableResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetKeyspaceShardingInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISetWritableResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetKeyspaceShardingInfoResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. - * @param message SetKeyspaceShardingInfoResponse message or plain object to encode + * Encodes the specified SetWritableResponse message, length delimited. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. + * @param message SetWritableResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetKeyspaceShardingInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISetWritableResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer. + * Decodes a SetWritableResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetKeyspaceShardingInfoResponse + * @returns SetWritableResponse * @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.SetKeyspaceShardingInfoResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SetWritableResponse; /** - * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a SetWritableResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetKeyspaceShardingInfoResponse + * @returns SetWritableResponse * @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.SetKeyspaceShardingInfoResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SetWritableResponse; /** - * Verifies a SetKeyspaceShardingInfoResponse message. + * Verifies a SetWritableResponse 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 SetKeyspaceShardingInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetWritableResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetKeyspaceShardingInfoResponse + * @returns SetWritableResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetKeyspaceShardingInfoResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SetWritableResponse; /** - * Creates a plain object from a SetKeyspaceShardingInfoResponse message. Also converts values to other types if specified. - * @param message SetKeyspaceShardingInfoResponse + * Creates a plain object from a SetWritableResponse message. Also converts values to other types if specified. + * @param message SetWritableResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetKeyspaceShardingInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SetWritableResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetKeyspaceShardingInfoResponse to JSON. + * Converts this SetWritableResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetKeyspaceShardingInfoResponse + * Gets the default type url for SetWritableResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetShardIsPrimaryServingRequest. */ - interface ISetShardIsPrimaryServingRequest { + /** Properties of a ShardReplicationAddRequest. */ + interface IShardReplicationAddRequest { - /** SetShardIsPrimaryServingRequest keyspace */ + /** ShardReplicationAddRequest keyspace */ keyspace?: (string|null); - /** SetShardIsPrimaryServingRequest shard */ + /** ShardReplicationAddRequest shard */ shard?: (string|null); - /** SetShardIsPrimaryServingRequest is_serving */ - is_serving?: (boolean|null); + /** ShardReplicationAddRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a SetShardIsPrimaryServingRequest. */ - class SetShardIsPrimaryServingRequest implements ISetShardIsPrimaryServingRequest { + /** Represents a ShardReplicationAddRequest. */ + class ShardReplicationAddRequest implements IShardReplicationAddRequest { /** - * Constructs a new SetShardIsPrimaryServingRequest. + * Constructs a new ShardReplicationAddRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetShardIsPrimaryServingRequest); + constructor(properties?: vtctldata.IShardReplicationAddRequest); - /** SetShardIsPrimaryServingRequest keyspace. */ + /** ShardReplicationAddRequest keyspace. */ public keyspace: string; - /** SetShardIsPrimaryServingRequest shard. */ + /** ShardReplicationAddRequest shard. */ public shard: string; - /** SetShardIsPrimaryServingRequest is_serving. */ - public is_serving: boolean; + /** ShardReplicationAddRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new SetShardIsPrimaryServingRequest instance using the specified properties. + * Creates a new ShardReplicationAddRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetShardIsPrimaryServingRequest instance + * @returns ShardReplicationAddRequest instance */ - public static create(properties?: vtctldata.ISetShardIsPrimaryServingRequest): vtctldata.SetShardIsPrimaryServingRequest; + public static create(properties?: vtctldata.IShardReplicationAddRequest): vtctldata.ShardReplicationAddRequest; /** - * Encodes the specified SetShardIsPrimaryServingRequest message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. - * @param message SetShardIsPrimaryServingRequest message or plain object to encode + * Encodes the specified ShardReplicationAddRequest message. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. + * @param message ShardReplicationAddRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetShardIsPrimaryServingRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetShardIsPrimaryServingRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. - * @param message SetShardIsPrimaryServingRequest message or plain object to encode + * Encodes the specified ShardReplicationAddRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. + * @param message ShardReplicationAddRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetShardIsPrimaryServingRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer. + * Decodes a ShardReplicationAddRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetShardIsPrimaryServingRequest + * @returns ShardReplicationAddRequest * @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.SetShardIsPrimaryServingRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationAddRequest; /** - * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationAddRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetShardIsPrimaryServingRequest + * @returns ShardReplicationAddRequest * @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.SetShardIsPrimaryServingRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationAddRequest; /** - * Verifies a SetShardIsPrimaryServingRequest message. + * Verifies a ShardReplicationAddRequest 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 SetShardIsPrimaryServingRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationAddRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetShardIsPrimaryServingRequest + * @returns ShardReplicationAddRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetShardIsPrimaryServingRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationAddRequest; /** - * Creates a plain object from a SetShardIsPrimaryServingRequest message. Also converts values to other types if specified. - * @param message SetShardIsPrimaryServingRequest + * Creates a plain object from a ShardReplicationAddRequest message. Also converts values to other types if specified. + * @param message ShardReplicationAddRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetShardIsPrimaryServingRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationAddRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetShardIsPrimaryServingRequest to JSON. + * Converts this ShardReplicationAddRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetShardIsPrimaryServingRequest + * Gets the default type url for ShardReplicationAddRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetShardIsPrimaryServingResponse. */ - interface ISetShardIsPrimaryServingResponse { - - /** SetShardIsPrimaryServingResponse shard */ - shard?: (topodata.IShard|null); + /** Properties of a ShardReplicationAddResponse. */ + interface IShardReplicationAddResponse { } - /** Represents a SetShardIsPrimaryServingResponse. */ - class SetShardIsPrimaryServingResponse implements ISetShardIsPrimaryServingResponse { + /** Represents a ShardReplicationAddResponse. */ + class ShardReplicationAddResponse implements IShardReplicationAddResponse { /** - * Constructs a new SetShardIsPrimaryServingResponse. + * Constructs a new ShardReplicationAddResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetShardIsPrimaryServingResponse); - - /** SetShardIsPrimaryServingResponse shard. */ - public shard?: (topodata.IShard|null); + constructor(properties?: vtctldata.IShardReplicationAddResponse); /** - * Creates a new SetShardIsPrimaryServingResponse instance using the specified properties. + * Creates a new ShardReplicationAddResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetShardIsPrimaryServingResponse instance + * @returns ShardReplicationAddResponse instance */ - public static create(properties?: vtctldata.ISetShardIsPrimaryServingResponse): vtctldata.SetShardIsPrimaryServingResponse; + public static create(properties?: vtctldata.IShardReplicationAddResponse): vtctldata.ShardReplicationAddResponse; /** - * Encodes the specified SetShardIsPrimaryServingResponse message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. - * @param message SetShardIsPrimaryServingResponse message or plain object to encode + * Encodes the specified ShardReplicationAddResponse message. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. + * @param message ShardReplicationAddResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetShardIsPrimaryServingResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetShardIsPrimaryServingResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. - * @param message SetShardIsPrimaryServingResponse message or plain object to encode + * Encodes the specified ShardReplicationAddResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. + * @param message ShardReplicationAddResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetShardIsPrimaryServingResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer. + * Decodes a ShardReplicationAddResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetShardIsPrimaryServingResponse + * @returns ShardReplicationAddResponse * @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.SetShardIsPrimaryServingResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationAddResponse; /** - * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationAddResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetShardIsPrimaryServingResponse + * @returns ShardReplicationAddResponse * @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.SetShardIsPrimaryServingResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationAddResponse; /** - * Verifies a SetShardIsPrimaryServingResponse message. + * Verifies a ShardReplicationAddResponse 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 SetShardIsPrimaryServingResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationAddResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetShardIsPrimaryServingResponse + * @returns ShardReplicationAddResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetShardIsPrimaryServingResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationAddResponse; /** - * Creates a plain object from a SetShardIsPrimaryServingResponse message. Also converts values to other types if specified. - * @param message SetShardIsPrimaryServingResponse + * Creates a plain object from a ShardReplicationAddResponse message. Also converts values to other types if specified. + * @param message ShardReplicationAddResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetShardIsPrimaryServingResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationAddResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetShardIsPrimaryServingResponse to JSON. + * Converts this ShardReplicationAddResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetShardIsPrimaryServingResponse + * Gets the default type url for ShardReplicationAddResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetShardTabletControlRequest. */ - interface ISetShardTabletControlRequest { + /** Properties of a ShardReplicationFixRequest. */ + interface IShardReplicationFixRequest { - /** SetShardTabletControlRequest keyspace */ + /** ShardReplicationFixRequest keyspace */ keyspace?: (string|null); - /** SetShardTabletControlRequest shard */ + /** ShardReplicationFixRequest shard */ shard?: (string|null); - /** SetShardTabletControlRequest tablet_type */ - tablet_type?: (topodata.TabletType|null); - - /** SetShardTabletControlRequest cells */ - cells?: (string[]|null); - - /** SetShardTabletControlRequest denied_tables */ - denied_tables?: (string[]|null); - - /** SetShardTabletControlRequest disable_query_service */ - disable_query_service?: (boolean|null); - - /** SetShardTabletControlRequest remove */ - remove?: (boolean|null); + /** ShardReplicationFixRequest cell */ + cell?: (string|null); } - /** Represents a SetShardTabletControlRequest. */ - class SetShardTabletControlRequest implements ISetShardTabletControlRequest { + /** Represents a ShardReplicationFixRequest. */ + class ShardReplicationFixRequest implements IShardReplicationFixRequest { /** - * Constructs a new SetShardTabletControlRequest. + * Constructs a new ShardReplicationFixRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetShardTabletControlRequest); + constructor(properties?: vtctldata.IShardReplicationFixRequest); - /** SetShardTabletControlRequest keyspace. */ + /** ShardReplicationFixRequest keyspace. */ public keyspace: string; - /** SetShardTabletControlRequest shard. */ + /** ShardReplicationFixRequest shard. */ public shard: string; - /** SetShardTabletControlRequest tablet_type. */ - public tablet_type: topodata.TabletType; - - /** SetShardTabletControlRequest cells. */ - public cells: string[]; - - /** SetShardTabletControlRequest denied_tables. */ - public denied_tables: string[]; - - /** SetShardTabletControlRequest disable_query_service. */ - public disable_query_service: boolean; - - /** SetShardTabletControlRequest remove. */ - public remove: boolean; + /** ShardReplicationFixRequest cell. */ + public cell: string; /** - * Creates a new SetShardTabletControlRequest instance using the specified properties. + * Creates a new ShardReplicationFixRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetShardTabletControlRequest instance + * @returns ShardReplicationFixRequest instance */ - public static create(properties?: vtctldata.ISetShardTabletControlRequest): vtctldata.SetShardTabletControlRequest; + public static create(properties?: vtctldata.IShardReplicationFixRequest): vtctldata.ShardReplicationFixRequest; /** - * Encodes the specified SetShardTabletControlRequest message. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. - * @param message SetShardTabletControlRequest message or plain object to encode + * Encodes the specified ShardReplicationFixRequest message. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. + * @param message ShardReplicationFixRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetShardTabletControlRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationFixRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetShardTabletControlRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. - * @param message SetShardTabletControlRequest message or plain object to encode + * Encodes the specified ShardReplicationFixRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. + * @param message ShardReplicationFixRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetShardTabletControlRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationFixRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetShardTabletControlRequest message from the specified reader or buffer. + * Decodes a ShardReplicationFixRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetShardTabletControlRequest + * @returns ShardReplicationFixRequest * @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.SetShardTabletControlRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationFixRequest; /** - * Decodes a SetShardTabletControlRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationFixRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetShardTabletControlRequest + * @returns ShardReplicationFixRequest * @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.SetShardTabletControlRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationFixRequest; /** - * Verifies a SetShardTabletControlRequest message. + * Verifies a ShardReplicationFixRequest 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 SetShardTabletControlRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationFixRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetShardTabletControlRequest + * @returns ShardReplicationFixRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetShardTabletControlRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationFixRequest; /** - * Creates a plain object from a SetShardTabletControlRequest message. Also converts values to other types if specified. - * @param message SetShardTabletControlRequest + * Creates a plain object from a ShardReplicationFixRequest message. Also converts values to other types if specified. + * @param message ShardReplicationFixRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetShardTabletControlRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationFixRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetShardTabletControlRequest to JSON. + * Converts this ShardReplicationFixRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetShardTabletControlRequest + * Gets the default type url for ShardReplicationFixRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetShardTabletControlResponse. */ - interface ISetShardTabletControlResponse { + /** Properties of a ShardReplicationFixResponse. */ + interface IShardReplicationFixResponse { - /** SetShardTabletControlResponse shard */ - shard?: (topodata.IShard|null); + /** ShardReplicationFixResponse error */ + error?: (topodata.IShardReplicationError|null); } - /** Represents a SetShardTabletControlResponse. */ - class SetShardTabletControlResponse implements ISetShardTabletControlResponse { + /** Represents a ShardReplicationFixResponse. */ + class ShardReplicationFixResponse implements IShardReplicationFixResponse { /** - * Constructs a new SetShardTabletControlResponse. + * Constructs a new ShardReplicationFixResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetShardTabletControlResponse); + constructor(properties?: vtctldata.IShardReplicationFixResponse); - /** SetShardTabletControlResponse shard. */ - public shard?: (topodata.IShard|null); + /** ShardReplicationFixResponse error. */ + public error?: (topodata.IShardReplicationError|null); /** - * Creates a new SetShardTabletControlResponse instance using the specified properties. + * Creates a new ShardReplicationFixResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetShardTabletControlResponse instance + * @returns ShardReplicationFixResponse instance */ - public static create(properties?: vtctldata.ISetShardTabletControlResponse): vtctldata.SetShardTabletControlResponse; + public static create(properties?: vtctldata.IShardReplicationFixResponse): vtctldata.ShardReplicationFixResponse; /** - * Encodes the specified SetShardTabletControlResponse message. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. - * @param message SetShardTabletControlResponse message or plain object to encode + * Encodes the specified ShardReplicationFixResponse message. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. + * @param message ShardReplicationFixResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetShardTabletControlResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationFixResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetShardTabletControlResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. - * @param message SetShardTabletControlResponse message or plain object to encode + * Encodes the specified ShardReplicationFixResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. + * @param message ShardReplicationFixResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetShardTabletControlResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationFixResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetShardTabletControlResponse message from the specified reader or buffer. + * Decodes a ShardReplicationFixResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetShardTabletControlResponse + * @returns ShardReplicationFixResponse * @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.SetShardTabletControlResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationFixResponse; /** - * Decodes a SetShardTabletControlResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationFixResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetShardTabletControlResponse + * @returns ShardReplicationFixResponse * @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.SetShardTabletControlResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationFixResponse; /** - * Verifies a SetShardTabletControlResponse message. + * Verifies a ShardReplicationFixResponse 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 SetShardTabletControlResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationFixResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetShardTabletControlResponse + * @returns ShardReplicationFixResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetShardTabletControlResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationFixResponse; /** - * Creates a plain object from a SetShardTabletControlResponse message. Also converts values to other types if specified. - * @param message SetShardTabletControlResponse + * Creates a plain object from a ShardReplicationFixResponse message. Also converts values to other types if specified. + * @param message ShardReplicationFixResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetShardTabletControlResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationFixResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetShardTabletControlResponse to JSON. + * Converts this ShardReplicationFixResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetShardTabletControlResponse + * Gets the default type url for ShardReplicationFixResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetWritableRequest. */ - interface ISetWritableRequest { + /** Properties of a ShardReplicationPositionsRequest. */ + interface IShardReplicationPositionsRequest { - /** SetWritableRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** ShardReplicationPositionsRequest keyspace */ + keyspace?: (string|null); - /** SetWritableRequest writable */ - writable?: (boolean|null); + /** ShardReplicationPositionsRequest shard */ + shard?: (string|null); } - /** Represents a SetWritableRequest. */ - class SetWritableRequest implements ISetWritableRequest { + /** Represents a ShardReplicationPositionsRequest. */ + class ShardReplicationPositionsRequest implements IShardReplicationPositionsRequest { /** - * Constructs a new SetWritableRequest. + * Constructs a new ShardReplicationPositionsRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetWritableRequest); + constructor(properties?: vtctldata.IShardReplicationPositionsRequest); - /** SetWritableRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** ShardReplicationPositionsRequest keyspace. */ + public keyspace: string; - /** SetWritableRequest writable. */ - public writable: boolean; + /** ShardReplicationPositionsRequest shard. */ + public shard: string; /** - * Creates a new SetWritableRequest instance using the specified properties. + * Creates a new ShardReplicationPositionsRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SetWritableRequest instance + * @returns ShardReplicationPositionsRequest instance */ - public static create(properties?: vtctldata.ISetWritableRequest): vtctldata.SetWritableRequest; + public static create(properties?: vtctldata.IShardReplicationPositionsRequest): vtctldata.ShardReplicationPositionsRequest; /** - * Encodes the specified SetWritableRequest message. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. - * @param message SetWritableRequest message or plain object to encode + * Encodes the specified ShardReplicationPositionsRequest message. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. + * @param message ShardReplicationPositionsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetWritableRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationPositionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetWritableRequest message, length delimited. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. - * @param message SetWritableRequest message or plain object to encode + * Encodes the specified ShardReplicationPositionsRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. + * @param message ShardReplicationPositionsRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetWritableRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationPositionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetWritableRequest message from the specified reader or buffer. + * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetWritableRequest + * @returns ShardReplicationPositionsRequest * @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.SetWritableRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationPositionsRequest; /** - * Decodes a SetWritableRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetWritableRequest + * @returns ShardReplicationPositionsRequest * @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.SetWritableRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationPositionsRequest; /** - * Verifies a SetWritableRequest message. + * Verifies a ShardReplicationPositionsRequest 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 SetWritableRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationPositionsRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetWritableRequest + * @returns ShardReplicationPositionsRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetWritableRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationPositionsRequest; /** - * Creates a plain object from a SetWritableRequest message. Also converts values to other types if specified. - * @param message SetWritableRequest + * Creates a plain object from a ShardReplicationPositionsRequest message. Also converts values to other types if specified. + * @param message ShardReplicationPositionsRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetWritableRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationPositionsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetWritableRequest to JSON. + * Converts this ShardReplicationPositionsRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetWritableRequest + * Gets the default type url for ShardReplicationPositionsRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SetWritableResponse. */ - interface ISetWritableResponse { + /** Properties of a ShardReplicationPositionsResponse. */ + interface IShardReplicationPositionsResponse { + + /** ShardReplicationPositionsResponse replication_statuses */ + replication_statuses?: ({ [k: string]: replicationdata.IStatus }|null); + + /** ShardReplicationPositionsResponse tablet_map */ + tablet_map?: ({ [k: string]: topodata.ITablet }|null); } - /** Represents a SetWritableResponse. */ - class SetWritableResponse implements ISetWritableResponse { + /** Represents a ShardReplicationPositionsResponse. */ + class ShardReplicationPositionsResponse implements IShardReplicationPositionsResponse { /** - * Constructs a new SetWritableResponse. + * Constructs a new ShardReplicationPositionsResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISetWritableResponse); + constructor(properties?: vtctldata.IShardReplicationPositionsResponse); + + /** ShardReplicationPositionsResponse replication_statuses. */ + public replication_statuses: { [k: string]: replicationdata.IStatus }; + + /** ShardReplicationPositionsResponse tablet_map. */ + public tablet_map: { [k: string]: topodata.ITablet }; /** - * Creates a new SetWritableResponse instance using the specified properties. + * Creates a new ShardReplicationPositionsResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SetWritableResponse instance + * @returns ShardReplicationPositionsResponse instance */ - public static create(properties?: vtctldata.ISetWritableResponse): vtctldata.SetWritableResponse; + public static create(properties?: vtctldata.IShardReplicationPositionsResponse): vtctldata.ShardReplicationPositionsResponse; /** - * Encodes the specified SetWritableResponse message. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. - * @param message SetWritableResponse message or plain object to encode + * Encodes the specified ShardReplicationPositionsResponse message. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. + * @param message ShardReplicationPositionsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISetWritableResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationPositionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SetWritableResponse message, length delimited. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. - * @param message SetWritableResponse message or plain object to encode + * Encodes the specified ShardReplicationPositionsResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. + * @param message ShardReplicationPositionsResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISetWritableResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationPositionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SetWritableResponse message from the specified reader or buffer. + * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SetWritableResponse + * @returns ShardReplicationPositionsResponse * @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.SetWritableResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationPositionsResponse; /** - * Decodes a SetWritableResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SetWritableResponse + * @returns ShardReplicationPositionsResponse * @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.SetWritableResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationPositionsResponse; /** - * Verifies a SetWritableResponse message. + * Verifies a ShardReplicationPositionsResponse 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 SetWritableResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationPositionsResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SetWritableResponse + * @returns ShardReplicationPositionsResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SetWritableResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationPositionsResponse; /** - * Creates a plain object from a SetWritableResponse message. Also converts values to other types if specified. - * @param message SetWritableResponse + * Creates a plain object from a ShardReplicationPositionsResponse message. Also converts values to other types if specified. + * @param message ShardReplicationPositionsResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SetWritableResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationPositionsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SetWritableResponse to JSON. + * Converts this ShardReplicationPositionsResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SetWritableResponse + * Gets the default type url for ShardReplicationPositionsResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationAddRequest. */ - interface IShardReplicationAddRequest { + /** Properties of a ShardReplicationRemoveRequest. */ + interface IShardReplicationRemoveRequest { - /** ShardReplicationAddRequest keyspace */ + /** ShardReplicationRemoveRequest keyspace */ keyspace?: (string|null); - /** ShardReplicationAddRequest shard */ + /** ShardReplicationRemoveRequest shard */ shard?: (string|null); - /** ShardReplicationAddRequest tablet_alias */ + /** ShardReplicationRemoveRequest tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a ShardReplicationAddRequest. */ - class ShardReplicationAddRequest implements IShardReplicationAddRequest { + /** Represents a ShardReplicationRemoveRequest. */ + class ShardReplicationRemoveRequest implements IShardReplicationRemoveRequest { /** - * Constructs a new ShardReplicationAddRequest. + * Constructs a new ShardReplicationRemoveRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationAddRequest); + constructor(properties?: vtctldata.IShardReplicationRemoveRequest); - /** ShardReplicationAddRequest keyspace. */ + /** ShardReplicationRemoveRequest keyspace. */ public keyspace: string; - /** ShardReplicationAddRequest shard. */ + /** ShardReplicationRemoveRequest shard. */ public shard: string; - /** ShardReplicationAddRequest tablet_alias. */ + /** ShardReplicationRemoveRequest tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new ShardReplicationAddRequest instance using the specified properties. + * Creates a new ShardReplicationRemoveRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationAddRequest instance + * @returns ShardReplicationRemoveRequest instance */ - public static create(properties?: vtctldata.IShardReplicationAddRequest): vtctldata.ShardReplicationAddRequest; + public static create(properties?: vtctldata.IShardReplicationRemoveRequest): vtctldata.ShardReplicationRemoveRequest; /** - * Encodes the specified ShardReplicationAddRequest message. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. - * @param message ShardReplicationAddRequest message or plain object to encode + * Encodes the specified ShardReplicationRemoveRequest message. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. + * @param message ShardReplicationRemoveRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationRemoveRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationAddRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. - * @param message ShardReplicationAddRequest message or plain object to encode + * Encodes the specified ShardReplicationRemoveRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. + * @param message ShardReplicationRemoveRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationRemoveRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationAddRequest message from the specified reader or buffer. + * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationAddRequest + * @returns ShardReplicationRemoveRequest * @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.ShardReplicationAddRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationRemoveRequest; /** - * Decodes a ShardReplicationAddRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationAddRequest + * @returns ShardReplicationRemoveRequest * @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.ShardReplicationAddRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationRemoveRequest; /** - * Verifies a ShardReplicationAddRequest message. + * Verifies a ShardReplicationRemoveRequest 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 ShardReplicationAddRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationRemoveRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationAddRequest + * @returns ShardReplicationRemoveRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationAddRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationRemoveRequest; /** - * Creates a plain object from a ShardReplicationAddRequest message. Also converts values to other types if specified. - * @param message ShardReplicationAddRequest + * Creates a plain object from a ShardReplicationRemoveRequest message. Also converts values to other types if specified. + * @param message ShardReplicationRemoveRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationAddRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationRemoveRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationAddRequest to JSON. + * Converts this ShardReplicationRemoveRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationAddRequest + * Gets the default type url for ShardReplicationRemoveRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationAddResponse. */ - interface IShardReplicationAddResponse { + /** Properties of a ShardReplicationRemoveResponse. */ + interface IShardReplicationRemoveResponse { } - /** Represents a ShardReplicationAddResponse. */ - class ShardReplicationAddResponse implements IShardReplicationAddResponse { + /** Represents a ShardReplicationRemoveResponse. */ + class ShardReplicationRemoveResponse implements IShardReplicationRemoveResponse { /** - * Constructs a new ShardReplicationAddResponse. + * Constructs a new ShardReplicationRemoveResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationAddResponse); + constructor(properties?: vtctldata.IShardReplicationRemoveResponse); /** - * Creates a new ShardReplicationAddResponse instance using the specified properties. + * Creates a new ShardReplicationRemoveResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationAddResponse instance + * @returns ShardReplicationRemoveResponse instance */ - public static create(properties?: vtctldata.IShardReplicationAddResponse): vtctldata.ShardReplicationAddResponse; + public static create(properties?: vtctldata.IShardReplicationRemoveResponse): vtctldata.ShardReplicationRemoveResponse; /** - * Encodes the specified ShardReplicationAddResponse message. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. - * @param message ShardReplicationAddResponse message or plain object to encode + * Encodes the specified ShardReplicationRemoveResponse message. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. + * @param message ShardReplicationRemoveResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IShardReplicationRemoveResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationAddResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. - * @param message ShardReplicationAddResponse message or plain object to encode + * Encodes the specified ShardReplicationRemoveResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. + * @param message ShardReplicationRemoveResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IShardReplicationRemoveResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationAddResponse message from the specified reader or buffer. + * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationAddResponse + * @returns ShardReplicationRemoveResponse * @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.ShardReplicationAddResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ShardReplicationRemoveResponse; /** - * Decodes a ShardReplicationAddResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationAddResponse + * @returns ShardReplicationRemoveResponse * @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.ShardReplicationAddResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ShardReplicationRemoveResponse; /** - * Verifies a ShardReplicationAddResponse message. + * Verifies a ShardReplicationRemoveResponse 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 ShardReplicationAddResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationRemoveResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationAddResponse + * @returns ShardReplicationRemoveResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationAddResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationRemoveResponse; /** - * Creates a plain object from a ShardReplicationAddResponse message. Also converts values to other types if specified. - * @param message ShardReplicationAddResponse + * Creates a plain object from a ShardReplicationRemoveResponse message. Also converts values to other types if specified. + * @param message ShardReplicationRemoveResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationAddResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ShardReplicationRemoveResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationAddResponse to JSON. + * Converts this ShardReplicationRemoveResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationAddResponse + * Gets the default type url for ShardReplicationRemoveResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationFixRequest. */ - interface IShardReplicationFixRequest { - - /** ShardReplicationFixRequest keyspace */ - keyspace?: (string|null); + /** Properties of a SleepTabletRequest. */ + interface ISleepTabletRequest { - /** ShardReplicationFixRequest shard */ - shard?: (string|null); + /** SleepTabletRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); - /** ShardReplicationFixRequest cell */ - cell?: (string|null); + /** SleepTabletRequest duration */ + duration?: (vttime.IDuration|null); } - /** Represents a ShardReplicationFixRequest. */ - class ShardReplicationFixRequest implements IShardReplicationFixRequest { + /** Represents a SleepTabletRequest. */ + class SleepTabletRequest implements ISleepTabletRequest { /** - * Constructs a new ShardReplicationFixRequest. + * Constructs a new SleepTabletRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationFixRequest); - - /** ShardReplicationFixRequest keyspace. */ - public keyspace: string; + constructor(properties?: vtctldata.ISleepTabletRequest); - /** ShardReplicationFixRequest shard. */ - public shard: string; + /** SleepTabletRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); - /** ShardReplicationFixRequest cell. */ - public cell: string; + /** SleepTabletRequest duration. */ + public duration?: (vttime.IDuration|null); /** - * Creates a new ShardReplicationFixRequest instance using the specified properties. + * Creates a new SleepTabletRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationFixRequest instance + * @returns SleepTabletRequest instance */ - public static create(properties?: vtctldata.IShardReplicationFixRequest): vtctldata.ShardReplicationFixRequest; + public static create(properties?: vtctldata.ISleepTabletRequest): vtctldata.SleepTabletRequest; /** - * Encodes the specified ShardReplicationFixRequest message. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. - * @param message ShardReplicationFixRequest message or plain object to encode + * Encodes the specified SleepTabletRequest message. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. + * @param message SleepTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationFixRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISleepTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationFixRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. - * @param message ShardReplicationFixRequest message or plain object to encode + * Encodes the specified SleepTabletRequest message, length delimited. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. + * @param message SleepTabletRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationFixRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISleepTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationFixRequest message from the specified reader or buffer. + * Decodes a SleepTabletRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationFixRequest + * @returns SleepTabletRequest * @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.ShardReplicationFixRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SleepTabletRequest; /** - * Decodes a ShardReplicationFixRequest message from the specified reader or buffer, length delimited. + * Decodes a SleepTabletRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationFixRequest + * @returns SleepTabletRequest * @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.ShardReplicationFixRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SleepTabletRequest; /** - * Verifies a ShardReplicationFixRequest message. + * Verifies a SleepTabletRequest 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 ShardReplicationFixRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SleepTabletRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationFixRequest + * @returns SleepTabletRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationFixRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SleepTabletRequest; /** - * Creates a plain object from a ShardReplicationFixRequest message. Also converts values to other types if specified. - * @param message ShardReplicationFixRequest + * Creates a plain object from a SleepTabletRequest message. Also converts values to other types if specified. + * @param message SleepTabletRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationFixRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SleepTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationFixRequest to JSON. + * Converts this SleepTabletRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationFixRequest + * Gets the default type url for SleepTabletRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationFixResponse. */ - interface IShardReplicationFixResponse { - - /** ShardReplicationFixResponse error */ - error?: (topodata.IShardReplicationError|null); + /** Properties of a SleepTabletResponse. */ + interface ISleepTabletResponse { } - /** Represents a ShardReplicationFixResponse. */ - class ShardReplicationFixResponse implements IShardReplicationFixResponse { + /** Represents a SleepTabletResponse. */ + class SleepTabletResponse implements ISleepTabletResponse { /** - * Constructs a new ShardReplicationFixResponse. + * Constructs a new SleepTabletResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationFixResponse); - - /** ShardReplicationFixResponse error. */ - public error?: (topodata.IShardReplicationError|null); + constructor(properties?: vtctldata.ISleepTabletResponse); /** - * Creates a new ShardReplicationFixResponse instance using the specified properties. + * Creates a new SleepTabletResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationFixResponse instance + * @returns SleepTabletResponse instance */ - public static create(properties?: vtctldata.IShardReplicationFixResponse): vtctldata.ShardReplicationFixResponse; + public static create(properties?: vtctldata.ISleepTabletResponse): vtctldata.SleepTabletResponse; /** - * Encodes the specified ShardReplicationFixResponse message. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. - * @param message ShardReplicationFixResponse message or plain object to encode + * Encodes the specified SleepTabletResponse message. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. + * @param message SleepTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationFixResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISleepTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationFixResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. - * @param message ShardReplicationFixResponse message or plain object to encode + * Encodes the specified SleepTabletResponse message, length delimited. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. + * @param message SleepTabletResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationFixResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISleepTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationFixResponse message from the specified reader or buffer. + * Decodes a SleepTabletResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationFixResponse + * @returns SleepTabletResponse * @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.ShardReplicationFixResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SleepTabletResponse; /** - * Decodes a ShardReplicationFixResponse message from the specified reader or buffer, length delimited. + * Decodes a SleepTabletResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationFixResponse + * @returns SleepTabletResponse * @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.ShardReplicationFixResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SleepTabletResponse; /** - * Verifies a ShardReplicationFixResponse message. + * Verifies a SleepTabletResponse 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 ShardReplicationFixResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SleepTabletResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationFixResponse + * @returns SleepTabletResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationFixResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SleepTabletResponse; /** - * Creates a plain object from a ShardReplicationFixResponse message. Also converts values to other types if specified. - * @param message ShardReplicationFixResponse + * Creates a plain object from a SleepTabletResponse message. Also converts values to other types if specified. + * @param message SleepTabletResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationFixResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SleepTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationFixResponse to JSON. + * Converts this SleepTabletResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationFixResponse + * Gets the default type url for SleepTabletResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationPositionsRequest. */ - interface IShardReplicationPositionsRequest { + /** Properties of a SourceShardAddRequest. */ + interface ISourceShardAddRequest { - /** ShardReplicationPositionsRequest keyspace */ + /** SourceShardAddRequest keyspace */ keyspace?: (string|null); - /** ShardReplicationPositionsRequest shard */ + /** SourceShardAddRequest shard */ shard?: (string|null); + + /** SourceShardAddRequest uid */ + uid?: (number|null); + + /** SourceShardAddRequest source_keyspace */ + source_keyspace?: (string|null); + + /** SourceShardAddRequest source_shard */ + source_shard?: (string|null); + + /** SourceShardAddRequest key_range */ + key_range?: (topodata.IKeyRange|null); + + /** SourceShardAddRequest tables */ + tables?: (string[]|null); } - /** Represents a ShardReplicationPositionsRequest. */ - class ShardReplicationPositionsRequest implements IShardReplicationPositionsRequest { + /** Represents a SourceShardAddRequest. */ + class SourceShardAddRequest implements ISourceShardAddRequest { /** - * Constructs a new ShardReplicationPositionsRequest. + * Constructs a new SourceShardAddRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationPositionsRequest); + constructor(properties?: vtctldata.ISourceShardAddRequest); - /** ShardReplicationPositionsRequest keyspace. */ + /** SourceShardAddRequest keyspace. */ public keyspace: string; - /** ShardReplicationPositionsRequest shard. */ + /** SourceShardAddRequest shard. */ public shard: string; + /** SourceShardAddRequest uid. */ + public uid: number; + + /** SourceShardAddRequest source_keyspace. */ + public source_keyspace: string; + + /** SourceShardAddRequest source_shard. */ + public source_shard: string; + + /** SourceShardAddRequest key_range. */ + public key_range?: (topodata.IKeyRange|null); + + /** SourceShardAddRequest tables. */ + public tables: string[]; + /** - * Creates a new ShardReplicationPositionsRequest instance using the specified properties. + * Creates a new SourceShardAddRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationPositionsRequest instance + * @returns SourceShardAddRequest instance */ - public static create(properties?: vtctldata.IShardReplicationPositionsRequest): vtctldata.ShardReplicationPositionsRequest; + public static create(properties?: vtctldata.ISourceShardAddRequest): vtctldata.SourceShardAddRequest; /** - * Encodes the specified ShardReplicationPositionsRequest message. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. - * @param message ShardReplicationPositionsRequest message or plain object to encode + * Encodes the specified SourceShardAddRequest message. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. + * @param message SourceShardAddRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationPositionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISourceShardAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationPositionsRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. - * @param message ShardReplicationPositionsRequest message or plain object to encode + * Encodes the specified SourceShardAddRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. + * @param message SourceShardAddRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationPositionsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISourceShardAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer. + * Decodes a SourceShardAddRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationPositionsRequest + * @returns SourceShardAddRequest * @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.ShardReplicationPositionsRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SourceShardAddRequest; /** - * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer, length delimited. + * Decodes a SourceShardAddRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationPositionsRequest + * @returns SourceShardAddRequest * @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.ShardReplicationPositionsRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SourceShardAddRequest; /** - * Verifies a ShardReplicationPositionsRequest message. + * Verifies a SourceShardAddRequest 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 ShardReplicationPositionsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardAddRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationPositionsRequest + * @returns SourceShardAddRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationPositionsRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardAddRequest; /** - * Creates a plain object from a ShardReplicationPositionsRequest message. Also converts values to other types if specified. - * @param message ShardReplicationPositionsRequest + * Creates a plain object from a SourceShardAddRequest message. Also converts values to other types if specified. + * @param message SourceShardAddRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationPositionsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SourceShardAddRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationPositionsRequest to JSON. + * Converts this SourceShardAddRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationPositionsRequest + * Gets the default type url for SourceShardAddRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationPositionsResponse. */ - interface IShardReplicationPositionsResponse { - - /** ShardReplicationPositionsResponse replication_statuses */ - replication_statuses?: ({ [k: string]: replicationdata.IStatus }|null); + /** Properties of a SourceShardAddResponse. */ + interface ISourceShardAddResponse { - /** ShardReplicationPositionsResponse tablet_map */ - tablet_map?: ({ [k: string]: topodata.ITablet }|null); + /** SourceShardAddResponse shard */ + shard?: (topodata.IShard|null); } - /** Represents a ShardReplicationPositionsResponse. */ - class ShardReplicationPositionsResponse implements IShardReplicationPositionsResponse { + /** Represents a SourceShardAddResponse. */ + class SourceShardAddResponse implements ISourceShardAddResponse { /** - * Constructs a new ShardReplicationPositionsResponse. + * Constructs a new SourceShardAddResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationPositionsResponse); - - /** ShardReplicationPositionsResponse replication_statuses. */ - public replication_statuses: { [k: string]: replicationdata.IStatus }; + constructor(properties?: vtctldata.ISourceShardAddResponse); - /** ShardReplicationPositionsResponse tablet_map. */ - public tablet_map: { [k: string]: topodata.ITablet }; + /** SourceShardAddResponse shard. */ + public shard?: (topodata.IShard|null); /** - * Creates a new ShardReplicationPositionsResponse instance using the specified properties. + * Creates a new SourceShardAddResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationPositionsResponse instance + * @returns SourceShardAddResponse instance */ - public static create(properties?: vtctldata.IShardReplicationPositionsResponse): vtctldata.ShardReplicationPositionsResponse; + public static create(properties?: vtctldata.ISourceShardAddResponse): vtctldata.SourceShardAddResponse; /** - * Encodes the specified ShardReplicationPositionsResponse message. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. - * @param message ShardReplicationPositionsResponse message or plain object to encode + * Encodes the specified SourceShardAddResponse message. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. + * @param message SourceShardAddResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationPositionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISourceShardAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationPositionsResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. - * @param message ShardReplicationPositionsResponse message or plain object to encode + * Encodes the specified SourceShardAddResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. + * @param message SourceShardAddResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationPositionsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISourceShardAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer. + * Decodes a SourceShardAddResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationPositionsResponse + * @returns SourceShardAddResponse * @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.ShardReplicationPositionsResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SourceShardAddResponse; /** - * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer, length delimited. + * Decodes a SourceShardAddResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationPositionsResponse + * @returns SourceShardAddResponse * @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.ShardReplicationPositionsResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SourceShardAddResponse; /** - * Verifies a ShardReplicationPositionsResponse message. + * Verifies a SourceShardAddResponse 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 ShardReplicationPositionsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardAddResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationPositionsResponse + * @returns SourceShardAddResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationPositionsResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardAddResponse; /** - * Creates a plain object from a ShardReplicationPositionsResponse message. Also converts values to other types if specified. - * @param message ShardReplicationPositionsResponse + * Creates a plain object from a SourceShardAddResponse message. Also converts values to other types if specified. + * @param message SourceShardAddResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationPositionsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SourceShardAddResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationPositionsResponse to JSON. + * Converts this SourceShardAddResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationPositionsResponse + * Gets the default type url for SourceShardAddResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationRemoveRequest. */ - interface IShardReplicationRemoveRequest { + /** Properties of a SourceShardDeleteRequest. */ + interface ISourceShardDeleteRequest { - /** ShardReplicationRemoveRequest keyspace */ + /** SourceShardDeleteRequest keyspace */ keyspace?: (string|null); - /** ShardReplicationRemoveRequest shard */ + /** SourceShardDeleteRequest shard */ shard?: (string|null); - /** ShardReplicationRemoveRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** SourceShardDeleteRequest uid */ + uid?: (number|null); } - /** Represents a ShardReplicationRemoveRequest. */ - class ShardReplicationRemoveRequest implements IShardReplicationRemoveRequest { + /** Represents a SourceShardDeleteRequest. */ + class SourceShardDeleteRequest implements ISourceShardDeleteRequest { /** - * Constructs a new ShardReplicationRemoveRequest. + * Constructs a new SourceShardDeleteRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationRemoveRequest); + constructor(properties?: vtctldata.ISourceShardDeleteRequest); - /** ShardReplicationRemoveRequest keyspace. */ + /** SourceShardDeleteRequest keyspace. */ public keyspace: string; - /** ShardReplicationRemoveRequest shard. */ + /** SourceShardDeleteRequest shard. */ public shard: string; - /** ShardReplicationRemoveRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** SourceShardDeleteRequest uid. */ + public uid: number; /** - * Creates a new ShardReplicationRemoveRequest instance using the specified properties. + * Creates a new SourceShardDeleteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationRemoveRequest instance + * @returns SourceShardDeleteRequest instance */ - public static create(properties?: vtctldata.IShardReplicationRemoveRequest): vtctldata.ShardReplicationRemoveRequest; + public static create(properties?: vtctldata.ISourceShardDeleteRequest): vtctldata.SourceShardDeleteRequest; /** - * Encodes the specified ShardReplicationRemoveRequest message. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. - * @param message ShardReplicationRemoveRequest message or plain object to encode + * Encodes the specified SourceShardDeleteRequest message. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. + * @param message SourceShardDeleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationRemoveRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISourceShardDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationRemoveRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. - * @param message ShardReplicationRemoveRequest message or plain object to encode + * Encodes the specified SourceShardDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. + * @param message SourceShardDeleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationRemoveRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISourceShardDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer. + * Decodes a SourceShardDeleteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationRemoveRequest + * @returns SourceShardDeleteRequest * @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.ShardReplicationRemoveRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SourceShardDeleteRequest; /** - * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer, length delimited. + * Decodes a SourceShardDeleteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationRemoveRequest + * @returns SourceShardDeleteRequest * @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.ShardReplicationRemoveRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SourceShardDeleteRequest; /** - * Verifies a ShardReplicationRemoveRequest message. + * Verifies a SourceShardDeleteRequest 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 ShardReplicationRemoveRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardDeleteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationRemoveRequest + * @returns SourceShardDeleteRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationRemoveRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardDeleteRequest; /** - * Creates a plain object from a ShardReplicationRemoveRequest message. Also converts values to other types if specified. - * @param message ShardReplicationRemoveRequest + * Creates a plain object from a SourceShardDeleteRequest message. Also converts values to other types if specified. + * @param message SourceShardDeleteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationRemoveRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SourceShardDeleteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationRemoveRequest to JSON. + * Converts this SourceShardDeleteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationRemoveRequest + * Gets the default type url for SourceShardDeleteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ShardReplicationRemoveResponse. */ - interface IShardReplicationRemoveResponse { + /** Properties of a SourceShardDeleteResponse. */ + interface ISourceShardDeleteResponse { + + /** SourceShardDeleteResponse shard */ + shard?: (topodata.IShard|null); } - /** Represents a ShardReplicationRemoveResponse. */ - class ShardReplicationRemoveResponse implements IShardReplicationRemoveResponse { + /** Represents a SourceShardDeleteResponse. */ + class SourceShardDeleteResponse implements ISourceShardDeleteResponse { /** - * Constructs a new ShardReplicationRemoveResponse. + * Constructs a new SourceShardDeleteResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IShardReplicationRemoveResponse); + constructor(properties?: vtctldata.ISourceShardDeleteResponse); + + /** SourceShardDeleteResponse shard. */ + public shard?: (topodata.IShard|null); /** - * Creates a new ShardReplicationRemoveResponse instance using the specified properties. + * Creates a new SourceShardDeleteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ShardReplicationRemoveResponse instance + * @returns SourceShardDeleteResponse instance */ - public static create(properties?: vtctldata.IShardReplicationRemoveResponse): vtctldata.ShardReplicationRemoveResponse; + public static create(properties?: vtctldata.ISourceShardDeleteResponse): vtctldata.SourceShardDeleteResponse; /** - * Encodes the specified ShardReplicationRemoveResponse message. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. - * @param message ShardReplicationRemoveResponse message or plain object to encode + * Encodes the specified SourceShardDeleteResponse message. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. + * @param message SourceShardDeleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IShardReplicationRemoveResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ISourceShardDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ShardReplicationRemoveResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. - * @param message ShardReplicationRemoveResponse message or plain object to encode + * Encodes the specified SourceShardDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. + * @param message SourceShardDeleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IShardReplicationRemoveResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ISourceShardDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer. + * Decodes a SourceShardDeleteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ShardReplicationRemoveResponse + * @returns SourceShardDeleteResponse * @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.ShardReplicationRemoveResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.SourceShardDeleteResponse; /** - * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer, length delimited. + * Decodes a SourceShardDeleteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ShardReplicationRemoveResponse + * @returns SourceShardDeleteResponse * @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.ShardReplicationRemoveResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.SourceShardDeleteResponse; /** - * Verifies a ShardReplicationRemoveResponse message. + * Verifies a SourceShardDeleteResponse 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 ShardReplicationRemoveResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardDeleteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ShardReplicationRemoveResponse + * @returns SourceShardDeleteResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ShardReplicationRemoveResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardDeleteResponse; /** - * Creates a plain object from a ShardReplicationRemoveResponse message. Also converts values to other types if specified. - * @param message ShardReplicationRemoveResponse + * Creates a plain object from a SourceShardDeleteResponse message. Also converts values to other types if specified. + * @param message SourceShardDeleteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ShardReplicationRemoveResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.SourceShardDeleteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ShardReplicationRemoveResponse to JSON. + * Converts this SourceShardDeleteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ShardReplicationRemoveResponse + * Gets the default type url for SourceShardDeleteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SleepTabletRequest. */ - interface ISleepTabletRequest { + /** Properties of a StartReplicationRequest. */ + interface IStartReplicationRequest { - /** SleepTabletRequest tablet_alias */ + /** StartReplicationRequest tablet_alias */ tablet_alias?: (topodata.ITabletAlias|null); - - /** SleepTabletRequest duration */ - duration?: (vttime.IDuration|null); } - /** Represents a SleepTabletRequest. */ - class SleepTabletRequest implements ISleepTabletRequest { + /** Represents a StartReplicationRequest. */ + class StartReplicationRequest implements IStartReplicationRequest { /** - * Constructs a new SleepTabletRequest. + * Constructs a new StartReplicationRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISleepTabletRequest); + constructor(properties?: vtctldata.IStartReplicationRequest); - /** SleepTabletRequest tablet_alias. */ + /** StartReplicationRequest tablet_alias. */ public tablet_alias?: (topodata.ITabletAlias|null); - /** SleepTabletRequest duration. */ - public duration?: (vttime.IDuration|null); - /** - * Creates a new SleepTabletRequest instance using the specified properties. + * Creates a new StartReplicationRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SleepTabletRequest instance + * @returns StartReplicationRequest instance */ - public static create(properties?: vtctldata.ISleepTabletRequest): vtctldata.SleepTabletRequest; + public static create(properties?: vtctldata.IStartReplicationRequest): vtctldata.StartReplicationRequest; /** - * Encodes the specified SleepTabletRequest message. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. - * @param message SleepTabletRequest message or plain object to encode + * Encodes the specified StartReplicationRequest message. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. + * @param message StartReplicationRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISleepTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IStartReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SleepTabletRequest message, length delimited. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. - * @param message SleepTabletRequest message or plain object to encode + * Encodes the specified StartReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. + * @param message StartReplicationRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISleepTabletRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IStartReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SleepTabletRequest message from the specified reader or buffer. + * Decodes a StartReplicationRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SleepTabletRequest + * @returns StartReplicationRequest * @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.SleepTabletRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.StartReplicationRequest; /** - * Decodes a SleepTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a StartReplicationRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SleepTabletRequest + * @returns StartReplicationRequest * @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.SleepTabletRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.StartReplicationRequest; /** - * Verifies a SleepTabletRequest message. + * Verifies a StartReplicationRequest 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 SleepTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StartReplicationRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SleepTabletRequest + * @returns StartReplicationRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SleepTabletRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.StartReplicationRequest; /** - * Creates a plain object from a SleepTabletRequest message. Also converts values to other types if specified. - * @param message SleepTabletRequest + * Creates a plain object from a StartReplicationRequest message. Also converts values to other types if specified. + * @param message StartReplicationRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SleepTabletRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.StartReplicationRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SleepTabletRequest to JSON. + * Converts this StartReplicationRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SleepTabletRequest + * Gets the default type url for StartReplicationRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SleepTabletResponse. */ - interface ISleepTabletResponse { + /** Properties of a StartReplicationResponse. */ + interface IStartReplicationResponse { } - /** Represents a SleepTabletResponse. */ - class SleepTabletResponse implements ISleepTabletResponse { + /** Represents a StartReplicationResponse. */ + class StartReplicationResponse implements IStartReplicationResponse { /** - * Constructs a new SleepTabletResponse. + * Constructs a new StartReplicationResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISleepTabletResponse); + constructor(properties?: vtctldata.IStartReplicationResponse); /** - * Creates a new SleepTabletResponse instance using the specified properties. + * Creates a new StartReplicationResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SleepTabletResponse instance + * @returns StartReplicationResponse instance */ - public static create(properties?: vtctldata.ISleepTabletResponse): vtctldata.SleepTabletResponse; + public static create(properties?: vtctldata.IStartReplicationResponse): vtctldata.StartReplicationResponse; /** - * Encodes the specified SleepTabletResponse message. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. - * @param message SleepTabletResponse message or plain object to encode + * Encodes the specified StartReplicationResponse message. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. + * @param message StartReplicationResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISleepTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IStartReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SleepTabletResponse message, length delimited. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. - * @param message SleepTabletResponse message or plain object to encode + * Encodes the specified StartReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. + * @param message StartReplicationResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISleepTabletResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IStartReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SleepTabletResponse message from the specified reader or buffer. + * Decodes a StartReplicationResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SleepTabletResponse + * @returns StartReplicationResponse * @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.SleepTabletResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.StartReplicationResponse; /** - * Decodes a SleepTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a StartReplicationResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SleepTabletResponse + * @returns StartReplicationResponse * @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.SleepTabletResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.StartReplicationResponse; /** - * Verifies a SleepTabletResponse message. + * Verifies a StartReplicationResponse 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 SleepTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StartReplicationResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SleepTabletResponse + * @returns StartReplicationResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SleepTabletResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.StartReplicationResponse; /** - * Creates a plain object from a SleepTabletResponse message. Also converts values to other types if specified. - * @param message SleepTabletResponse + * Creates a plain object from a StartReplicationResponse message. Also converts values to other types if specified. + * @param message StartReplicationResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SleepTabletResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.StartReplicationResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SleepTabletResponse to JSON. + * Converts this StartReplicationResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SleepTabletResponse + * Gets the default type url for StartReplicationResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SourceShardAddRequest. */ - interface ISourceShardAddRequest { - - /** SourceShardAddRequest keyspace */ - keyspace?: (string|null); - - /** SourceShardAddRequest shard */ - shard?: (string|null); - - /** SourceShardAddRequest uid */ - uid?: (number|null); - - /** SourceShardAddRequest source_keyspace */ - source_keyspace?: (string|null); - - /** SourceShardAddRequest source_shard */ - source_shard?: (string|null); - - /** SourceShardAddRequest key_range */ - key_range?: (topodata.IKeyRange|null); + /** Properties of a StopReplicationRequest. */ + interface IStopReplicationRequest { - /** SourceShardAddRequest tables */ - tables?: (string[]|null); + /** StopReplicationRequest tablet_alias */ + tablet_alias?: (topodata.ITabletAlias|null); } - /** Represents a SourceShardAddRequest. */ - class SourceShardAddRequest implements ISourceShardAddRequest { + /** Represents a StopReplicationRequest. */ + class StopReplicationRequest implements IStopReplicationRequest { /** - * Constructs a new SourceShardAddRequest. + * Constructs a new StopReplicationRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISourceShardAddRequest); - - /** SourceShardAddRequest keyspace. */ - public keyspace: string; - - /** SourceShardAddRequest shard. */ - public shard: string; - - /** SourceShardAddRequest uid. */ - public uid: number; - - /** SourceShardAddRequest source_keyspace. */ - public source_keyspace: string; - - /** SourceShardAddRequest source_shard. */ - public source_shard: string; - - /** SourceShardAddRequest key_range. */ - public key_range?: (topodata.IKeyRange|null); + constructor(properties?: vtctldata.IStopReplicationRequest); - /** SourceShardAddRequest tables. */ - public tables: string[]; + /** StopReplicationRequest tablet_alias. */ + public tablet_alias?: (topodata.ITabletAlias|null); /** - * Creates a new SourceShardAddRequest instance using the specified properties. + * Creates a new StopReplicationRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SourceShardAddRequest instance + * @returns StopReplicationRequest instance */ - public static create(properties?: vtctldata.ISourceShardAddRequest): vtctldata.SourceShardAddRequest; + public static create(properties?: vtctldata.IStopReplicationRequest): vtctldata.StopReplicationRequest; /** - * Encodes the specified SourceShardAddRequest message. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. - * @param message SourceShardAddRequest message or plain object to encode + * Encodes the specified StopReplicationRequest message. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. + * @param message StopReplicationRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISourceShardAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IStopReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SourceShardAddRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. - * @param message SourceShardAddRequest message or plain object to encode + * Encodes the specified StopReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. + * @param message StopReplicationRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISourceShardAddRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IStopReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SourceShardAddRequest message from the specified reader or buffer. + * Decodes a StopReplicationRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SourceShardAddRequest + * @returns StopReplicationRequest * @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.SourceShardAddRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.StopReplicationRequest; /** - * Decodes a SourceShardAddRequest message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SourceShardAddRequest + * @returns StopReplicationRequest * @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.SourceShardAddRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.StopReplicationRequest; /** - * Verifies a SourceShardAddRequest message. + * Verifies a StopReplicationRequest 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 SourceShardAddRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SourceShardAddRequest + * @returns StopReplicationRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardAddRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.StopReplicationRequest; /** - * Creates a plain object from a SourceShardAddRequest message. Also converts values to other types if specified. - * @param message SourceShardAddRequest + * Creates a plain object from a StopReplicationRequest message. Also converts values to other types if specified. + * @param message StopReplicationRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SourceShardAddRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.StopReplicationRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SourceShardAddRequest to JSON. + * Converts this StopReplicationRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SourceShardAddRequest + * Gets the default type url for StopReplicationRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SourceShardAddResponse. */ - interface ISourceShardAddResponse { - - /** SourceShardAddResponse shard */ - shard?: (topodata.IShard|null); + /** Properties of a StopReplicationResponse. */ + interface IStopReplicationResponse { } - /** Represents a SourceShardAddResponse. */ - class SourceShardAddResponse implements ISourceShardAddResponse { + /** Represents a StopReplicationResponse. */ + class StopReplicationResponse implements IStopReplicationResponse { /** - * Constructs a new SourceShardAddResponse. + * Constructs a new StopReplicationResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISourceShardAddResponse); - - /** SourceShardAddResponse shard. */ - public shard?: (topodata.IShard|null); + constructor(properties?: vtctldata.IStopReplicationResponse); /** - * Creates a new SourceShardAddResponse instance using the specified properties. + * Creates a new StopReplicationResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SourceShardAddResponse instance + * @returns StopReplicationResponse instance */ - public static create(properties?: vtctldata.ISourceShardAddResponse): vtctldata.SourceShardAddResponse; + public static create(properties?: vtctldata.IStopReplicationResponse): vtctldata.StopReplicationResponse; /** - * Encodes the specified SourceShardAddResponse message. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. - * @param message SourceShardAddResponse message or plain object to encode + * Encodes the specified StopReplicationResponse message. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. + * @param message StopReplicationResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISourceShardAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IStopReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SourceShardAddResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. - * @param message SourceShardAddResponse message or plain object to encode + * Encodes the specified StopReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. + * @param message StopReplicationResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISourceShardAddResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IStopReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SourceShardAddResponse message from the specified reader or buffer. + * Decodes a StopReplicationResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SourceShardAddResponse + * @returns StopReplicationResponse * @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.SourceShardAddResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.StopReplicationResponse; /** - * Decodes a SourceShardAddResponse message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SourceShardAddResponse + * @returns StopReplicationResponse * @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.SourceShardAddResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.StopReplicationResponse; /** - * Verifies a SourceShardAddResponse message. + * Verifies a StopReplicationResponse 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 SourceShardAddResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SourceShardAddResponse + * @returns StopReplicationResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardAddResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.StopReplicationResponse; /** - * Creates a plain object from a SourceShardAddResponse message. Also converts values to other types if specified. - * @param message SourceShardAddResponse + * Creates a plain object from a StopReplicationResponse message. Also converts values to other types if specified. + * @param message StopReplicationResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SourceShardAddResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.StopReplicationResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SourceShardAddResponse to JSON. + * Converts this StopReplicationResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SourceShardAddResponse + * Gets the default type url for StopReplicationResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SourceShardDeleteRequest. */ - interface ISourceShardDeleteRequest { - - /** SourceShardDeleteRequest keyspace */ - keyspace?: (string|null); - - /** SourceShardDeleteRequest shard */ - shard?: (string|null); + /** Properties of a TabletExternallyReparentedRequest. */ + interface ITabletExternallyReparentedRequest { - /** SourceShardDeleteRequest uid */ - uid?: (number|null); + /** TabletExternallyReparentedRequest tablet */ + tablet?: (topodata.ITabletAlias|null); } - /** Represents a SourceShardDeleteRequest. */ - class SourceShardDeleteRequest implements ISourceShardDeleteRequest { + /** Represents a TabletExternallyReparentedRequest. */ + class TabletExternallyReparentedRequest implements ITabletExternallyReparentedRequest { /** - * Constructs a new SourceShardDeleteRequest. + * Constructs a new TabletExternallyReparentedRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISourceShardDeleteRequest); - - /** SourceShardDeleteRequest keyspace. */ - public keyspace: string; - - /** SourceShardDeleteRequest shard. */ - public shard: string; + constructor(properties?: vtctldata.ITabletExternallyReparentedRequest); - /** SourceShardDeleteRequest uid. */ - public uid: number; + /** TabletExternallyReparentedRequest tablet. */ + public tablet?: (topodata.ITabletAlias|null); /** - * Creates a new SourceShardDeleteRequest instance using the specified properties. + * Creates a new TabletExternallyReparentedRequest instance using the specified properties. * @param [properties] Properties to set - * @returns SourceShardDeleteRequest instance + * @returns TabletExternallyReparentedRequest instance */ - public static create(properties?: vtctldata.ISourceShardDeleteRequest): vtctldata.SourceShardDeleteRequest; + public static create(properties?: vtctldata.ITabletExternallyReparentedRequest): vtctldata.TabletExternallyReparentedRequest; /** - * Encodes the specified SourceShardDeleteRequest message. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. - * @param message SourceShardDeleteRequest message or plain object to encode + * Encodes the specified TabletExternallyReparentedRequest message. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. + * @param message TabletExternallyReparentedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISourceShardDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ITabletExternallyReparentedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SourceShardDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. - * @param message SourceShardDeleteRequest message or plain object to encode + * Encodes the specified TabletExternallyReparentedRequest message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. + * @param message TabletExternallyReparentedRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISourceShardDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ITabletExternallyReparentedRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SourceShardDeleteRequest message from the specified reader or buffer. + * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SourceShardDeleteRequest + * @returns TabletExternallyReparentedRequest * @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.SourceShardDeleteRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.TabletExternallyReparentedRequest; /** - * Decodes a SourceShardDeleteRequest message from the specified reader or buffer, length delimited. + * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SourceShardDeleteRequest + * @returns TabletExternallyReparentedRequest * @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.SourceShardDeleteRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.TabletExternallyReparentedRequest; /** - * Verifies a SourceShardDeleteRequest message. + * Verifies a TabletExternallyReparentedRequest 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 SourceShardDeleteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a TabletExternallyReparentedRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SourceShardDeleteRequest + * @returns TabletExternallyReparentedRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardDeleteRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.TabletExternallyReparentedRequest; /** - * Creates a plain object from a SourceShardDeleteRequest message. Also converts values to other types if specified. - * @param message SourceShardDeleteRequest + * Creates a plain object from a TabletExternallyReparentedRequest message. Also converts values to other types if specified. + * @param message TabletExternallyReparentedRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SourceShardDeleteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.TabletExternallyReparentedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SourceShardDeleteRequest to JSON. + * Converts this TabletExternallyReparentedRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SourceShardDeleteRequest + * Gets the default type url for TabletExternallyReparentedRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a SourceShardDeleteResponse. */ - interface ISourceShardDeleteResponse { + /** Properties of a TabletExternallyReparentedResponse. */ + interface ITabletExternallyReparentedResponse { - /** SourceShardDeleteResponse shard */ - shard?: (topodata.IShard|null); + /** TabletExternallyReparentedResponse keyspace */ + keyspace?: (string|null); + + /** TabletExternallyReparentedResponse shard */ + shard?: (string|null); + + /** TabletExternallyReparentedResponse new_primary */ + new_primary?: (topodata.ITabletAlias|null); + + /** TabletExternallyReparentedResponse old_primary */ + old_primary?: (topodata.ITabletAlias|null); } - /** Represents a SourceShardDeleteResponse. */ - class SourceShardDeleteResponse implements ISourceShardDeleteResponse { + /** Represents a TabletExternallyReparentedResponse. */ + class TabletExternallyReparentedResponse implements ITabletExternallyReparentedResponse { /** - * Constructs a new SourceShardDeleteResponse. + * Constructs a new TabletExternallyReparentedResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ISourceShardDeleteResponse); + constructor(properties?: vtctldata.ITabletExternallyReparentedResponse); - /** SourceShardDeleteResponse shard. */ - public shard?: (topodata.IShard|null); + /** TabletExternallyReparentedResponse keyspace. */ + public keyspace: string; + + /** TabletExternallyReparentedResponse shard. */ + public shard: string; + + /** TabletExternallyReparentedResponse new_primary. */ + public new_primary?: (topodata.ITabletAlias|null); + + /** TabletExternallyReparentedResponse old_primary. */ + public old_primary?: (topodata.ITabletAlias|null); /** - * Creates a new SourceShardDeleteResponse instance using the specified properties. + * Creates a new TabletExternallyReparentedResponse instance using the specified properties. * @param [properties] Properties to set - * @returns SourceShardDeleteResponse instance + * @returns TabletExternallyReparentedResponse instance */ - public static create(properties?: vtctldata.ISourceShardDeleteResponse): vtctldata.SourceShardDeleteResponse; + public static create(properties?: vtctldata.ITabletExternallyReparentedResponse): vtctldata.TabletExternallyReparentedResponse; /** - * Encodes the specified SourceShardDeleteResponse message. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. - * @param message SourceShardDeleteResponse message or plain object to encode + * Encodes the specified TabletExternallyReparentedResponse message. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. + * @param message TabletExternallyReparentedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ISourceShardDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.ITabletExternallyReparentedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified SourceShardDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. - * @param message SourceShardDeleteResponse message or plain object to encode + * Encodes the specified TabletExternallyReparentedResponse message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. + * @param message TabletExternallyReparentedResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ISourceShardDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.ITabletExternallyReparentedResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a SourceShardDeleteResponse message from the specified reader or buffer. + * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns SourceShardDeleteResponse + * @returns TabletExternallyReparentedResponse * @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.SourceShardDeleteResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.TabletExternallyReparentedResponse; /** - * Decodes a SourceShardDeleteResponse message from the specified reader or buffer, length delimited. + * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns SourceShardDeleteResponse + * @returns TabletExternallyReparentedResponse * @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.SourceShardDeleteResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.TabletExternallyReparentedResponse; /** - * Verifies a SourceShardDeleteResponse message. + * Verifies a TabletExternallyReparentedResponse 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 SourceShardDeleteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a TabletExternallyReparentedResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns SourceShardDeleteResponse + * @returns TabletExternallyReparentedResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.SourceShardDeleteResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.TabletExternallyReparentedResponse; /** - * Creates a plain object from a SourceShardDeleteResponse message. Also converts values to other types if specified. - * @param message SourceShardDeleteResponse + * Creates a plain object from a TabletExternallyReparentedResponse message. Also converts values to other types if specified. + * @param message TabletExternallyReparentedResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.SourceShardDeleteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.TabletExternallyReparentedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this SourceShardDeleteResponse to JSON. + * Converts this TabletExternallyReparentedResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for SourceShardDeleteResponse + * Gets the default type url for TabletExternallyReparentedResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StartReplicationRequest. */ - interface IStartReplicationRequest { + /** Properties of an UpdateCellInfoRequest. */ + interface IUpdateCellInfoRequest { - /** StartReplicationRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateCellInfoRequest name */ + name?: (string|null); + + /** UpdateCellInfoRequest cell_info */ + cell_info?: (topodata.ICellInfo|null); } - /** Represents a StartReplicationRequest. */ - class StartReplicationRequest implements IStartReplicationRequest { + /** Represents an UpdateCellInfoRequest. */ + class UpdateCellInfoRequest implements IUpdateCellInfoRequest { /** - * Constructs a new StartReplicationRequest. + * Constructs a new UpdateCellInfoRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IStartReplicationRequest); + constructor(properties?: vtctldata.IUpdateCellInfoRequest); - /** StartReplicationRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateCellInfoRequest name. */ + public name: string; + + /** UpdateCellInfoRequest cell_info. */ + public cell_info?: (topodata.ICellInfo|null); /** - * Creates a new StartReplicationRequest instance using the specified properties. + * Creates a new UpdateCellInfoRequest instance using the specified properties. * @param [properties] Properties to set - * @returns StartReplicationRequest instance + * @returns UpdateCellInfoRequest instance */ - public static create(properties?: vtctldata.IStartReplicationRequest): vtctldata.StartReplicationRequest; + public static create(properties?: vtctldata.IUpdateCellInfoRequest): vtctldata.UpdateCellInfoRequest; /** - * Encodes the specified StartReplicationRequest message. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. - * @param message StartReplicationRequest message or plain object to encode + * Encodes the specified UpdateCellInfoRequest message. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. + * @param message UpdateCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IStartReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StartReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. - * @param message StartReplicationRequest message or plain object to encode + * Encodes the specified UpdateCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. + * @param message UpdateCellInfoRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IStartReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StartReplicationRequest message from the specified reader or buffer. + * Decodes an UpdateCellInfoRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StartReplicationRequest + * @returns UpdateCellInfoRequest * @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.StartReplicationRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateCellInfoRequest; /** - * Decodes a StartReplicationRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellInfoRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StartReplicationRequest + * @returns UpdateCellInfoRequest * @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.StartReplicationRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateCellInfoRequest; /** - * Verifies a StartReplicationRequest message. + * Verifies an UpdateCellInfoRequest 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 StartReplicationRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StartReplicationRequest + * @returns UpdateCellInfoRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.StartReplicationRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellInfoRequest; /** - * Creates a plain object from a StartReplicationRequest message. Also converts values to other types if specified. - * @param message StartReplicationRequest + * Creates a plain object from an UpdateCellInfoRequest message. Also converts values to other types if specified. + * @param message UpdateCellInfoRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.StartReplicationRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StartReplicationRequest to JSON. + * Converts this UpdateCellInfoRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StartReplicationRequest + * Gets the default type url for UpdateCellInfoRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StartReplicationResponse. */ - interface IStartReplicationResponse { + /** Properties of an UpdateCellInfoResponse. */ + interface IUpdateCellInfoResponse { + + /** UpdateCellInfoResponse name */ + name?: (string|null); + + /** UpdateCellInfoResponse cell_info */ + cell_info?: (topodata.ICellInfo|null); } - /** Represents a StartReplicationResponse. */ - class StartReplicationResponse implements IStartReplicationResponse { + /** Represents an UpdateCellInfoResponse. */ + class UpdateCellInfoResponse implements IUpdateCellInfoResponse { /** - * Constructs a new StartReplicationResponse. + * Constructs a new UpdateCellInfoResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IStartReplicationResponse); + constructor(properties?: vtctldata.IUpdateCellInfoResponse); + + /** UpdateCellInfoResponse name. */ + public name: string; + + /** UpdateCellInfoResponse cell_info. */ + public cell_info?: (topodata.ICellInfo|null); /** - * Creates a new StartReplicationResponse instance using the specified properties. + * Creates a new UpdateCellInfoResponse instance using the specified properties. * @param [properties] Properties to set - * @returns StartReplicationResponse instance + * @returns UpdateCellInfoResponse instance */ - public static create(properties?: vtctldata.IStartReplicationResponse): vtctldata.StartReplicationResponse; + public static create(properties?: vtctldata.IUpdateCellInfoResponse): vtctldata.UpdateCellInfoResponse; /** - * Encodes the specified StartReplicationResponse message. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. - * @param message StartReplicationResponse message or plain object to encode + * Encodes the specified UpdateCellInfoResponse message. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. + * @param message UpdateCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IStartReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StartReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. - * @param message StartReplicationResponse message or plain object to encode + * Encodes the specified UpdateCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. + * @param message UpdateCellInfoResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IStartReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StartReplicationResponse message from the specified reader or buffer. + * Decodes an UpdateCellInfoResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StartReplicationResponse + * @returns UpdateCellInfoResponse * @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.StartReplicationResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateCellInfoResponse; /** - * Decodes a StartReplicationResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellInfoResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StartReplicationResponse + * @returns UpdateCellInfoResponse * @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.StartReplicationResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateCellInfoResponse; /** - * Verifies a StartReplicationResponse message. + * Verifies an UpdateCellInfoResponse 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 StartReplicationResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StartReplicationResponse + * @returns UpdateCellInfoResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.StartReplicationResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellInfoResponse; /** - * Creates a plain object from a StartReplicationResponse message. Also converts values to other types if specified. - * @param message StartReplicationResponse + * Creates a plain object from an UpdateCellInfoResponse message. Also converts values to other types if specified. + * @param message UpdateCellInfoResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.StartReplicationResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StartReplicationResponse to JSON. + * Converts this UpdateCellInfoResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StartReplicationResponse + * Gets the default type url for UpdateCellInfoResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StopReplicationRequest. */ - interface IStopReplicationRequest { + /** Properties of an UpdateCellsAliasRequest. */ + interface IUpdateCellsAliasRequest { - /** StopReplicationRequest tablet_alias */ - tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateCellsAliasRequest name */ + name?: (string|null); + + /** UpdateCellsAliasRequest cells_alias */ + cells_alias?: (topodata.ICellsAlias|null); } - /** Represents a StopReplicationRequest. */ - class StopReplicationRequest implements IStopReplicationRequest { + /** Represents an UpdateCellsAliasRequest. */ + class UpdateCellsAliasRequest implements IUpdateCellsAliasRequest { /** - * Constructs a new StopReplicationRequest. + * Constructs a new UpdateCellsAliasRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IStopReplicationRequest); + constructor(properties?: vtctldata.IUpdateCellsAliasRequest); - /** StopReplicationRequest tablet_alias. */ - public tablet_alias?: (topodata.ITabletAlias|null); + /** UpdateCellsAliasRequest name. */ + public name: string; + + /** UpdateCellsAliasRequest cells_alias. */ + public cells_alias?: (topodata.ICellsAlias|null); /** - * Creates a new StopReplicationRequest instance using the specified properties. + * Creates a new UpdateCellsAliasRequest instance using the specified properties. * @param [properties] Properties to set - * @returns StopReplicationRequest instance + * @returns UpdateCellsAliasRequest instance */ - public static create(properties?: vtctldata.IStopReplicationRequest): vtctldata.StopReplicationRequest; + public static create(properties?: vtctldata.IUpdateCellsAliasRequest): vtctldata.UpdateCellsAliasRequest; /** - * Encodes the specified StopReplicationRequest message. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. - * @param message StopReplicationRequest message or plain object to encode + * Encodes the specified UpdateCellsAliasRequest message. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. + * @param message UpdateCellsAliasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IStopReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StopReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. - * @param message StopReplicationRequest message or plain object to encode + * Encodes the specified UpdateCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. + * @param message UpdateCellsAliasRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IStopReplicationRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StopReplicationRequest message from the specified reader or buffer. + * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StopReplicationRequest + * @returns UpdateCellsAliasRequest * @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.StopReplicationRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateCellsAliasRequest; /** - * Decodes a StopReplicationRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StopReplicationRequest + * @returns UpdateCellsAliasRequest * @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.StopReplicationRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateCellsAliasRequest; /** - * Verifies a StopReplicationRequest message. + * Verifies an UpdateCellsAliasRequest 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 StopReplicationRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellsAliasRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StopReplicationRequest + * @returns UpdateCellsAliasRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.StopReplicationRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellsAliasRequest; /** - * Creates a plain object from a StopReplicationRequest message. Also converts values to other types if specified. - * @param message StopReplicationRequest + * Creates a plain object from an UpdateCellsAliasRequest message. Also converts values to other types if specified. + * @param message UpdateCellsAliasRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.StopReplicationRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StopReplicationRequest to JSON. + * Converts this UpdateCellsAliasRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StopReplicationRequest + * Gets the default type url for UpdateCellsAliasRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a StopReplicationResponse. */ - interface IStopReplicationResponse { + /** Properties of an UpdateCellsAliasResponse. */ + interface IUpdateCellsAliasResponse { + + /** UpdateCellsAliasResponse name */ + name?: (string|null); + + /** UpdateCellsAliasResponse cells_alias */ + cells_alias?: (topodata.ICellsAlias|null); } - /** Represents a StopReplicationResponse. */ - class StopReplicationResponse implements IStopReplicationResponse { + /** Represents an UpdateCellsAliasResponse. */ + class UpdateCellsAliasResponse implements IUpdateCellsAliasResponse { /** - * Constructs a new StopReplicationResponse. + * Constructs a new UpdateCellsAliasResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IStopReplicationResponse); + constructor(properties?: vtctldata.IUpdateCellsAliasResponse); + + /** UpdateCellsAliasResponse name. */ + public name: string; + + /** UpdateCellsAliasResponse cells_alias. */ + public cells_alias?: (topodata.ICellsAlias|null); /** - * Creates a new StopReplicationResponse instance using the specified properties. + * Creates a new UpdateCellsAliasResponse instance using the specified properties. * @param [properties] Properties to set - * @returns StopReplicationResponse instance + * @returns UpdateCellsAliasResponse instance */ - public static create(properties?: vtctldata.IStopReplicationResponse): vtctldata.StopReplicationResponse; + public static create(properties?: vtctldata.IUpdateCellsAliasResponse): vtctldata.UpdateCellsAliasResponse; /** - * Encodes the specified StopReplicationResponse message. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. - * @param message StopReplicationResponse message or plain object to encode + * Encodes the specified UpdateCellsAliasResponse message. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. + * @param message UpdateCellsAliasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IStopReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IUpdateCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified StopReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. - * @param message StopReplicationResponse message or plain object to encode + * Encodes the specified UpdateCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. + * @param message UpdateCellsAliasResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IStopReplicationResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IUpdateCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a StopReplicationResponse message from the specified reader or buffer. + * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns StopReplicationResponse + * @returns UpdateCellsAliasResponse * @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.StopReplicationResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.UpdateCellsAliasResponse; /** - * Decodes a StopReplicationResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns StopReplicationResponse + * @returns UpdateCellsAliasResponse * @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.StopReplicationResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.UpdateCellsAliasResponse; /** - * Verifies a StopReplicationResponse message. + * Verifies an UpdateCellsAliasResponse 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 StopReplicationResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellsAliasResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns StopReplicationResponse + * @returns UpdateCellsAliasResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.StopReplicationResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellsAliasResponse; /** - * Creates a plain object from a StopReplicationResponse message. Also converts values to other types if specified. - * @param message StopReplicationResponse + * Creates a plain object from an UpdateCellsAliasResponse message. Also converts values to other types if specified. + * @param message UpdateCellsAliasResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.StopReplicationResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.UpdateCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this StopReplicationResponse to JSON. + * Converts this UpdateCellsAliasResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for StopReplicationResponse + * Gets the default type url for UpdateCellsAliasResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a TabletExternallyReparentedRequest. */ - interface ITabletExternallyReparentedRequest { + /** Properties of a ValidateRequest. */ + interface IValidateRequest { - /** TabletExternallyReparentedRequest tablet */ - tablet?: (topodata.ITabletAlias|null); + /** ValidateRequest ping_tablets */ + ping_tablets?: (boolean|null); } - /** Represents a TabletExternallyReparentedRequest. */ - class TabletExternallyReparentedRequest implements ITabletExternallyReparentedRequest { + /** Represents a ValidateRequest. */ + class ValidateRequest implements IValidateRequest { /** - * Constructs a new TabletExternallyReparentedRequest. + * Constructs a new ValidateRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ITabletExternallyReparentedRequest); + constructor(properties?: vtctldata.IValidateRequest); - /** TabletExternallyReparentedRequest tablet. */ - public tablet?: (topodata.ITabletAlias|null); + /** ValidateRequest ping_tablets. */ + public ping_tablets: boolean; /** - * Creates a new TabletExternallyReparentedRequest instance using the specified properties. + * Creates a new ValidateRequest instance using the specified properties. * @param [properties] Properties to set - * @returns TabletExternallyReparentedRequest instance + * @returns ValidateRequest instance */ - public static create(properties?: vtctldata.ITabletExternallyReparentedRequest): vtctldata.TabletExternallyReparentedRequest; + public static create(properties?: vtctldata.IValidateRequest): vtctldata.ValidateRequest; /** - * Encodes the specified TabletExternallyReparentedRequest message. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. - * @param message TabletExternallyReparentedRequest message or plain object to encode + * Encodes the specified ValidateRequest message. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. + * @param message ValidateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ITabletExternallyReparentedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TabletExternallyReparentedRequest message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. - * @param message TabletExternallyReparentedRequest message or plain object to encode + * Encodes the specified ValidateRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. + * @param message ValidateRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ITabletExternallyReparentedRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer. + * Decodes a ValidateRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TabletExternallyReparentedRequest + * @returns ValidateRequest * @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.TabletExternallyReparentedRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateRequest; /** - * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TabletExternallyReparentedRequest + * @returns ValidateRequest * @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.TabletExternallyReparentedRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateRequest; /** - * Verifies a TabletExternallyReparentedRequest message. + * Verifies a ValidateRequest 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 TabletExternallyReparentedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TabletExternallyReparentedRequest + * @returns ValidateRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.TabletExternallyReparentedRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateRequest; /** - * Creates a plain object from a TabletExternallyReparentedRequest message. Also converts values to other types if specified. - * @param message TabletExternallyReparentedRequest + * Creates a plain object from a ValidateRequest message. Also converts values to other types if specified. + * @param message ValidateRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.TabletExternallyReparentedRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TabletExternallyReparentedRequest to JSON. + * Converts this ValidateRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TabletExternallyReparentedRequest + * Gets the default type url for ValidateRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a TabletExternallyReparentedResponse. */ - interface ITabletExternallyReparentedResponse { - - /** TabletExternallyReparentedResponse keyspace */ - keyspace?: (string|null); - - /** TabletExternallyReparentedResponse shard */ - shard?: (string|null); + /** Properties of a ValidateResponse. */ + interface IValidateResponse { - /** TabletExternallyReparentedResponse new_primary */ - new_primary?: (topodata.ITabletAlias|null); + /** ValidateResponse results */ + results?: (string[]|null); - /** TabletExternallyReparentedResponse old_primary */ - old_primary?: (topodata.ITabletAlias|null); + /** ValidateResponse results_by_keyspace */ + results_by_keyspace?: ({ [k: string]: vtctldata.IValidateKeyspaceResponse }|null); } - /** Represents a TabletExternallyReparentedResponse. */ - class TabletExternallyReparentedResponse implements ITabletExternallyReparentedResponse { + /** Represents a ValidateResponse. */ + class ValidateResponse implements IValidateResponse { /** - * Constructs a new TabletExternallyReparentedResponse. + * Constructs a new ValidateResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.ITabletExternallyReparentedResponse); - - /** TabletExternallyReparentedResponse keyspace. */ - public keyspace: string; - - /** TabletExternallyReparentedResponse shard. */ - public shard: string; + constructor(properties?: vtctldata.IValidateResponse); - /** TabletExternallyReparentedResponse new_primary. */ - public new_primary?: (topodata.ITabletAlias|null); + /** ValidateResponse results. */ + public results: string[]; - /** TabletExternallyReparentedResponse old_primary. */ - public old_primary?: (topodata.ITabletAlias|null); + /** ValidateResponse results_by_keyspace. */ + public results_by_keyspace: { [k: string]: vtctldata.IValidateKeyspaceResponse }; /** - * Creates a new TabletExternallyReparentedResponse instance using the specified properties. + * Creates a new ValidateResponse instance using the specified properties. * @param [properties] Properties to set - * @returns TabletExternallyReparentedResponse instance + * @returns ValidateResponse instance */ - public static create(properties?: vtctldata.ITabletExternallyReparentedResponse): vtctldata.TabletExternallyReparentedResponse; + public static create(properties?: vtctldata.IValidateResponse): vtctldata.ValidateResponse; /** - * Encodes the specified TabletExternallyReparentedResponse message. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. - * @param message TabletExternallyReparentedResponse message or plain object to encode + * Encodes the specified ValidateResponse message. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. + * @param message ValidateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.ITabletExternallyReparentedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified TabletExternallyReparentedResponse message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. - * @param message TabletExternallyReparentedResponse message or plain object to encode + * Encodes the specified ValidateResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. + * @param message ValidateResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.ITabletExternallyReparentedResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer. + * Decodes a ValidateResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns TabletExternallyReparentedResponse + * @returns ValidateResponse * @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.TabletExternallyReparentedResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateResponse; /** - * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns TabletExternallyReparentedResponse + * @returns ValidateResponse * @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.TabletExternallyReparentedResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateResponse; /** - * Verifies a TabletExternallyReparentedResponse message. + * Verifies a ValidateResponse 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 TabletExternallyReparentedResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns TabletExternallyReparentedResponse + * @returns ValidateResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.TabletExternallyReparentedResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateResponse; /** - * Creates a plain object from a TabletExternallyReparentedResponse message. Also converts values to other types if specified. - * @param message TabletExternallyReparentedResponse + * Creates a plain object from a ValidateResponse message. Also converts values to other types if specified. + * @param message ValidateResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.TabletExternallyReparentedResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this TabletExternallyReparentedResponse to JSON. + * Converts this ValidateResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for TabletExternallyReparentedResponse + * Gets the default type url for ValidateResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateCellInfoRequest. */ - interface IUpdateCellInfoRequest { + /** Properties of a ValidateKeyspaceRequest. */ + interface IValidateKeyspaceRequest { - /** UpdateCellInfoRequest name */ - name?: (string|null); + /** ValidateKeyspaceRequest keyspace */ + keyspace?: (string|null); - /** UpdateCellInfoRequest cell_info */ - cell_info?: (topodata.ICellInfo|null); + /** ValidateKeyspaceRequest ping_tablets */ + ping_tablets?: (boolean|null); } - /** Represents an UpdateCellInfoRequest. */ - class UpdateCellInfoRequest implements IUpdateCellInfoRequest { + /** Represents a ValidateKeyspaceRequest. */ + class ValidateKeyspaceRequest implements IValidateKeyspaceRequest { /** - * Constructs a new UpdateCellInfoRequest. + * Constructs a new ValidateKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateCellInfoRequest); + constructor(properties?: vtctldata.IValidateKeyspaceRequest); - /** UpdateCellInfoRequest name. */ - public name: string; + /** ValidateKeyspaceRequest keyspace. */ + public keyspace: string; - /** UpdateCellInfoRequest cell_info. */ - public cell_info?: (topodata.ICellInfo|null); + /** ValidateKeyspaceRequest ping_tablets. */ + public ping_tablets: boolean; /** - * Creates a new UpdateCellInfoRequest instance using the specified properties. + * Creates a new ValidateKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateCellInfoRequest instance + * @returns ValidateKeyspaceRequest instance */ - public static create(properties?: vtctldata.IUpdateCellInfoRequest): vtctldata.UpdateCellInfoRequest; + public static create(properties?: vtctldata.IValidateKeyspaceRequest): vtctldata.ValidateKeyspaceRequest; /** - * Encodes the specified UpdateCellInfoRequest message. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. - * @param message UpdateCellInfoRequest message or plain object to encode + * Encodes the specified ValidateKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. + * @param message ValidateKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. - * @param message UpdateCellInfoRequest message or plain object to encode + * Encodes the specified ValidateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. + * @param message ValidateKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateCellInfoRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateCellInfoRequest message from the specified reader or buffer. + * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateCellInfoRequest + * @returns ValidateKeyspaceRequest * @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.UpdateCellInfoRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateKeyspaceRequest; /** - * Decodes an UpdateCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateCellInfoRequest + * @returns ValidateKeyspaceRequest * @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.UpdateCellInfoRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateKeyspaceRequest; /** - * Verifies an UpdateCellInfoRequest message. + * Verifies a ValidateKeyspaceRequest 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 UpdateCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateCellInfoRequest + * @returns ValidateKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellInfoRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateKeyspaceRequest; /** - * Creates a plain object from an UpdateCellInfoRequest message. Also converts values to other types if specified. - * @param message UpdateCellInfoRequest + * Creates a plain object from a ValidateKeyspaceRequest message. Also converts values to other types if specified. + * @param message ValidateKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateCellInfoRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateCellInfoRequest to JSON. + * Converts this ValidateKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateCellInfoRequest + * Gets the default type url for ValidateKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateCellInfoResponse. */ - interface IUpdateCellInfoResponse { + /** Properties of a ValidateKeyspaceResponse. */ + interface IValidateKeyspaceResponse { - /** UpdateCellInfoResponse name */ - name?: (string|null); + /** ValidateKeyspaceResponse results */ + results?: (string[]|null); - /** UpdateCellInfoResponse cell_info */ - cell_info?: (topodata.ICellInfo|null); + /** ValidateKeyspaceResponse results_by_shard */ + results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); } - /** Represents an UpdateCellInfoResponse. */ - class UpdateCellInfoResponse implements IUpdateCellInfoResponse { + /** Represents a ValidateKeyspaceResponse. */ + class ValidateKeyspaceResponse implements IValidateKeyspaceResponse { /** - * Constructs a new UpdateCellInfoResponse. + * Constructs a new ValidateKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateCellInfoResponse); + constructor(properties?: vtctldata.IValidateKeyspaceResponse); - /** UpdateCellInfoResponse name. */ - public name: string; + /** ValidateKeyspaceResponse results. */ + public results: string[]; - /** UpdateCellInfoResponse cell_info. */ - public cell_info?: (topodata.ICellInfo|null); + /** ValidateKeyspaceResponse results_by_shard. */ + public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; /** - * Creates a new UpdateCellInfoResponse instance using the specified properties. + * Creates a new ValidateKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateCellInfoResponse instance + * @returns ValidateKeyspaceResponse instance */ - public static create(properties?: vtctldata.IUpdateCellInfoResponse): vtctldata.UpdateCellInfoResponse; + public static create(properties?: vtctldata.IValidateKeyspaceResponse): vtctldata.ValidateKeyspaceResponse; /** - * Encodes the specified UpdateCellInfoResponse message. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. - * @param message UpdateCellInfoResponse message or plain object to encode + * Encodes the specified ValidateKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. + * @param message ValidateKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. - * @param message UpdateCellInfoResponse message or plain object to encode + * Encodes the specified ValidateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. + * @param message ValidateKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateCellInfoResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateCellInfoResponse message from the specified reader or buffer. + * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateCellInfoResponse + * @returns ValidateKeyspaceResponse * @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.UpdateCellInfoResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateKeyspaceResponse; /** - * Decodes an UpdateCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateCellInfoResponse + * @returns ValidateKeyspaceResponse * @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.UpdateCellInfoResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateKeyspaceResponse; /** - * Verifies an UpdateCellInfoResponse message. + * Verifies a ValidateKeyspaceResponse 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 UpdateCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateCellInfoResponse + * @returns ValidateKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellInfoResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateKeyspaceResponse; /** - * Creates a plain object from an UpdateCellInfoResponse message. Also converts values to other types if specified. - * @param message UpdateCellInfoResponse + * Creates a plain object from a ValidateKeyspaceResponse message. Also converts values to other types if specified. + * @param message ValidateKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateCellInfoResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateCellInfoResponse to JSON. + * Converts this ValidateKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateCellInfoResponse + * Gets the default type url for ValidateKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateCellsAliasRequest. */ - interface IUpdateCellsAliasRequest { + /** Properties of a ValidateSchemaKeyspaceRequest. */ + interface IValidateSchemaKeyspaceRequest { - /** UpdateCellsAliasRequest name */ - name?: (string|null); + /** ValidateSchemaKeyspaceRequest keyspace */ + keyspace?: (string|null); - /** UpdateCellsAliasRequest cells_alias */ - cells_alias?: (topodata.ICellsAlias|null); + /** ValidateSchemaKeyspaceRequest exclude_tables */ + exclude_tables?: (string[]|null); + + /** ValidateSchemaKeyspaceRequest include_views */ + include_views?: (boolean|null); + + /** ValidateSchemaKeyspaceRequest skip_no_primary */ + skip_no_primary?: (boolean|null); + + /** ValidateSchemaKeyspaceRequest include_vschema */ + include_vschema?: (boolean|null); } - /** Represents an UpdateCellsAliasRequest. */ - class UpdateCellsAliasRequest implements IUpdateCellsAliasRequest { + /** Represents a ValidateSchemaKeyspaceRequest. */ + class ValidateSchemaKeyspaceRequest implements IValidateSchemaKeyspaceRequest { /** - * Constructs a new UpdateCellsAliasRequest. + * Constructs a new ValidateSchemaKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateCellsAliasRequest); + constructor(properties?: vtctldata.IValidateSchemaKeyspaceRequest); - /** UpdateCellsAliasRequest name. */ - public name: string; + /** ValidateSchemaKeyspaceRequest keyspace. */ + public keyspace: string; - /** UpdateCellsAliasRequest cells_alias. */ - public cells_alias?: (topodata.ICellsAlias|null); + /** ValidateSchemaKeyspaceRequest exclude_tables. */ + public exclude_tables: string[]; + + /** ValidateSchemaKeyspaceRequest include_views. */ + public include_views: boolean; + + /** ValidateSchemaKeyspaceRequest skip_no_primary. */ + public skip_no_primary: boolean; + + /** ValidateSchemaKeyspaceRequest include_vschema. */ + public include_vschema: boolean; /** - * Creates a new UpdateCellsAliasRequest instance using the specified properties. + * Creates a new ValidateSchemaKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateCellsAliasRequest instance + * @returns ValidateSchemaKeyspaceRequest instance */ - public static create(properties?: vtctldata.IUpdateCellsAliasRequest): vtctldata.UpdateCellsAliasRequest; + public static create(properties?: vtctldata.IValidateSchemaKeyspaceRequest): vtctldata.ValidateSchemaKeyspaceRequest; /** - * Encodes the specified UpdateCellsAliasRequest message. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. - * @param message UpdateCellsAliasRequest message or plain object to encode + * Encodes the specified ValidateSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. + * @param message ValidateSchemaKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. - * @param message UpdateCellsAliasRequest message or plain object to encode + * Encodes the specified ValidateSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. + * @param message ValidateSchemaKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateCellsAliasRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer. + * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateCellsAliasRequest + * @returns ValidateSchemaKeyspaceRequest * @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.UpdateCellsAliasRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateSchemaKeyspaceRequest; /** - * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateCellsAliasRequest + * @returns ValidateSchemaKeyspaceRequest * @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.UpdateCellsAliasRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateSchemaKeyspaceRequest; /** - * Verifies an UpdateCellsAliasRequest message. + * Verifies a ValidateSchemaKeyspaceRequest 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 UpdateCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateCellsAliasRequest + * @returns ValidateSchemaKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellsAliasRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateSchemaKeyspaceRequest; /** - * Creates a plain object from an UpdateCellsAliasRequest message. Also converts values to other types if specified. - * @param message UpdateCellsAliasRequest + * Creates a plain object from a ValidateSchemaKeyspaceRequest message. Also converts values to other types if specified. + * @param message ValidateSchemaKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateCellsAliasRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateSchemaKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateCellsAliasRequest to JSON. + * Converts this ValidateSchemaKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateCellsAliasRequest + * Gets the default type url for ValidateSchemaKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of an UpdateCellsAliasResponse. */ - interface IUpdateCellsAliasResponse { + /** Properties of a ValidateSchemaKeyspaceResponse. */ + interface IValidateSchemaKeyspaceResponse { - /** UpdateCellsAliasResponse name */ - name?: (string|null); + /** ValidateSchemaKeyspaceResponse results */ + results?: (string[]|null); - /** UpdateCellsAliasResponse cells_alias */ - cells_alias?: (topodata.ICellsAlias|null); + /** ValidateSchemaKeyspaceResponse results_by_shard */ + results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); } - /** Represents an UpdateCellsAliasResponse. */ - class UpdateCellsAliasResponse implements IUpdateCellsAliasResponse { + /** Represents a ValidateSchemaKeyspaceResponse. */ + class ValidateSchemaKeyspaceResponse implements IValidateSchemaKeyspaceResponse { /** - * Constructs a new UpdateCellsAliasResponse. + * Constructs a new ValidateSchemaKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IUpdateCellsAliasResponse); + constructor(properties?: vtctldata.IValidateSchemaKeyspaceResponse); - /** UpdateCellsAliasResponse name. */ - public name: string; + /** ValidateSchemaKeyspaceResponse results. */ + public results: string[]; - /** UpdateCellsAliasResponse cells_alias. */ - public cells_alias?: (topodata.ICellsAlias|null); + /** ValidateSchemaKeyspaceResponse results_by_shard. */ + public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; /** - * Creates a new UpdateCellsAliasResponse instance using the specified properties. + * Creates a new ValidateSchemaKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns UpdateCellsAliasResponse instance + * @returns ValidateSchemaKeyspaceResponse instance */ - public static create(properties?: vtctldata.IUpdateCellsAliasResponse): vtctldata.UpdateCellsAliasResponse; + public static create(properties?: vtctldata.IValidateSchemaKeyspaceResponse): vtctldata.ValidateSchemaKeyspaceResponse; /** - * Encodes the specified UpdateCellsAliasResponse message. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. - * @param message UpdateCellsAliasResponse message or plain object to encode + * Encodes the specified ValidateSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. + * @param message ValidateSchemaKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IUpdateCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified UpdateCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. - * @param message UpdateCellsAliasResponse message or plain object to encode + * Encodes the specified ValidateSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. + * @param message ValidateSchemaKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IUpdateCellsAliasResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer. + * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns UpdateCellsAliasResponse + * @returns ValidateSchemaKeyspaceResponse * @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.UpdateCellsAliasResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateSchemaKeyspaceResponse; /** - * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns UpdateCellsAliasResponse + * @returns ValidateSchemaKeyspaceResponse * @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.UpdateCellsAliasResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateSchemaKeyspaceResponse; /** - * Verifies an UpdateCellsAliasResponse message. + * Verifies a ValidateSchemaKeyspaceResponse 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 UpdateCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns UpdateCellsAliasResponse + * @returns ValidateSchemaKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.UpdateCellsAliasResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateSchemaKeyspaceResponse; /** - * Creates a plain object from an UpdateCellsAliasResponse message. Also converts values to other types if specified. - * @param message UpdateCellsAliasResponse + * Creates a plain object from a ValidateSchemaKeyspaceResponse message. Also converts values to other types if specified. + * @param message ValidateSchemaKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.UpdateCellsAliasResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateSchemaKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this UpdateCellsAliasResponse to JSON. + * Converts this ValidateSchemaKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for UpdateCellsAliasResponse + * Gets the default type url for ValidateSchemaKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateRequest. */ - interface IValidateRequest { + /** Properties of a ValidateShardRequest. */ + interface IValidateShardRequest { - /** ValidateRequest ping_tablets */ + /** ValidateShardRequest keyspace */ + keyspace?: (string|null); + + /** ValidateShardRequest shard */ + shard?: (string|null); + + /** ValidateShardRequest ping_tablets */ ping_tablets?: (boolean|null); } - /** Represents a ValidateRequest. */ - class ValidateRequest implements IValidateRequest { + /** Represents a ValidateShardRequest. */ + class ValidateShardRequest implements IValidateShardRequest { /** - * Constructs a new ValidateRequest. + * Constructs a new ValidateShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateRequest); + constructor(properties?: vtctldata.IValidateShardRequest); - /** ValidateRequest ping_tablets. */ + /** ValidateShardRequest keyspace. */ + public keyspace: string; + + /** ValidateShardRequest shard. */ + public shard: string; + + /** ValidateShardRequest ping_tablets. */ public ping_tablets: boolean; /** - * Creates a new ValidateRequest instance using the specified properties. + * Creates a new ValidateShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateRequest instance + * @returns ValidateShardRequest instance */ - public static create(properties?: vtctldata.IValidateRequest): vtctldata.ValidateRequest; + public static create(properties?: vtctldata.IValidateShardRequest): vtctldata.ValidateShardRequest; /** - * Encodes the specified ValidateRequest message. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. - * @param message ValidateRequest message or plain object to encode + * Encodes the specified ValidateShardRequest message. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. + * @param message ValidateShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. - * @param message ValidateRequest message or plain object to encode + * Encodes the specified ValidateShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. + * @param message ValidateShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateRequest message from the specified reader or buffer. + * Decodes a ValidateShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateRequest + * @returns ValidateShardRequest * @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.ValidateRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateShardRequest; /** - * Decodes a ValidateRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateRequest + * @returns ValidateShardRequest * @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.ValidateRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateShardRequest; /** - * Verifies a ValidateRequest message. + * Verifies a ValidateShardRequest 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 ValidateRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateRequest + * @returns ValidateShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateShardRequest; /** - * Creates a plain object from a ValidateRequest message. Also converts values to other types if specified. - * @param message ValidateRequest + * Creates a plain object from a ValidateShardRequest message. Also converts values to other types if specified. + * @param message ValidateShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateRequest to JSON. + * Converts this ValidateShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateRequest + * Gets the default type url for ValidateShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateResponse. */ - interface IValidateResponse { + /** Properties of a ValidateShardResponse. */ + interface IValidateShardResponse { - /** ValidateResponse results */ + /** ValidateShardResponse results */ results?: (string[]|null); - - /** ValidateResponse results_by_keyspace */ - results_by_keyspace?: ({ [k: string]: vtctldata.IValidateKeyspaceResponse }|null); } - /** Represents a ValidateResponse. */ - class ValidateResponse implements IValidateResponse { + /** Represents a ValidateShardResponse. */ + class ValidateShardResponse implements IValidateShardResponse { /** - * Constructs a new ValidateResponse. + * Constructs a new ValidateShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateResponse); + constructor(properties?: vtctldata.IValidateShardResponse); - /** ValidateResponse results. */ + /** ValidateShardResponse results. */ public results: string[]; - /** ValidateResponse results_by_keyspace. */ - public results_by_keyspace: { [k: string]: vtctldata.IValidateKeyspaceResponse }; - /** - * Creates a new ValidateResponse instance using the specified properties. + * Creates a new ValidateShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateResponse instance + * @returns ValidateShardResponse instance */ - public static create(properties?: vtctldata.IValidateResponse): vtctldata.ValidateResponse; + public static create(properties?: vtctldata.IValidateShardResponse): vtctldata.ValidateShardResponse; /** - * Encodes the specified ValidateResponse message. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. - * @param message ValidateResponse message or plain object to encode + * Encodes the specified ValidateShardResponse message. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. + * @param message ValidateShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. - * @param message ValidateResponse message or plain object to encode + * Encodes the specified ValidateShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. + * @param message ValidateShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateResponse message from the specified reader or buffer. + * Decodes a ValidateShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateResponse + * @returns ValidateShardResponse * @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.ValidateResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateShardResponse; /** - * Decodes a ValidateResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateResponse + * @returns ValidateShardResponse * @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.ValidateResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateShardResponse; /** - * Verifies a ValidateResponse message. + * Verifies a ValidateShardResponse 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 ValidateResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateResponse + * @returns ValidateShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateShardResponse; /** - * Creates a plain object from a ValidateResponse message. Also converts values to other types if specified. - * @param message ValidateResponse + * Creates a plain object from a ValidateShardResponse message. Also converts values to other types if specified. + * @param message ValidateShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateResponse to JSON. + * Converts this ValidateShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateResponse + * Gets the default type url for ValidateShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateKeyspaceRequest. */ - interface IValidateKeyspaceRequest { + /** Properties of a ValidateVersionKeyspaceRequest. */ + interface IValidateVersionKeyspaceRequest { - /** ValidateKeyspaceRequest keyspace */ + /** ValidateVersionKeyspaceRequest keyspace */ keyspace?: (string|null); - - /** ValidateKeyspaceRequest ping_tablets */ - ping_tablets?: (boolean|null); } - /** Represents a ValidateKeyspaceRequest. */ - class ValidateKeyspaceRequest implements IValidateKeyspaceRequest { + /** Represents a ValidateVersionKeyspaceRequest. */ + class ValidateVersionKeyspaceRequest implements IValidateVersionKeyspaceRequest { /** - * Constructs a new ValidateKeyspaceRequest. + * Constructs a new ValidateVersionKeyspaceRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateKeyspaceRequest); + constructor(properties?: vtctldata.IValidateVersionKeyspaceRequest); - /** ValidateKeyspaceRequest keyspace. */ + /** ValidateVersionKeyspaceRequest keyspace. */ public keyspace: string; - /** ValidateKeyspaceRequest ping_tablets. */ - public ping_tablets: boolean; - /** - * Creates a new ValidateKeyspaceRequest instance using the specified properties. + * Creates a new ValidateVersionKeyspaceRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateKeyspaceRequest instance + * @returns ValidateVersionKeyspaceRequest instance */ - public static create(properties?: vtctldata.IValidateKeyspaceRequest): vtctldata.ValidateKeyspaceRequest; + public static create(properties?: vtctldata.IValidateVersionKeyspaceRequest): vtctldata.ValidateVersionKeyspaceRequest; /** - * Encodes the specified ValidateKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. - * @param message ValidateKeyspaceRequest message or plain object to encode + * Encodes the specified ValidateVersionKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. + * @param message ValidateVersionKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVersionKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. - * @param message ValidateKeyspaceRequest message or plain object to encode + * Encodes the specified ValidateVersionKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. + * @param message ValidateVersionKeyspaceRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVersionKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer. + * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateKeyspaceRequest + * @returns ValidateVersionKeyspaceRequest * @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.ValidateKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVersionKeyspaceRequest; /** - * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateKeyspaceRequest + * @returns ValidateVersionKeyspaceRequest * @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.ValidateKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVersionKeyspaceRequest; /** - * Verifies a ValidateKeyspaceRequest message. + * Verifies a ValidateVersionKeyspaceRequest 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 ValidateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateKeyspaceRequest + * @returns ValidateVersionKeyspaceRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionKeyspaceRequest; /** - * Creates a plain object from a ValidateKeyspaceRequest message. Also converts values to other types if specified. - * @param message ValidateKeyspaceRequest + * Creates a plain object from a ValidateVersionKeyspaceRequest message. Also converts values to other types if specified. + * @param message ValidateVersionKeyspaceRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVersionKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateKeyspaceRequest to JSON. + * Converts this ValidateVersionKeyspaceRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateKeyspaceRequest + * Gets the default type url for ValidateVersionKeyspaceRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateKeyspaceResponse. */ - interface IValidateKeyspaceResponse { + /** Properties of a ValidateVersionKeyspaceResponse. */ + interface IValidateVersionKeyspaceResponse { - /** ValidateKeyspaceResponse results */ + /** ValidateVersionKeyspaceResponse results */ results?: (string[]|null); - /** ValidateKeyspaceResponse results_by_shard */ + /** ValidateVersionKeyspaceResponse results_by_shard */ results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); } - /** Represents a ValidateKeyspaceResponse. */ - class ValidateKeyspaceResponse implements IValidateKeyspaceResponse { + /** Represents a ValidateVersionKeyspaceResponse. */ + class ValidateVersionKeyspaceResponse implements IValidateVersionKeyspaceResponse { /** - * Constructs a new ValidateKeyspaceResponse. + * Constructs a new ValidateVersionKeyspaceResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateKeyspaceResponse); + constructor(properties?: vtctldata.IValidateVersionKeyspaceResponse); - /** ValidateKeyspaceResponse results. */ + /** ValidateVersionKeyspaceResponse results. */ public results: string[]; - /** ValidateKeyspaceResponse results_by_shard. */ + /** ValidateVersionKeyspaceResponse results_by_shard. */ public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; /** - * Creates a new ValidateKeyspaceResponse instance using the specified properties. + * Creates a new ValidateVersionKeyspaceResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateKeyspaceResponse instance + * @returns ValidateVersionKeyspaceResponse instance */ - public static create(properties?: vtctldata.IValidateKeyspaceResponse): vtctldata.ValidateKeyspaceResponse; + public static create(properties?: vtctldata.IValidateVersionKeyspaceResponse): vtctldata.ValidateVersionKeyspaceResponse; /** - * Encodes the specified ValidateKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. - * @param message ValidateKeyspaceResponse message or plain object to encode + * Encodes the specified ValidateVersionKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. + * @param message ValidateVersionKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVersionKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. - * @param message ValidateKeyspaceResponse message or plain object to encode + * Encodes the specified ValidateVersionKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. + * @param message ValidateVersionKeyspaceResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVersionKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer. + * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateKeyspaceResponse + * @returns ValidateVersionKeyspaceResponse * @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.ValidateKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVersionKeyspaceResponse; /** - * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateKeyspaceResponse + * @returns ValidateVersionKeyspaceResponse * @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.ValidateKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVersionKeyspaceResponse; /** - * Verifies a ValidateKeyspaceResponse message. + * Verifies a ValidateVersionKeyspaceResponse 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 ValidateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateKeyspaceResponse + * @returns ValidateVersionKeyspaceResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionKeyspaceResponse; /** - * Creates a plain object from a ValidateKeyspaceResponse message. Also converts values to other types if specified. - * @param message ValidateKeyspaceResponse + * Creates a plain object from a ValidateVersionKeyspaceResponse message. Also converts values to other types if specified. + * @param message ValidateVersionKeyspaceResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVersionKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateKeyspaceResponse to JSON. + * Converts this ValidateVersionKeyspaceResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateKeyspaceResponse + * Gets the default type url for ValidateVersionKeyspaceResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateSchemaKeyspaceRequest. */ - interface IValidateSchemaKeyspaceRequest { + /** Properties of a ValidateVersionShardRequest. */ + interface IValidateVersionShardRequest { - /** ValidateSchemaKeyspaceRequest keyspace */ + /** ValidateVersionShardRequest keyspace */ keyspace?: (string|null); - /** ValidateSchemaKeyspaceRequest exclude_tables */ - exclude_tables?: (string[]|null); - - /** ValidateSchemaKeyspaceRequest include_views */ - include_views?: (boolean|null); - - /** ValidateSchemaKeyspaceRequest skip_no_primary */ - skip_no_primary?: (boolean|null); - - /** ValidateSchemaKeyspaceRequest include_vschema */ - include_vschema?: (boolean|null); + /** ValidateVersionShardRequest shard */ + shard?: (string|null); } - /** Represents a ValidateSchemaKeyspaceRequest. */ - class ValidateSchemaKeyspaceRequest implements IValidateSchemaKeyspaceRequest { + /** Represents a ValidateVersionShardRequest. */ + class ValidateVersionShardRequest implements IValidateVersionShardRequest { /** - * Constructs a new ValidateSchemaKeyspaceRequest. + * Constructs a new ValidateVersionShardRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateSchemaKeyspaceRequest); + constructor(properties?: vtctldata.IValidateVersionShardRequest); - /** ValidateSchemaKeyspaceRequest keyspace. */ + /** ValidateVersionShardRequest keyspace. */ public keyspace: string; - /** ValidateSchemaKeyspaceRequest exclude_tables. */ - public exclude_tables: string[]; - - /** ValidateSchemaKeyspaceRequest include_views. */ - public include_views: boolean; - - /** ValidateSchemaKeyspaceRequest skip_no_primary. */ - public skip_no_primary: boolean; - - /** ValidateSchemaKeyspaceRequest include_vschema. */ - public include_vschema: boolean; + /** ValidateVersionShardRequest shard. */ + public shard: string; /** - * Creates a new ValidateSchemaKeyspaceRequest instance using the specified properties. + * Creates a new ValidateVersionShardRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateSchemaKeyspaceRequest instance + * @returns ValidateVersionShardRequest instance */ - public static create(properties?: vtctldata.IValidateSchemaKeyspaceRequest): vtctldata.ValidateSchemaKeyspaceRequest; + public static create(properties?: vtctldata.IValidateVersionShardRequest): vtctldata.ValidateVersionShardRequest; /** - * Encodes the specified ValidateSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. - * @param message ValidateSchemaKeyspaceRequest message or plain object to encode + * Encodes the specified ValidateVersionShardRequest message. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. + * @param message ValidateVersionShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVersionShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. - * @param message ValidateSchemaKeyspaceRequest message or plain object to encode + * Encodes the specified ValidateVersionShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. + * @param message ValidateVersionShardRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateSchemaKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVersionShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer. + * Decodes a ValidateVersionShardRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateSchemaKeyspaceRequest + * @returns ValidateVersionShardRequest * @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.ValidateSchemaKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVersionShardRequest; /** - * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionShardRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateSchemaKeyspaceRequest + * @returns ValidateVersionShardRequest * @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.ValidateSchemaKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVersionShardRequest; /** - * Verifies a ValidateSchemaKeyspaceRequest message. + * Verifies a ValidateVersionShardRequest 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 ValidateSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionShardRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateSchemaKeyspaceRequest + * @returns ValidateVersionShardRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateSchemaKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionShardRequest; /** - * Creates a plain object from a ValidateSchemaKeyspaceRequest message. Also converts values to other types if specified. - * @param message ValidateSchemaKeyspaceRequest + * Creates a plain object from a ValidateVersionShardRequest message. Also converts values to other types if specified. + * @param message ValidateVersionShardRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateSchemaKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVersionShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateSchemaKeyspaceRequest to JSON. + * Converts this ValidateVersionShardRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateSchemaKeyspaceRequest + * Gets the default type url for ValidateVersionShardRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateSchemaKeyspaceResponse. */ - interface IValidateSchemaKeyspaceResponse { + /** Properties of a ValidateVersionShardResponse. */ + interface IValidateVersionShardResponse { - /** ValidateSchemaKeyspaceResponse results */ + /** ValidateVersionShardResponse results */ results?: (string[]|null); - - /** ValidateSchemaKeyspaceResponse results_by_shard */ - results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); } - /** Represents a ValidateSchemaKeyspaceResponse. */ - class ValidateSchemaKeyspaceResponse implements IValidateSchemaKeyspaceResponse { + /** Represents a ValidateVersionShardResponse. */ + class ValidateVersionShardResponse implements IValidateVersionShardResponse { /** - * Constructs a new ValidateSchemaKeyspaceResponse. + * Constructs a new ValidateVersionShardResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateSchemaKeyspaceResponse); + constructor(properties?: vtctldata.IValidateVersionShardResponse); - /** ValidateSchemaKeyspaceResponse results. */ + /** ValidateVersionShardResponse results. */ public results: string[]; - /** ValidateSchemaKeyspaceResponse results_by_shard. */ - public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; - /** - * Creates a new ValidateSchemaKeyspaceResponse instance using the specified properties. + * Creates a new ValidateVersionShardResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateSchemaKeyspaceResponse instance + * @returns ValidateVersionShardResponse instance */ - public static create(properties?: vtctldata.IValidateSchemaKeyspaceResponse): vtctldata.ValidateSchemaKeyspaceResponse; + public static create(properties?: vtctldata.IValidateVersionShardResponse): vtctldata.ValidateVersionShardResponse; /** - * Encodes the specified ValidateSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. - * @param message ValidateSchemaKeyspaceResponse message or plain object to encode + * Encodes the specified ValidateVersionShardResponse message. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. + * @param message ValidateVersionShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVersionShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. - * @param message ValidateSchemaKeyspaceResponse message or plain object to encode + * Encodes the specified ValidateVersionShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. + * @param message ValidateVersionShardResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateSchemaKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVersionShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer. + * Decodes a ValidateVersionShardResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateSchemaKeyspaceResponse + * @returns ValidateVersionShardResponse * @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.ValidateSchemaKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVersionShardResponse; /** - * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionShardResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateSchemaKeyspaceResponse + * @returns ValidateVersionShardResponse * @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.ValidateSchemaKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVersionShardResponse; /** - * Verifies a ValidateSchemaKeyspaceResponse message. + * Verifies a ValidateVersionShardResponse 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 ValidateSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionShardResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateSchemaKeyspaceResponse + * @returns ValidateVersionShardResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateSchemaKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionShardResponse; /** - * Creates a plain object from a ValidateSchemaKeyspaceResponse message. Also converts values to other types if specified. - * @param message ValidateSchemaKeyspaceResponse + * Creates a plain object from a ValidateVersionShardResponse message. Also converts values to other types if specified. + * @param message ValidateVersionShardResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateSchemaKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVersionShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateSchemaKeyspaceResponse to JSON. + * Converts this ValidateVersionShardResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateSchemaKeyspaceResponse + * Gets the default type url for ValidateVersionShardResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateShardRequest. */ - interface IValidateShardRequest { + /** Properties of a ValidateVSchemaRequest. */ + interface IValidateVSchemaRequest { - /** ValidateShardRequest keyspace */ + /** ValidateVSchemaRequest keyspace */ keyspace?: (string|null); - /** ValidateShardRequest shard */ - shard?: (string|null); + /** ValidateVSchemaRequest shards */ + shards?: (string[]|null); - /** ValidateShardRequest ping_tablets */ - ping_tablets?: (boolean|null); + /** ValidateVSchemaRequest exclude_tables */ + exclude_tables?: (string[]|null); + + /** ValidateVSchemaRequest include_views */ + include_views?: (boolean|null); } - /** Represents a ValidateShardRequest. */ - class ValidateShardRequest implements IValidateShardRequest { + /** Represents a ValidateVSchemaRequest. */ + class ValidateVSchemaRequest implements IValidateVSchemaRequest { /** - * Constructs a new ValidateShardRequest. + * Constructs a new ValidateVSchemaRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateShardRequest); + constructor(properties?: vtctldata.IValidateVSchemaRequest); - /** ValidateShardRequest keyspace. */ + /** ValidateVSchemaRequest keyspace. */ public keyspace: string; - /** ValidateShardRequest shard. */ - public shard: string; + /** ValidateVSchemaRequest shards. */ + public shards: string[]; - /** ValidateShardRequest ping_tablets. */ - public ping_tablets: boolean; + /** ValidateVSchemaRequest exclude_tables. */ + public exclude_tables: string[]; + + /** ValidateVSchemaRequest include_views. */ + public include_views: boolean; /** - * Creates a new ValidateShardRequest instance using the specified properties. + * Creates a new ValidateVSchemaRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateShardRequest instance + * @returns ValidateVSchemaRequest instance */ - public static create(properties?: vtctldata.IValidateShardRequest): vtctldata.ValidateShardRequest; + public static create(properties?: vtctldata.IValidateVSchemaRequest): vtctldata.ValidateVSchemaRequest; /** - * Encodes the specified ValidateShardRequest message. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. - * @param message ValidateShardRequest message or plain object to encode + * Encodes the specified ValidateVSchemaRequest message. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. + * @param message ValidateVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. - * @param message ValidateShardRequest message or plain object to encode + * Encodes the specified ValidateVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. + * @param message ValidateVSchemaRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateShardRequest message from the specified reader or buffer. + * Decodes a ValidateVSchemaRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateShardRequest + * @returns ValidateVSchemaRequest * @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.ValidateShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVSchemaRequest; /** - * Decodes a ValidateShardRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVSchemaRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateShardRequest + * @returns ValidateVSchemaRequest * @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.ValidateShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVSchemaRequest; /** - * Verifies a ValidateShardRequest message. + * Verifies a ValidateVSchemaRequest 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 ValidateShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateShardRequest + * @returns ValidateVSchemaRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVSchemaRequest; /** - * Creates a plain object from a ValidateShardRequest message. Also converts values to other types if specified. - * @param message ValidateShardRequest + * Creates a plain object from a ValidateVSchemaRequest message. Also converts values to other types if specified. + * @param message ValidateVSchemaRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateShardRequest to JSON. + * Converts this ValidateVSchemaRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateShardRequest + * Gets the default type url for ValidateVSchemaRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateShardResponse. */ - interface IValidateShardResponse { + /** Properties of a ValidateVSchemaResponse. */ + interface IValidateVSchemaResponse { - /** ValidateShardResponse results */ + /** ValidateVSchemaResponse results */ results?: (string[]|null); + + /** ValidateVSchemaResponse results_by_shard */ + results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); } - /** Represents a ValidateShardResponse. */ - class ValidateShardResponse implements IValidateShardResponse { + /** Represents a ValidateVSchemaResponse. */ + class ValidateVSchemaResponse implements IValidateVSchemaResponse { /** - * Constructs a new ValidateShardResponse. + * Constructs a new ValidateVSchemaResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateShardResponse); + constructor(properties?: vtctldata.IValidateVSchemaResponse); - /** ValidateShardResponse results. */ + /** ValidateVSchemaResponse results. */ public results: string[]; + /** ValidateVSchemaResponse results_by_shard. */ + public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; + /** - * Creates a new ValidateShardResponse instance using the specified properties. + * Creates a new ValidateVSchemaResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateShardResponse instance + * @returns ValidateVSchemaResponse instance */ - public static create(properties?: vtctldata.IValidateShardResponse): vtctldata.ValidateShardResponse; + public static create(properties?: vtctldata.IValidateVSchemaResponse): vtctldata.ValidateVSchemaResponse; /** - * Encodes the specified ValidateShardResponse message. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. - * @param message ValidateShardResponse message or plain object to encode + * Encodes the specified ValidateVSchemaResponse message. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. + * @param message ValidateVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IValidateVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. - * @param message ValidateShardResponse message or plain object to encode + * Encodes the specified ValidateVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. + * @param message ValidateVSchemaResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IValidateVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateShardResponse message from the specified reader or buffer. + * Decodes a ValidateVSchemaResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateShardResponse + * @returns ValidateVSchemaResponse * @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.ValidateShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.ValidateVSchemaResponse; /** - * Decodes a ValidateShardResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVSchemaResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateShardResponse + * @returns ValidateVSchemaResponse * @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.ValidateShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.ValidateVSchemaResponse; /** - * Verifies a ValidateShardResponse message. + * Verifies a ValidateVSchemaResponse 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 ValidateShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateShardResponse + * @returns ValidateVSchemaResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVSchemaResponse; /** - * Creates a plain object from a ValidateShardResponse message. Also converts values to other types if specified. - * @param message ValidateShardResponse + * Creates a plain object from a ValidateVSchemaResponse message. Also converts values to other types if specified. + * @param message ValidateVSchemaResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.ValidateVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateShardResponse to JSON. + * Converts this ValidateVSchemaResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateShardResponse + * Gets the default type url for ValidateVSchemaResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVersionKeyspaceRequest. */ - interface IValidateVersionKeyspaceRequest { + /** Properties of a WorkflowDeleteRequest. */ + interface IWorkflowDeleteRequest { - /** ValidateVersionKeyspaceRequest keyspace */ + /** WorkflowDeleteRequest keyspace */ keyspace?: (string|null); + + /** WorkflowDeleteRequest workflow */ + workflow?: (string|null); + + /** WorkflowDeleteRequest keep_data */ + keep_data?: (boolean|null); + + /** WorkflowDeleteRequest keep_routing_rules */ + keep_routing_rules?: (boolean|null); } - /** Represents a ValidateVersionKeyspaceRequest. */ - class ValidateVersionKeyspaceRequest implements IValidateVersionKeyspaceRequest { + /** Represents a WorkflowDeleteRequest. */ + class WorkflowDeleteRequest implements IWorkflowDeleteRequest { /** - * Constructs a new ValidateVersionKeyspaceRequest. + * Constructs a new WorkflowDeleteRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVersionKeyspaceRequest); + constructor(properties?: vtctldata.IWorkflowDeleteRequest); - /** ValidateVersionKeyspaceRequest keyspace. */ + /** WorkflowDeleteRequest keyspace. */ public keyspace: string; + /** WorkflowDeleteRequest workflow. */ + public workflow: string; + + /** WorkflowDeleteRequest keep_data. */ + public keep_data: boolean; + + /** WorkflowDeleteRequest keep_routing_rules. */ + public keep_routing_rules: boolean; + /** - * Creates a new ValidateVersionKeyspaceRequest instance using the specified properties. + * Creates a new WorkflowDeleteRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVersionKeyspaceRequest instance + * @returns WorkflowDeleteRequest instance */ - public static create(properties?: vtctldata.IValidateVersionKeyspaceRequest): vtctldata.ValidateVersionKeyspaceRequest; + public static create(properties?: vtctldata.IWorkflowDeleteRequest): vtctldata.WorkflowDeleteRequest; /** - * Encodes the specified ValidateVersionKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. - * @param message ValidateVersionKeyspaceRequest message or plain object to encode + * Encodes the specified WorkflowDeleteRequest message. Does not implicitly {@link vtctldata.WorkflowDeleteRequest.verify|verify} messages. + * @param message WorkflowDeleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVersionKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVersionKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. - * @param message ValidateVersionKeyspaceRequest message or plain object to encode + * Encodes the specified WorkflowDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteRequest.verify|verify} messages. + * @param message WorkflowDeleteRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVersionKeyspaceRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowDeleteRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer. + * Decodes a WorkflowDeleteRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVersionKeyspaceRequest + * @returns WorkflowDeleteRequest * @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.ValidateVersionKeyspaceRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowDeleteRequest; /** - * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowDeleteRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVersionKeyspaceRequest + * @returns WorkflowDeleteRequest * @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.ValidateVersionKeyspaceRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowDeleteRequest; /** - * Verifies a ValidateVersionKeyspaceRequest message. + * Verifies a WorkflowDeleteRequest 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 ValidateVersionKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowDeleteRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVersionKeyspaceRequest + * @returns WorkflowDeleteRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionKeyspaceRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowDeleteRequest; /** - * Creates a plain object from a ValidateVersionKeyspaceRequest message. Also converts values to other types if specified. - * @param message ValidateVersionKeyspaceRequest + * Creates a plain object from a WorkflowDeleteRequest message. Also converts values to other types if specified. + * @param message WorkflowDeleteRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVersionKeyspaceRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowDeleteRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVersionKeyspaceRequest to JSON. + * Converts this WorkflowDeleteRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVersionKeyspaceRequest + * Gets the default type url for WorkflowDeleteRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVersionKeyspaceResponse. */ - interface IValidateVersionKeyspaceResponse { + /** Properties of a WorkflowDeleteResponse. */ + interface IWorkflowDeleteResponse { - /** ValidateVersionKeyspaceResponse results */ - results?: (string[]|null); + /** WorkflowDeleteResponse summary */ + summary?: (string|null); - /** ValidateVersionKeyspaceResponse results_by_shard */ - results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); + /** WorkflowDeleteResponse details */ + details?: (vtctldata.WorkflowDeleteResponse.ITabletInfo[]|null); } - /** Represents a ValidateVersionKeyspaceResponse. */ - class ValidateVersionKeyspaceResponse implements IValidateVersionKeyspaceResponse { + /** Represents a WorkflowDeleteResponse. */ + class WorkflowDeleteResponse implements IWorkflowDeleteResponse { /** - * Constructs a new ValidateVersionKeyspaceResponse. + * Constructs a new WorkflowDeleteResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVersionKeyspaceResponse); + constructor(properties?: vtctldata.IWorkflowDeleteResponse); - /** ValidateVersionKeyspaceResponse results. */ - public results: string[]; + /** WorkflowDeleteResponse summary. */ + public summary: string; - /** ValidateVersionKeyspaceResponse results_by_shard. */ - public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; + /** WorkflowDeleteResponse details. */ + public details: vtctldata.WorkflowDeleteResponse.ITabletInfo[]; /** - * Creates a new ValidateVersionKeyspaceResponse instance using the specified properties. + * Creates a new WorkflowDeleteResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVersionKeyspaceResponse instance + * @returns WorkflowDeleteResponse instance */ - public static create(properties?: vtctldata.IValidateVersionKeyspaceResponse): vtctldata.ValidateVersionKeyspaceResponse; + public static create(properties?: vtctldata.IWorkflowDeleteResponse): vtctldata.WorkflowDeleteResponse; /** - * Encodes the specified ValidateVersionKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. - * @param message ValidateVersionKeyspaceResponse message or plain object to encode + * Encodes the specified WorkflowDeleteResponse message. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.verify|verify} messages. + * @param message WorkflowDeleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVersionKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVersionKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. - * @param message ValidateVersionKeyspaceResponse message or plain object to encode + * Encodes the specified WorkflowDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.verify|verify} messages. + * @param message WorkflowDeleteResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVersionKeyspaceResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowDeleteResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer. + * Decodes a WorkflowDeleteResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVersionKeyspaceResponse + * @returns WorkflowDeleteResponse * @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.ValidateVersionKeyspaceResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowDeleteResponse; /** - * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowDeleteResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVersionKeyspaceResponse + * @returns WorkflowDeleteResponse * @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.ValidateVersionKeyspaceResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowDeleteResponse; /** - * Verifies a ValidateVersionKeyspaceResponse message. + * Verifies a WorkflowDeleteResponse 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 ValidateVersionKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowDeleteResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVersionKeyspaceResponse + * @returns WorkflowDeleteResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionKeyspaceResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowDeleteResponse; /** - * Creates a plain object from a ValidateVersionKeyspaceResponse message. Also converts values to other types if specified. - * @param message ValidateVersionKeyspaceResponse + * Creates a plain object from a WorkflowDeleteResponse message. Also converts values to other types if specified. + * @param message WorkflowDeleteResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVersionKeyspaceResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowDeleteResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVersionKeyspaceResponse to JSON. + * Converts this WorkflowDeleteResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVersionKeyspaceResponse + * Gets the default type url for WorkflowDeleteResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVersionShardRequest. */ - interface IValidateVersionShardRequest { + namespace WorkflowDeleteResponse { - /** ValidateVersionShardRequest keyspace */ + /** Properties of a TabletInfo. */ + interface ITabletInfo { + + /** TabletInfo tablet */ + tablet?: (topodata.ITabletAlias|null); + + /** TabletInfo deleted */ + deleted?: (boolean|null); + } + + /** Represents a TabletInfo. */ + class TabletInfo implements ITabletInfo { + + /** + * Constructs a new TabletInfo. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.WorkflowDeleteResponse.ITabletInfo); + + /** TabletInfo tablet. */ + public tablet?: (topodata.ITabletAlias|null); + + /** TabletInfo deleted. */ + public deleted: boolean; + + /** + * Creates a new TabletInfo instance using the specified properties. + * @param [properties] Properties to set + * @returns TabletInfo instance + */ + public static create(properties?: vtctldata.WorkflowDeleteResponse.ITabletInfo): vtctldata.WorkflowDeleteResponse.TabletInfo; + + /** + * Encodes the specified TabletInfo message. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.TabletInfo.verify|verify} messages. + * @param message TabletInfo message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.WorkflowDeleteResponse.ITabletInfo, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified TabletInfo message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.TabletInfo.verify|verify} messages. + * @param message TabletInfo message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.WorkflowDeleteResponse.ITabletInfo, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a TabletInfo message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns TabletInfo + * @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.WorkflowDeleteResponse.TabletInfo; + + /** + * Decodes a TabletInfo message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns TabletInfo + * @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.WorkflowDeleteResponse.TabletInfo; + + /** + * Verifies a TabletInfo 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 TabletInfo message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns TabletInfo + */ + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowDeleteResponse.TabletInfo; + + /** + * Creates a plain object from a TabletInfo message. Also converts values to other types if specified. + * @param message TabletInfo + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.WorkflowDeleteResponse.TabletInfo, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this TabletInfo to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for TabletInfo + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + } + + /** Properties of a WorkflowStatusRequest. */ + interface IWorkflowStatusRequest { + + /** WorkflowStatusRequest keyspace */ keyspace?: (string|null); - /** ValidateVersionShardRequest shard */ - shard?: (string|null); + /** WorkflowStatusRequest workflow */ + workflow?: (string|null); } - /** Represents a ValidateVersionShardRequest. */ - class ValidateVersionShardRequest implements IValidateVersionShardRequest { + /** Represents a WorkflowStatusRequest. */ + class WorkflowStatusRequest implements IWorkflowStatusRequest { /** - * Constructs a new ValidateVersionShardRequest. + * Constructs a new WorkflowStatusRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVersionShardRequest); + constructor(properties?: vtctldata.IWorkflowStatusRequest); - /** ValidateVersionShardRequest keyspace. */ + /** WorkflowStatusRequest keyspace. */ public keyspace: string; - /** ValidateVersionShardRequest shard. */ - public shard: string; + /** WorkflowStatusRequest workflow. */ + public workflow: string; /** - * Creates a new ValidateVersionShardRequest instance using the specified properties. + * Creates a new WorkflowStatusRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVersionShardRequest instance + * @returns WorkflowStatusRequest instance */ - public static create(properties?: vtctldata.IValidateVersionShardRequest): vtctldata.ValidateVersionShardRequest; + public static create(properties?: vtctldata.IWorkflowStatusRequest): vtctldata.WorkflowStatusRequest; /** - * Encodes the specified ValidateVersionShardRequest message. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. - * @param message ValidateVersionShardRequest message or plain object to encode + * Encodes the specified WorkflowStatusRequest message. Does not implicitly {@link vtctldata.WorkflowStatusRequest.verify|verify} messages. + * @param message WorkflowStatusRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVersionShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVersionShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. - * @param message ValidateVersionShardRequest message or plain object to encode + * Encodes the specified WorkflowStatusRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusRequest.verify|verify} messages. + * @param message WorkflowStatusRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVersionShardRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowStatusRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVersionShardRequest message from the specified reader or buffer. + * Decodes a WorkflowStatusRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVersionShardRequest + * @returns WorkflowStatusRequest * @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.ValidateVersionShardRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowStatusRequest; /** - * Decodes a ValidateVersionShardRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowStatusRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVersionShardRequest + * @returns WorkflowStatusRequest * @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.ValidateVersionShardRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowStatusRequest; /** - * Verifies a ValidateVersionShardRequest message. + * Verifies a WorkflowStatusRequest 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 ValidateVersionShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowStatusRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVersionShardRequest + * @returns WorkflowStatusRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionShardRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowStatusRequest; /** - * Creates a plain object from a ValidateVersionShardRequest message. Also converts values to other types if specified. - * @param message ValidateVersionShardRequest + * Creates a plain object from a WorkflowStatusRequest message. Also converts values to other types if specified. + * @param message WorkflowStatusRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVersionShardRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowStatusRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVersionShardRequest to JSON. + * Converts this WorkflowStatusRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVersionShardRequest + * Gets the default type url for WorkflowStatusRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVersionShardResponse. */ - interface IValidateVersionShardResponse { + /** Properties of a WorkflowStatusResponse. */ + interface IWorkflowStatusResponse { - /** ValidateVersionShardResponse results */ - results?: (string[]|null); + /** WorkflowStatusResponse table_copy_state */ + table_copy_state?: ({ [k: string]: vtctldata.WorkflowStatusResponse.ITableCopyState }|null); + + /** WorkflowStatusResponse shard_streams */ + shard_streams?: ({ [k: string]: vtctldata.WorkflowStatusResponse.IShardStreams }|null); } - /** Represents a ValidateVersionShardResponse. */ - class ValidateVersionShardResponse implements IValidateVersionShardResponse { + /** Represents a WorkflowStatusResponse. */ + class WorkflowStatusResponse implements IWorkflowStatusResponse { /** - * Constructs a new ValidateVersionShardResponse. + * Constructs a new WorkflowStatusResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVersionShardResponse); + constructor(properties?: vtctldata.IWorkflowStatusResponse); - /** ValidateVersionShardResponse results. */ - public results: string[]; + /** WorkflowStatusResponse table_copy_state. */ + public table_copy_state: { [k: string]: vtctldata.WorkflowStatusResponse.ITableCopyState }; + + /** WorkflowStatusResponse shard_streams. */ + public shard_streams: { [k: string]: vtctldata.WorkflowStatusResponse.IShardStreams }; /** - * Creates a new ValidateVersionShardResponse instance using the specified properties. + * Creates a new WorkflowStatusResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVersionShardResponse instance + * @returns WorkflowStatusResponse instance */ - public static create(properties?: vtctldata.IValidateVersionShardResponse): vtctldata.ValidateVersionShardResponse; + public static create(properties?: vtctldata.IWorkflowStatusResponse): vtctldata.WorkflowStatusResponse; /** - * Encodes the specified ValidateVersionShardResponse message. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. - * @param message ValidateVersionShardResponse message or plain object to encode + * Encodes the specified WorkflowStatusResponse message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.verify|verify} messages. + * @param message WorkflowStatusResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVersionShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVersionShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. - * @param message ValidateVersionShardResponse message or plain object to encode + * Encodes the specified WorkflowStatusResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.verify|verify} messages. + * @param message WorkflowStatusResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVersionShardResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowStatusResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVersionShardResponse message from the specified reader or buffer. + * Decodes a WorkflowStatusResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVersionShardResponse + * @returns WorkflowStatusResponse * @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.ValidateVersionShardResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowStatusResponse; /** - * Decodes a ValidateVersionShardResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowStatusResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVersionShardResponse + * @returns WorkflowStatusResponse * @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.ValidateVersionShardResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowStatusResponse; /** - * Verifies a ValidateVersionShardResponse message. + * Verifies a WorkflowStatusResponse 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 ValidateVersionShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowStatusResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVersionShardResponse + * @returns WorkflowStatusResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVersionShardResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowStatusResponse; /** - * Creates a plain object from a ValidateVersionShardResponse message. Also converts values to other types if specified. - * @param message ValidateVersionShardResponse + * Creates a plain object from a WorkflowStatusResponse message. Also converts values to other types if specified. + * @param message WorkflowStatusResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVersionShardResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowStatusResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVersionShardResponse to JSON. + * Converts this WorkflowStatusResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVersionShardResponse + * Gets the default type url for WorkflowStatusResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVSchemaRequest. */ - interface IValidateVSchemaRequest { + namespace WorkflowStatusResponse { - /** ValidateVSchemaRequest keyspace */ + /** Properties of a TableCopyState. */ + interface ITableCopyState { + + /** TableCopyState rows_copied */ + rows_copied?: (number|Long|null); + + /** TableCopyState rows_total */ + rows_total?: (number|Long|null); + + /** TableCopyState rows_percentage */ + rows_percentage?: (number|null); + + /** TableCopyState bytes_copied */ + bytes_copied?: (number|Long|null); + + /** TableCopyState bytes_total */ + bytes_total?: (number|Long|null); + + /** TableCopyState bytes_percentage */ + bytes_percentage?: (number|null); + } + + /** Represents a TableCopyState. */ + class TableCopyState implements ITableCopyState { + + /** + * Constructs a new TableCopyState. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.WorkflowStatusResponse.ITableCopyState); + + /** TableCopyState rows_copied. */ + public rows_copied: (number|Long); + + /** TableCopyState rows_total. */ + public rows_total: (number|Long); + + /** TableCopyState rows_percentage. */ + public rows_percentage: number; + + /** TableCopyState bytes_copied. */ + public bytes_copied: (number|Long); + + /** TableCopyState bytes_total. */ + public bytes_total: (number|Long); + + /** TableCopyState bytes_percentage. */ + public bytes_percentage: number; + + /** + * Creates a new TableCopyState instance using the specified properties. + * @param [properties] Properties to set + * @returns TableCopyState instance + */ + public static create(properties?: vtctldata.WorkflowStatusResponse.ITableCopyState): vtctldata.WorkflowStatusResponse.TableCopyState; + + /** + * Encodes the specified TableCopyState message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.TableCopyState.verify|verify} messages. + * @param message TableCopyState message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.WorkflowStatusResponse.ITableCopyState, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified TableCopyState message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.TableCopyState.verify|verify} messages. + * @param message TableCopyState message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.WorkflowStatusResponse.ITableCopyState, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a TableCopyState message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns TableCopyState + * @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.WorkflowStatusResponse.TableCopyState; + + /** + * Decodes a TableCopyState message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns TableCopyState + * @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.WorkflowStatusResponse.TableCopyState; + + /** + * Verifies a TableCopyState 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 TableCopyState message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns TableCopyState + */ + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowStatusResponse.TableCopyState; + + /** + * Creates a plain object from a TableCopyState message. Also converts values to other types if specified. + * @param message TableCopyState + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.WorkflowStatusResponse.TableCopyState, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this TableCopyState to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for TableCopyState + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a ShardStreamState. */ + interface IShardStreamState { + + /** ShardStreamState id */ + id?: (number|null); + + /** ShardStreamState tablet */ + tablet?: (topodata.ITabletAlias|null); + + /** ShardStreamState source_shard */ + source_shard?: (string|null); + + /** ShardStreamState position */ + position?: (string|null); + + /** ShardStreamState status */ + status?: (string|null); + + /** ShardStreamState info */ + info?: (string|null); + } + + /** Represents a ShardStreamState. */ + class ShardStreamState implements IShardStreamState { + + /** + * Constructs a new ShardStreamState. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.WorkflowStatusResponse.IShardStreamState); + + /** ShardStreamState id. */ + public id: number; + + /** ShardStreamState tablet. */ + public tablet?: (topodata.ITabletAlias|null); + + /** ShardStreamState source_shard. */ + public source_shard: string; + + /** ShardStreamState position. */ + public position: string; + + /** ShardStreamState status. */ + public status: string; + + /** ShardStreamState info. */ + public info: string; + + /** + * Creates a new ShardStreamState instance using the specified properties. + * @param [properties] Properties to set + * @returns ShardStreamState instance + */ + public static create(properties?: vtctldata.WorkflowStatusResponse.IShardStreamState): vtctldata.WorkflowStatusResponse.ShardStreamState; + + /** + * Encodes the specified ShardStreamState message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreamState.verify|verify} messages. + * @param message ShardStreamState message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.WorkflowStatusResponse.IShardStreamState, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ShardStreamState message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreamState.verify|verify} messages. + * @param message ShardStreamState message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.WorkflowStatusResponse.IShardStreamState, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ShardStreamState message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ShardStreamState + * @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.WorkflowStatusResponse.ShardStreamState; + + /** + * Decodes a ShardStreamState message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ShardStreamState + * @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.WorkflowStatusResponse.ShardStreamState; + + /** + * Verifies a ShardStreamState 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 ShardStreamState message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ShardStreamState + */ + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowStatusResponse.ShardStreamState; + + /** + * Creates a plain object from a ShardStreamState message. Also converts values to other types if specified. + * @param message ShardStreamState + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.WorkflowStatusResponse.ShardStreamState, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ShardStreamState to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ShardStreamState + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a ShardStreams. */ + interface IShardStreams { + + /** ShardStreams streams */ + streams?: (vtctldata.WorkflowStatusResponse.IShardStreamState[]|null); + } + + /** Represents a ShardStreams. */ + class ShardStreams implements IShardStreams { + + /** + * Constructs a new ShardStreams. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.WorkflowStatusResponse.IShardStreams); + + /** ShardStreams streams. */ + public streams: vtctldata.WorkflowStatusResponse.IShardStreamState[]; + + /** + * Creates a new ShardStreams instance using the specified properties. + * @param [properties] Properties to set + * @returns ShardStreams instance + */ + public static create(properties?: vtctldata.WorkflowStatusResponse.IShardStreams): vtctldata.WorkflowStatusResponse.ShardStreams; + + /** + * Encodes the specified ShardStreams message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreams.verify|verify} messages. + * @param message ShardStreams message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.WorkflowStatusResponse.IShardStreams, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ShardStreams message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreams.verify|verify} messages. + * @param message ShardStreams message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.WorkflowStatusResponse.IShardStreams, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ShardStreams message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ShardStreams + * @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.WorkflowStatusResponse.ShardStreams; + + /** + * Decodes a ShardStreams message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ShardStreams + * @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.WorkflowStatusResponse.ShardStreams; + + /** + * Verifies a ShardStreams 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 ShardStreams message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ShardStreams + */ + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowStatusResponse.ShardStreams; + + /** + * Creates a plain object from a ShardStreams message. Also converts values to other types if specified. + * @param message ShardStreams + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.WorkflowStatusResponse.ShardStreams, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ShardStreams to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ShardStreams + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + } + + /** Properties of a WorkflowSwitchTrafficRequest. */ + interface IWorkflowSwitchTrafficRequest { + + /** WorkflowSwitchTrafficRequest keyspace */ keyspace?: (string|null); - /** ValidateVSchemaRequest shards */ - shards?: (string[]|null); + /** WorkflowSwitchTrafficRequest workflow */ + workflow?: (string|null); - /** ValidateVSchemaRequest exclude_tables */ - exclude_tables?: (string[]|null); + /** WorkflowSwitchTrafficRequest cells */ + cells?: (string[]|null); - /** ValidateVSchemaRequest include_views */ - include_views?: (boolean|null); + /** WorkflowSwitchTrafficRequest tablet_types */ + tablet_types?: (topodata.TabletType[]|null); + + /** WorkflowSwitchTrafficRequest max_replication_lag_allowed */ + max_replication_lag_allowed?: (vttime.IDuration|null); + + /** WorkflowSwitchTrafficRequest enable_reverse_replication */ + enable_reverse_replication?: (boolean|null); + + /** WorkflowSwitchTrafficRequest direction */ + direction?: (number|null); + + /** WorkflowSwitchTrafficRequest timeout */ + timeout?: (vttime.IDuration|null); + + /** WorkflowSwitchTrafficRequest dry_run */ + dry_run?: (boolean|null); } - /** Represents a ValidateVSchemaRequest. */ - class ValidateVSchemaRequest implements IValidateVSchemaRequest { + /** Represents a WorkflowSwitchTrafficRequest. */ + class WorkflowSwitchTrafficRequest implements IWorkflowSwitchTrafficRequest { /** - * Constructs a new ValidateVSchemaRequest. + * Constructs a new WorkflowSwitchTrafficRequest. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVSchemaRequest); + constructor(properties?: vtctldata.IWorkflowSwitchTrafficRequest); - /** ValidateVSchemaRequest keyspace. */ + /** WorkflowSwitchTrafficRequest keyspace. */ public keyspace: string; - /** ValidateVSchemaRequest shards. */ - public shards: string[]; + /** WorkflowSwitchTrafficRequest workflow. */ + public workflow: string; - /** ValidateVSchemaRequest exclude_tables. */ - public exclude_tables: string[]; + /** WorkflowSwitchTrafficRequest cells. */ + public cells: string[]; - /** ValidateVSchemaRequest include_views. */ - public include_views: boolean; + /** WorkflowSwitchTrafficRequest tablet_types. */ + public tablet_types: topodata.TabletType[]; + + /** WorkflowSwitchTrafficRequest max_replication_lag_allowed. */ + public max_replication_lag_allowed?: (vttime.IDuration|null); + + /** WorkflowSwitchTrafficRequest enable_reverse_replication. */ + public enable_reverse_replication: boolean; + + /** WorkflowSwitchTrafficRequest direction. */ + public direction: number; + + /** WorkflowSwitchTrafficRequest timeout. */ + public timeout?: (vttime.IDuration|null); + + /** WorkflowSwitchTrafficRequest dry_run. */ + public dry_run: boolean; /** - * Creates a new ValidateVSchemaRequest instance using the specified properties. + * Creates a new WorkflowSwitchTrafficRequest instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVSchemaRequest instance + * @returns WorkflowSwitchTrafficRequest instance */ - public static create(properties?: vtctldata.IValidateVSchemaRequest): vtctldata.ValidateVSchemaRequest; + public static create(properties?: vtctldata.IWorkflowSwitchTrafficRequest): vtctldata.WorkflowSwitchTrafficRequest; /** - * Encodes the specified ValidateVSchemaRequest message. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. - * @param message ValidateVSchemaRequest message or plain object to encode + * Encodes the specified WorkflowSwitchTrafficRequest message. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficRequest.verify|verify} messages. + * @param message WorkflowSwitchTrafficRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowSwitchTrafficRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. - * @param message ValidateVSchemaRequest message or plain object to encode + * Encodes the specified WorkflowSwitchTrafficRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficRequest.verify|verify} messages. + * @param message WorkflowSwitchTrafficRequest message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVSchemaRequest, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowSwitchTrafficRequest, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVSchemaRequest message from the specified reader or buffer. + * Decodes a WorkflowSwitchTrafficRequest message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVSchemaRequest + * @returns WorkflowSwitchTrafficRequest * @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.ValidateVSchemaRequest; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowSwitchTrafficRequest; /** - * Decodes a ValidateVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowSwitchTrafficRequest message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVSchemaRequest + * @returns WorkflowSwitchTrafficRequest * @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.ValidateVSchemaRequest; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowSwitchTrafficRequest; /** - * Verifies a ValidateVSchemaRequest message. + * Verifies a WorkflowSwitchTrafficRequest 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 ValidateVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowSwitchTrafficRequest message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVSchemaRequest + * @returns WorkflowSwitchTrafficRequest */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVSchemaRequest; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowSwitchTrafficRequest; /** - * Creates a plain object from a ValidateVSchemaRequest message. Also converts values to other types if specified. - * @param message ValidateVSchemaRequest + * Creates a plain object from a WorkflowSwitchTrafficRequest message. Also converts values to other types if specified. + * @param message WorkflowSwitchTrafficRequest * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVSchemaRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowSwitchTrafficRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVSchemaRequest to JSON. + * Converts this WorkflowSwitchTrafficRequest to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVSchemaRequest + * Gets the default type url for WorkflowSwitchTrafficRequest * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a ValidateVSchemaResponse. */ - interface IValidateVSchemaResponse { + /** Properties of a WorkflowSwitchTrafficResponse. */ + interface IWorkflowSwitchTrafficResponse { - /** ValidateVSchemaResponse results */ - results?: (string[]|null); + /** WorkflowSwitchTrafficResponse summary */ + summary?: (string|null); - /** ValidateVSchemaResponse results_by_shard */ - results_by_shard?: ({ [k: string]: vtctldata.IValidateShardResponse }|null); + /** WorkflowSwitchTrafficResponse start_state */ + start_state?: (string|null); + + /** WorkflowSwitchTrafficResponse current_state */ + current_state?: (string|null); + + /** WorkflowSwitchTrafficResponse dry_run_results */ + dry_run_results?: (string[]|null); } - /** Represents a ValidateVSchemaResponse. */ - class ValidateVSchemaResponse implements IValidateVSchemaResponse { + /** Represents a WorkflowSwitchTrafficResponse. */ + class WorkflowSwitchTrafficResponse implements IWorkflowSwitchTrafficResponse { /** - * Constructs a new ValidateVSchemaResponse. + * Constructs a new WorkflowSwitchTrafficResponse. * @param [properties] Properties to set */ - constructor(properties?: vtctldata.IValidateVSchemaResponse); + constructor(properties?: vtctldata.IWorkflowSwitchTrafficResponse); - /** ValidateVSchemaResponse results. */ - public results: string[]; + /** WorkflowSwitchTrafficResponse summary. */ + public summary: string; - /** ValidateVSchemaResponse results_by_shard. */ - public results_by_shard: { [k: string]: vtctldata.IValidateShardResponse }; + /** WorkflowSwitchTrafficResponse start_state. */ + public start_state: string; + + /** WorkflowSwitchTrafficResponse current_state. */ + public current_state: string; + + /** WorkflowSwitchTrafficResponse dry_run_results. */ + public dry_run_results: string[]; /** - * Creates a new ValidateVSchemaResponse instance using the specified properties. + * Creates a new WorkflowSwitchTrafficResponse instance using the specified properties. * @param [properties] Properties to set - * @returns ValidateVSchemaResponse instance + * @returns WorkflowSwitchTrafficResponse instance */ - public static create(properties?: vtctldata.IValidateVSchemaResponse): vtctldata.ValidateVSchemaResponse; + public static create(properties?: vtctldata.IWorkflowSwitchTrafficResponse): vtctldata.WorkflowSwitchTrafficResponse; /** - * Encodes the specified ValidateVSchemaResponse message. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. - * @param message ValidateVSchemaResponse message or plain object to encode + * Encodes the specified WorkflowSwitchTrafficResponse message. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficResponse.verify|verify} messages. + * @param message WorkflowSwitchTrafficResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encode(message: vtctldata.IValidateVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encode(message: vtctldata.IWorkflowSwitchTrafficResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Encodes the specified ValidateVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. - * @param message ValidateVSchemaResponse message or plain object to encode + * Encodes the specified WorkflowSwitchTrafficResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficResponse.verify|verify} messages. + * @param message WorkflowSwitchTrafficResponse message or plain object to encode * @param [writer] Writer to encode to * @returns Writer */ - public static encodeDelimited(message: vtctldata.IValidateVSchemaResponse, writer?: $protobuf.Writer): $protobuf.Writer; + public static encodeDelimited(message: vtctldata.IWorkflowSwitchTrafficResponse, writer?: $protobuf.Writer): $protobuf.Writer; /** - * Decodes a ValidateVSchemaResponse message from the specified reader or buffer. + * Decodes a WorkflowSwitchTrafficResponse message from the specified reader or buffer. * @param reader Reader or buffer to decode from * @param [length] Message length if known beforehand - * @returns ValidateVSchemaResponse + * @returns WorkflowSwitchTrafficResponse * @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.ValidateVSchemaResponse; + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): vtctldata.WorkflowSwitchTrafficResponse; /** - * Decodes a ValidateVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowSwitchTrafficResponse message from the specified reader or buffer, length delimited. * @param reader Reader or buffer to decode from - * @returns ValidateVSchemaResponse + * @returns WorkflowSwitchTrafficResponse * @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.ValidateVSchemaResponse; + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): vtctldata.WorkflowSwitchTrafficResponse; /** - * Verifies a ValidateVSchemaResponse message. + * Verifies a WorkflowSwitchTrafficResponse 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 ValidateVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowSwitchTrafficResponse message from a plain object. Also converts values to their respective internal types. * @param object Plain object - * @returns ValidateVSchemaResponse + * @returns WorkflowSwitchTrafficResponse */ - public static fromObject(object: { [k: string]: any }): vtctldata.ValidateVSchemaResponse; + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowSwitchTrafficResponse; /** - * Creates a plain object from a ValidateVSchemaResponse message. Also converts values to other types if specified. - * @param message ValidateVSchemaResponse + * Creates a plain object from a WorkflowSwitchTrafficResponse message. Also converts values to other types if specified. + * @param message WorkflowSwitchTrafficResponse * @param [options] Conversion options * @returns Plain object */ - public static toObject(message: vtctldata.ValidateVSchemaResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + public static toObject(message: vtctldata.WorkflowSwitchTrafficResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; /** - * Converts this ValidateVSchemaResponse to JSON. + * Converts this WorkflowSwitchTrafficResponse to JSON. * @returns JSON object */ public toJSON(): { [k: string]: any }; /** - * Gets the default type url for ValidateVSchemaResponse + * Gets the default type url for WorkflowSwitchTrafficResponse * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns The default type url */ @@ -59872,7 +62558,7 @@ export namespace vtctldata { keyspace?: (string|null); /** WorkflowUpdateRequest tablet_request */ - tablet_request?: (tabletmanagerdata.IUpdateVRWorkflowRequest|null); + tablet_request?: (tabletmanagerdata.IUpdateVReplicationWorkflowRequest|null); } /** Represents a WorkflowUpdateRequest. */ @@ -59888,7 +62574,7 @@ export namespace vtctldata { public keyspace: string; /** WorkflowUpdateRequest tablet_request. */ - public tablet_request?: (tabletmanagerdata.IUpdateVRWorkflowRequest|null); + public tablet_request?: (tabletmanagerdata.IUpdateVReplicationWorkflowRequest|null); /** * Creates a new WorkflowUpdateRequest instance using the specified properties. @@ -60077,7 +62763,7 @@ export namespace vtctldata { interface ITabletInfo { /** TabletInfo tablet */ - tablet?: (string|null); + tablet?: (topodata.ITabletAlias|null); /** TabletInfo changed */ changed?: (boolean|null); @@ -60093,7 +62779,7 @@ export namespace vtctldata { constructor(properties?: vtctldata.WorkflowUpdateResponse.ITabletInfo); /** TabletInfo tablet. */ - public tablet: string; + public tablet?: (topodata.ITabletAlias|null); /** TabletInfo changed. */ public changed: boolean; diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index 9492ff2ff1f..6030c37f7c9 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -39617,6 +39617,22 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { */ const tabletmanagerdata = {}; + /** + * TabletSelectionPreference enum. + * @name tabletmanagerdata.TabletSelectionPreference + * @enum {number} + * @property {number} ANY=0 ANY value + * @property {number} INORDER=1 INORDER value + * @property {number} UNKNOWN=3 UNKNOWN value + */ + tabletmanagerdata.TabletSelectionPreference = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "ANY"] = 0; + values[valuesById[1] = "INORDER"] = 1; + values[valuesById[3] = "UNKNOWN"] = 3; + return values; + })(); + tabletmanagerdata.TableDefinition = (function() { /** @@ -60210,29 +60226,36 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return RestoreFromBackupResponse; })(); - tabletmanagerdata.VDiffRequest = (function() { + tabletmanagerdata.CreateVReplicationWorkflowRequest = (function() { /** - * Properties of a VDiffRequest. + * Properties of a CreateVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @interface IVDiffRequest - * @property {string|null} [keyspace] VDiffRequest keyspace - * @property {string|null} [workflow] VDiffRequest workflow - * @property {string|null} [action] VDiffRequest action - * @property {string|null} [action_arg] VDiffRequest action_arg - * @property {string|null} [vdiff_uuid] VDiffRequest vdiff_uuid - * @property {tabletmanagerdata.IVDiffOptions|null} [options] VDiffRequest options + * @interface ICreateVReplicationWorkflowRequest + * @property {string|null} [workflow] CreateVReplicationWorkflowRequest workflow + * @property {Array.|null} [binlog_source] CreateVReplicationWorkflowRequest binlog_source + * @property {Array.|null} [cells] CreateVReplicationWorkflowRequest cells + * @property {Array.|null} [tablet_types] CreateVReplicationWorkflowRequest tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] CreateVReplicationWorkflowRequest tablet_selection_preference + * @property {binlogdata.VReplicationWorkflowType|null} [workflow_type] CreateVReplicationWorkflowRequest workflow_type + * @property {binlogdata.VReplicationWorkflowSubType|null} [workflow_sub_type] CreateVReplicationWorkflowRequest workflow_sub_type + * @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 */ /** - * Constructs a new VDiffRequest. + * Constructs a new CreateVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffRequest. - * @implements IVDiffRequest + * @classdesc Represents a CreateVReplicationWorkflowRequest. + * @implements ICreateVReplicationWorkflowRequest * @constructor - * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + * @param {tabletmanagerdata.ICreateVReplicationWorkflowRequest=} [properties] Properties to set */ - function VDiffRequest(properties) { + function CreateVReplicationWorkflowRequest(properties) { + this.binlog_source = []; + this.cells = []; + this.tablet_types = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -60240,145 +60263,218 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffRequest keyspace. - * @member {string} keyspace - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.keyspace = ""; + CreateVReplicationWorkflowRequest.prototype.workflow = ""; /** - * VDiffRequest workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest binlog_source. + * @member {Array.} binlog_source + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.workflow = ""; + CreateVReplicationWorkflowRequest.prototype.binlog_source = $util.emptyArray; /** - * VDiffRequest action. - * @member {string} action - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest cells. + * @member {Array.} cells + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.action = ""; + CreateVReplicationWorkflowRequest.prototype.cells = $util.emptyArray; /** - * VDiffRequest action_arg. - * @member {string} action_arg - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest tablet_types. + * @member {Array.} tablet_types + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.action_arg = ""; + CreateVReplicationWorkflowRequest.prototype.tablet_types = $util.emptyArray; /** - * VDiffRequest vdiff_uuid. - * @member {string} vdiff_uuid - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.vdiff_uuid = ""; + CreateVReplicationWorkflowRequest.prototype.tablet_selection_preference = 0; /** - * VDiffRequest options. - * @member {tabletmanagerdata.IVDiffOptions|null|undefined} options - * @memberof tabletmanagerdata.VDiffRequest + * CreateVReplicationWorkflowRequest workflow_type. + * @member {binlogdata.VReplicationWorkflowType} workflow_type + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance */ - VDiffRequest.prototype.options = null; + CreateVReplicationWorkflowRequest.prototype.workflow_type = 0; /** - * Creates a new VDiffRequest instance using the specified properties. + * CreateVReplicationWorkflowRequest workflow_sub_type. + * @member {binlogdata.VReplicationWorkflowSubType} workflow_sub_type + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest + * @instance + */ + CreateVReplicationWorkflowRequest.prototype.workflow_sub_type = 0; + + /** + * CreateVReplicationWorkflowRequest defer_secondary_keys. + * @member {boolean} defer_secondary_keys + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest + * @instance + */ + CreateVReplicationWorkflowRequest.prototype.defer_secondary_keys = false; + + /** + * CreateVReplicationWorkflowRequest auto_start. + * @member {boolean} auto_start + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest + * @instance + */ + CreateVReplicationWorkflowRequest.prototype.auto_start = false; + + /** + * CreateVReplicationWorkflowRequest stop_after_copy. + * @member {boolean} stop_after_copy + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest + * @instance + */ + CreateVReplicationWorkflowRequest.prototype.stop_after_copy = false; + + /** + * Creates a new CreateVReplicationWorkflowRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest instance + * @param {tabletmanagerdata.ICreateVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.CreateVReplicationWorkflowRequest} CreateVReplicationWorkflowRequest instance */ - VDiffRequest.create = function create(properties) { - return new VDiffRequest(properties); + CreateVReplicationWorkflowRequest.create = function create(properties) { + return new CreateVReplicationWorkflowRequest(properties); }; /** - * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * Encodes the specified CreateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {tabletmanagerdata.ICreateVReplicationWorkflowRequest} message CreateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffRequest.encode = function encode(message, writer) { + CreateVReplicationWorkflowRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); - if (message.action != null && Object.hasOwnProperty.call(message, "action")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.action); - if (message.action_arg != null && Object.hasOwnProperty.call(message, "action_arg")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.action_arg); - if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.vdiff_uuid); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.tabletmanagerdata.VDiffOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.binlog_source != null && message.binlog_source.length) + for (let i = 0; i < message.binlog_source.length; ++i) + $root.binlogdata.BinlogSource.encode(message.binlog_source[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 4, wireType 2 =*/34).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 5, wireType 0 =*/40).int32(message.tablet_selection_preference); + if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.workflow_type); + if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) + writer.uint32(/* id 7, wireType 0 =*/56).int32(message.workflow_sub_type); + if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.defer_secondary_keys); + if (message.auto_start != null && Object.hasOwnProperty.call(message, "auto_start")) + 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); return writer; }; /** - * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * Encodes the specified CreateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {tabletmanagerdata.ICreateVReplicationWorkflowRequest} message CreateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffRequest.encodeDelimited = function encodeDelimited(message, writer) { + CreateVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffRequest message from the specified reader or buffer. + * Decodes a CreateVReplicationWorkflowRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.CreateVReplicationWorkflowRequest} CreateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffRequest.decode = function decode(reader, length) { + CreateVReplicationWorkflowRequest.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.tabletmanagerdata.VDiffRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.CreateVReplicationWorkflowRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.workflow = reader.string(); break; } case 2: { - message.workflow = reader.string(); + if (!(message.binlog_source && message.binlog_source.length)) + message.binlog_source = []; + message.binlog_source.push($root.binlogdata.BinlogSource.decode(reader, reader.uint32())); break; } case 3: { - message.action = reader.string(); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } case 4: { - message.action_arg = reader.string(); + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); break; } case 5: { - message.vdiff_uuid = reader.string(); + message.tablet_selection_preference = reader.int32(); break; } case 6: { - message.options = $root.tabletmanagerdata.VDiffOptions.decode(reader, reader.uint32()); + message.workflow_type = reader.int32(); + break; + } + case 7: { + message.workflow_sub_type = reader.int32(); + break; + } + case 8: { + message.defer_secondary_keys = reader.bool(); + break; + } + case 9: { + message.auto_start = reader.bool(); + break; + } + case 10: { + message.stop_after_copy = reader.bool(); break; } default: @@ -60390,170 +60486,384 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.CreateVReplicationWorkflowRequest} CreateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffRequest.decodeDelimited = function decodeDelimited(reader) { + CreateVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffRequest message. + * Verifies a CreateVReplicationWorkflowRequest message. * @function verify - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffRequest.verify = function verify(message) { + CreateVReplicationWorkflowRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; if (message.workflow != null && message.hasOwnProperty("workflow")) if (!$util.isString(message.workflow)) return "workflow: string expected"; - if (message.action != null && message.hasOwnProperty("action")) - if (!$util.isString(message.action)) - return "action: string expected"; - if (message.action_arg != null && message.hasOwnProperty("action_arg")) - if (!$util.isString(message.action_arg)) - return "action_arg: string expected"; - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - if (!$util.isString(message.vdiff_uuid)) - return "vdiff_uuid: string expected"; - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.tabletmanagerdata.VDiffOptions.verify(message.options); - if (error) - return "options." + error; + if (message.binlog_source != null && message.hasOwnProperty("binlog_source")) { + if (!Array.isArray(message.binlog_source)) + return "binlog_source: array expected"; + for (let i = 0; i < message.binlog_source.length; ++i) { + let error = $root.binlogdata.BinlogSource.verify(message.binlog_source[i]); + if (error) + return "binlog_source." + error; + } + } + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + switch (message.workflow_type) { + default: + return "workflow_type: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + break; + } + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + switch (message.workflow_sub_type) { + default: + return "workflow_sub_type: enum value expected"; + case 0: + case 1: + break; + } + 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.auto_start != null && message.hasOwnProperty("auto_start")) + if (typeof message.auto_start !== "boolean") + return "auto_start: boolean expected"; + if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) + if (typeof message.stop_after_copy !== "boolean") + return "stop_after_copy: boolean expected"; return null; }; /** - * Creates a VDiffRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.CreateVReplicationWorkflowRequest} CreateVReplicationWorkflowRequest */ - VDiffRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffRequest) + CreateVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.CreateVReplicationWorkflowRequest) return object; - let message = new $root.tabletmanagerdata.VDiffRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); + let message = new $root.tabletmanagerdata.CreateVReplicationWorkflowRequest(); if (object.workflow != null) message.workflow = String(object.workflow); - if (object.action != null) - message.action = String(object.action); - if (object.action_arg != null) - message.action_arg = String(object.action_arg); - if (object.vdiff_uuid != null) - message.vdiff_uuid = String(object.vdiff_uuid); - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".tabletmanagerdata.VDiffRequest.options: object expected"); - message.options = $root.tabletmanagerdata.VDiffOptions.fromObject(object.options); + if (object.binlog_source) { + if (!Array.isArray(object.binlog_source)) + throw TypeError(".tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source: array expected"); + message.binlog_source = []; + for (let i = 0; i < object.binlog_source.length; ++i) { + if (typeof object.binlog_source[i] !== "object") + throw TypeError(".tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source: object expected"); + message.binlog_source[i] = $root.binlogdata.BinlogSource.fromObject(object.binlog_source[i]); + } + } + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".tabletmanagerdata.CreateVReplicationWorkflowRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } + } + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; + break; + } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + switch (object.workflow_type) { + default: + if (typeof object.workflow_type === "number") { + message.workflow_type = object.workflow_type; + break; + } + break; + case "Materialize": + case 0: + message.workflow_type = 0; + break; + case "MoveTables": + case 1: + message.workflow_type = 1; + break; + case "CreateLookupIndex": + case 2: + message.workflow_type = 2; + break; + case "Migrate": + case 3: + message.workflow_type = 3; + break; + case "Reshard": + case 4: + message.workflow_type = 4; + break; + case "OnlineDDL": + case 5: + message.workflow_type = 5; + break; } + switch (object.workflow_sub_type) { + default: + if (typeof object.workflow_sub_type === "number") { + message.workflow_sub_type = object.workflow_sub_type; + break; + } + break; + case "None": + case 0: + message.workflow_sub_type = 0; + break; + case "Partial": + case 1: + message.workflow_sub_type = 1; + break; + } + if (object.defer_secondary_keys != null) + message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + if (object.auto_start != null) + message.auto_start = Boolean(object.auto_start); + if (object.stop_after_copy != null) + message.stop_after_copy = Boolean(object.stop_after_copy); return message; }; /** - * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. + * Creates a plain object from a CreateVReplicationWorkflowRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.VDiffRequest} message VDiffRequest + * @param {tabletmanagerdata.CreateVReplicationWorkflowRequest} message CreateVReplicationWorkflowRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffRequest.toObject = function toObject(message, options) { + CreateVReplicationWorkflowRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.binlog_source = []; + object.cells = []; + object.tablet_types = []; + } if (options.defaults) { - object.keyspace = ""; object.workflow = ""; - object.action = ""; - object.action_arg = ""; - object.vdiff_uuid = ""; - object.options = null; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.workflow_type = options.enums === String ? "Materialize" : 0; + object.workflow_sub_type = options.enums === String ? "None" : 0; + object.defer_secondary_keys = false; + object.auto_start = false; + object.stop_after_copy = false; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; - if (message.action != null && message.hasOwnProperty("action")) - object.action = message.action; - if (message.action_arg != null && message.hasOwnProperty("action_arg")) - object.action_arg = message.action_arg; - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - object.vdiff_uuid = message.vdiff_uuid; - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.tabletmanagerdata.VDiffOptions.toObject(message.options, options); + if (message.binlog_source && message.binlog_source.length) { + object.binlog_source = []; + for (let j = 0; j < message.binlog_source.length; ++j) + object.binlog_source[j] = $root.binlogdata.BinlogSource.toObject(message.binlog_source[j], options); + } + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + object.workflow_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowType[message.workflow_type] === undefined ? message.workflow_type : $root.binlogdata.VReplicationWorkflowType[message.workflow_type] : message.workflow_type; + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + object.workflow_sub_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] === undefined ? message.workflow_sub_type : $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] : message.workflow_sub_type; + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + object.defer_secondary_keys = message.defer_secondary_keys; + if (message.auto_start != null && message.hasOwnProperty("auto_start")) + 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; return object; }; /** - * Converts this VDiffRequest to JSON. + * Converts this CreateVReplicationWorkflowRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @instance * @returns {Object.} JSON object */ - VDiffRequest.prototype.toJSON = function toJSON() { + CreateVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffRequest + * Gets the default type url for CreateVReplicationWorkflowRequest * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffRequest"; + return typeUrlPrefix + "/tabletmanagerdata.CreateVReplicationWorkflowRequest"; }; - return VDiffRequest; + return CreateVReplicationWorkflowRequest; })(); - tabletmanagerdata.VDiffResponse = (function() { + tabletmanagerdata.CreateVReplicationWorkflowResponse = (function() { /** - * Properties of a VDiffResponse. + * Properties of a CreateVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @interface IVDiffResponse - * @property {number|Long|null} [id] VDiffResponse id - * @property {query.IQueryResult|null} [output] VDiffResponse output - * @property {string|null} [vdiff_uuid] VDiffResponse vdiff_uuid + * @interface ICreateVReplicationWorkflowResponse + * @property {query.IQueryResult|null} [result] CreateVReplicationWorkflowResponse result */ /** - * Constructs a new VDiffResponse. + * Constructs a new CreateVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffResponse. - * @implements IVDiffResponse + * @classdesc Represents a CreateVReplicationWorkflowResponse. + * @implements ICreateVReplicationWorkflowResponse * @constructor - * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set + * @param {tabletmanagerdata.ICreateVReplicationWorkflowResponse=} [properties] Properties to set */ - function VDiffResponse(properties) { + function CreateVReplicationWorkflowResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -60561,103 +60871,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffResponse id. - * @member {number|Long} id - * @memberof tabletmanagerdata.VDiffResponse - * @instance - */ - VDiffResponse.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * VDiffResponse output. - * @member {query.IQueryResult|null|undefined} output - * @memberof tabletmanagerdata.VDiffResponse - * @instance - */ - VDiffResponse.prototype.output = null; - - /** - * VDiffResponse vdiff_uuid. - * @member {string} vdiff_uuid - * @memberof tabletmanagerdata.VDiffResponse + * CreateVReplicationWorkflowResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @instance */ - VDiffResponse.prototype.vdiff_uuid = ""; + CreateVReplicationWorkflowResponse.prototype.result = null; /** - * Creates a new VDiffResponse instance using the specified properties. + * Creates a new CreateVReplicationWorkflowResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse instance + * @param {tabletmanagerdata.ICreateVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.CreateVReplicationWorkflowResponse} CreateVReplicationWorkflowResponse instance */ - VDiffResponse.create = function create(properties) { - return new VDiffResponse(properties); + CreateVReplicationWorkflowResponse.create = function create(properties) { + return new CreateVReplicationWorkflowResponse(properties); }; /** - * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * Encodes the specified CreateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {tabletmanagerdata.ICreateVReplicationWorkflowResponse} message CreateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffResponse.encode = function encode(message, writer) { + CreateVReplicationWorkflowResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); - if (message.output != null && Object.hasOwnProperty.call(message, "output")) - $root.query.QueryResult.encode(message.output, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.vdiff_uuid); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * Encodes the specified CreateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.CreateVReplicationWorkflowResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {tabletmanagerdata.ICreateVReplicationWorkflowResponse} message CreateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffResponse.encodeDelimited = function encodeDelimited(message, writer) { + CreateVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffResponse message from the specified reader or buffer. + * Decodes a CreateVReplicationWorkflowResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.CreateVReplicationWorkflowResponse} CreateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffResponse.decode = function decode(reader, length) { + CreateVReplicationWorkflowResponse.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.tabletmanagerdata.VDiffResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.CreateVReplicationWorkflowResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.id = reader.int64(); - break; - } - case 2: { - message.output = $root.query.QueryResult.decode(reader, reader.uint32()); - break; - } - case 3: { - message.vdiff_uuid = reader.string(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -60669,160 +60951,127 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.CreateVReplicationWorkflowResponse} CreateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffResponse.decodeDelimited = function decodeDelimited(reader) { + CreateVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffResponse message. + * Verifies a CreateVReplicationWorkflowResponse message. * @function verify - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffResponse.verify = function verify(message) { + CreateVReplicationWorkflowResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) - return "id: integer|Long expected"; - if (message.output != null && message.hasOwnProperty("output")) { - let error = $root.query.QueryResult.verify(message.output); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "output." + error; + return "result." + error; } - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - if (!$util.isString(message.vdiff_uuid)) - return "vdiff_uuid: string expected"; return null; }; /** - * Creates a VDiffResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.CreateVReplicationWorkflowResponse} CreateVReplicationWorkflowResponse */ - VDiffResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffResponse) + CreateVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.CreateVReplicationWorkflowResponse) return object; - let message = new $root.tabletmanagerdata.VDiffResponse(); - if (object.id != null) - if ($util.Long) - (message.id = $util.Long.fromValue(object.id)).unsigned = false; - else if (typeof object.id === "string") - message.id = parseInt(object.id, 10); - else if (typeof object.id === "number") - message.id = object.id; - else if (typeof object.id === "object") - message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); - if (object.output != null) { - if (typeof object.output !== "object") - throw TypeError(".tabletmanagerdata.VDiffResponse.output: object expected"); - message.output = $root.query.QueryResult.fromObject(object.output); + let message = new $root.tabletmanagerdata.CreateVReplicationWorkflowResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".tabletmanagerdata.CreateVReplicationWorkflowResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.vdiff_uuid != null) - message.vdiff_uuid = String(object.vdiff_uuid); return message; }; /** - * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. + * Creates a plain object from a CreateVReplicationWorkflowResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.VDiffResponse} message VDiffResponse + * @param {tabletmanagerdata.CreateVReplicationWorkflowResponse} message CreateVReplicationWorkflowResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffResponse.toObject = function toObject(message, options) { + CreateVReplicationWorkflowResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.id = options.longs === String ? "0" : 0; - object.output = null; - object.vdiff_uuid = ""; - } - if (message.id != null && message.hasOwnProperty("id")) - if (typeof message.id === "number") - object.id = options.longs === String ? String(message.id) : message.id; - else - object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; - if (message.output != null && message.hasOwnProperty("output")) - object.output = $root.query.QueryResult.toObject(message.output, options); - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - object.vdiff_uuid = message.vdiff_uuid; + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this VDiffResponse to JSON. + * Converts this CreateVReplicationWorkflowResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @instance * @returns {Object.} JSON object */ - VDiffResponse.prototype.toJSON = function toJSON() { + CreateVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffResponse + * Gets the default type url for CreateVReplicationWorkflowResponse * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.CreateVReplicationWorkflowResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffResponse"; + return typeUrlPrefix + "/tabletmanagerdata.CreateVReplicationWorkflowResponse"; }; - return VDiffResponse; + return CreateVReplicationWorkflowResponse; })(); - tabletmanagerdata.VDiffPickerOptions = (function() { + tabletmanagerdata.DeleteVReplicationWorkflowRequest = (function() { /** - * Properties of a VDiffPickerOptions. + * Properties of a DeleteVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @interface IVDiffPickerOptions - * @property {string|null} [tablet_types] VDiffPickerOptions tablet_types - * @property {string|null} [source_cell] VDiffPickerOptions source_cell - * @property {string|null} [target_cell] VDiffPickerOptions target_cell + * @interface IDeleteVReplicationWorkflowRequest + * @property {string|null} [workflow] DeleteVReplicationWorkflowRequest workflow */ /** - * Constructs a new VDiffPickerOptions. + * Constructs a new DeleteVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffPickerOptions. - * @implements IVDiffPickerOptions + * @classdesc Represents a DeleteVReplicationWorkflowRequest. + * @implements IDeleteVReplicationWorkflowRequest * @constructor - * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowRequest=} [properties] Properties to set */ - function VDiffPickerOptions(properties) { + function DeleteVReplicationWorkflowRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -60830,103 +61079,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffPickerOptions tablet_types. - * @member {string} tablet_types - * @memberof tabletmanagerdata.VDiffPickerOptions - * @instance - */ - VDiffPickerOptions.prototype.tablet_types = ""; - - /** - * VDiffPickerOptions source_cell. - * @member {string} source_cell - * @memberof tabletmanagerdata.VDiffPickerOptions - * @instance - */ - VDiffPickerOptions.prototype.source_cell = ""; - - /** - * VDiffPickerOptions target_cell. - * @member {string} target_cell - * @memberof tabletmanagerdata.VDiffPickerOptions + * DeleteVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @instance */ - VDiffPickerOptions.prototype.target_cell = ""; + DeleteVReplicationWorkflowRequest.prototype.workflow = ""; /** - * Creates a new VDiffPickerOptions instance using the specified properties. + * Creates a new DeleteVReplicationWorkflowRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions instance + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowRequest} DeleteVReplicationWorkflowRequest instance */ - VDiffPickerOptions.create = function create(properties) { - return new VDiffPickerOptions(properties); + DeleteVReplicationWorkflowRequest.create = function create(properties) { + return new DeleteVReplicationWorkflowRequest(properties); }; /** - * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * Encodes the specified DeleteVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowRequest} message DeleteVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffPickerOptions.encode = function encode(message, writer) { + DeleteVReplicationWorkflowRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tablet_types); - if (message.source_cell != null && Object.hasOwnProperty.call(message, "source_cell")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_cell); - if (message.target_cell != null && Object.hasOwnProperty.call(message, "target_cell")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_cell); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); return writer; }; /** - * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * Encodes the specified DeleteVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowRequest} message DeleteVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffPickerOptions.encodeDelimited = function encodeDelimited(message, writer) { + DeleteVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer. + * Decodes a DeleteVReplicationWorkflowRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowRequest} DeleteVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffPickerOptions.decode = function decode(reader, length) { + DeleteVReplicationWorkflowRequest.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.tabletmanagerdata.VDiffPickerOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.DeleteVReplicationWorkflowRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_types = reader.string(); - break; - } - case 2: { - message.source_cell = reader.string(); - break; - } - case 3: { - message.target_cell = reader.string(); + message.workflow = reader.string(); break; } default: @@ -60938,141 +61159,122 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. + * Decodes a DeleteVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowRequest} DeleteVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffPickerOptions.decodeDelimited = function decodeDelimited(reader) { + DeleteVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffPickerOptions message. + * Verifies a DeleteVReplicationWorkflowRequest message. * @function verify - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffPickerOptions.verify = function verify(message) { + DeleteVReplicationWorkflowRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - if (!$util.isString(message.tablet_types)) - return "tablet_types: string expected"; - if (message.source_cell != null && message.hasOwnProperty("source_cell")) - if (!$util.isString(message.source_cell)) - return "source_cell: string expected"; - if (message.target_cell != null && message.hasOwnProperty("target_cell")) - if (!$util.isString(message.target_cell)) - return "target_cell: string expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; return null; }; /** - * Creates a VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowRequest} DeleteVReplicationWorkflowRequest */ - VDiffPickerOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffPickerOptions) + DeleteVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.DeleteVReplicationWorkflowRequest) return object; - let message = new $root.tabletmanagerdata.VDiffPickerOptions(); - if (object.tablet_types != null) - message.tablet_types = String(object.tablet_types); - if (object.source_cell != null) - message.source_cell = String(object.source_cell); - if (object.target_cell != null) - message.target_cell = String(object.target_cell); + let message = new $root.tabletmanagerdata.DeleteVReplicationWorkflowRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); return message; }; /** - * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. + * Creates a plain object from a DeleteVReplicationWorkflowRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.VDiffPickerOptions} message VDiffPickerOptions + * @param {tabletmanagerdata.DeleteVReplicationWorkflowRequest} message DeleteVReplicationWorkflowRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffPickerOptions.toObject = function toObject(message, options) { + DeleteVReplicationWorkflowRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.tablet_types = ""; - object.source_cell = ""; - object.target_cell = ""; - } - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - object.tablet_types = message.tablet_types; - if (message.source_cell != null && message.hasOwnProperty("source_cell")) - object.source_cell = message.source_cell; - if (message.target_cell != null && message.hasOwnProperty("target_cell")) - object.target_cell = message.target_cell; + if (options.defaults) + object.workflow = ""; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; return object; }; /** - * Converts this VDiffPickerOptions to JSON. + * Converts this DeleteVReplicationWorkflowRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @instance * @returns {Object.} JSON object */ - VDiffPickerOptions.prototype.toJSON = function toJSON() { + DeleteVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffPickerOptions + * Gets the default type url for DeleteVReplicationWorkflowRequest * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffPickerOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffPickerOptions"; + return typeUrlPrefix + "/tabletmanagerdata.DeleteVReplicationWorkflowRequest"; }; - return VDiffPickerOptions; + return DeleteVReplicationWorkflowRequest; })(); - tabletmanagerdata.VDiffReportOptions = (function() { + tabletmanagerdata.DeleteVReplicationWorkflowResponse = (function() { /** - * Properties of a VDiffReportOptions. + * Properties of a DeleteVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @interface IVDiffReportOptions - * @property {boolean|null} [only_pks] VDiffReportOptions only_pks - * @property {boolean|null} [debug_query] VDiffReportOptions debug_query - * @property {string|null} [format] VDiffReportOptions format + * @interface IDeleteVReplicationWorkflowResponse + * @property {query.IQueryResult|null} [result] DeleteVReplicationWorkflowResponse result */ /** - * Constructs a new VDiffReportOptions. + * Constructs a new DeleteVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffReportOptions. - * @implements IVDiffReportOptions + * @classdesc Represents a DeleteVReplicationWorkflowResponse. + * @implements IDeleteVReplicationWorkflowResponse * @constructor - * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowResponse=} [properties] Properties to set */ - function VDiffReportOptions(properties) { + function DeleteVReplicationWorkflowResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -61080,103 +61282,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffReportOptions only_pks. - * @member {boolean} only_pks - * @memberof tabletmanagerdata.VDiffReportOptions - * @instance - */ - VDiffReportOptions.prototype.only_pks = false; - - /** - * VDiffReportOptions debug_query. - * @member {boolean} debug_query - * @memberof tabletmanagerdata.VDiffReportOptions - * @instance - */ - VDiffReportOptions.prototype.debug_query = false; - - /** - * VDiffReportOptions format. - * @member {string} format - * @memberof tabletmanagerdata.VDiffReportOptions + * DeleteVReplicationWorkflowResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @instance */ - VDiffReportOptions.prototype.format = ""; + DeleteVReplicationWorkflowResponse.prototype.result = null; /** - * Creates a new VDiffReportOptions instance using the specified properties. + * Creates a new DeleteVReplicationWorkflowResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions instance + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowResponse} DeleteVReplicationWorkflowResponse instance */ - VDiffReportOptions.create = function create(properties) { - return new VDiffReportOptions(properties); + DeleteVReplicationWorkflowResponse.create = function create(properties) { + return new DeleteVReplicationWorkflowResponse(properties); }; /** - * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * Encodes the specified DeleteVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowResponse} message DeleteVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffReportOptions.encode = function encode(message, writer) { + DeleteVReplicationWorkflowResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.only_pks != null && Object.hasOwnProperty.call(message, "only_pks")) - writer.uint32(/* id 1, wireType 0 =*/8).bool(message.only_pks); - if (message.debug_query != null && Object.hasOwnProperty.call(message, "debug_query")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.debug_query); - if (message.format != null && Object.hasOwnProperty.call(message, "format")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.format); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * Encodes the specified DeleteVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.DeleteVReplicationWorkflowResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode + * @param {tabletmanagerdata.IDeleteVReplicationWorkflowResponse} message DeleteVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffReportOptions.encodeDelimited = function encodeDelimited(message, writer) { + DeleteVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer. + * Decodes a DeleteVReplicationWorkflowResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowResponse} DeleteVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffReportOptions.decode = function decode(reader, length) { + DeleteVReplicationWorkflowResponse.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.tabletmanagerdata.VDiffReportOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.DeleteVReplicationWorkflowResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.only_pks = reader.bool(); - break; - } - case 2: { - message.debug_query = reader.bool(); - break; - } - case 3: { - message.format = reader.string(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -61188,146 +61362,127 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. + * Decodes a DeleteVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowResponse} DeleteVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffReportOptions.decodeDelimited = function decodeDelimited(reader) { + DeleteVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffReportOptions message. + * Verifies a DeleteVReplicationWorkflowResponse message. * @function verify - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffReportOptions.verify = function verify(message) { + DeleteVReplicationWorkflowResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.only_pks != null && message.hasOwnProperty("only_pks")) - if (typeof message.only_pks !== "boolean") - return "only_pks: boolean expected"; - if (message.debug_query != null && message.hasOwnProperty("debug_query")) - if (typeof message.debug_query !== "boolean") - return "debug_query: boolean expected"; - if (message.format != null && message.hasOwnProperty("format")) - if (!$util.isString(message.format)) - return "format: string expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } return null; }; /** - * Creates a VDiffReportOptions message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.DeleteVReplicationWorkflowResponse} DeleteVReplicationWorkflowResponse */ - VDiffReportOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffReportOptions) + DeleteVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.DeleteVReplicationWorkflowResponse) return object; - let message = new $root.tabletmanagerdata.VDiffReportOptions(); - if (object.only_pks != null) - message.only_pks = Boolean(object.only_pks); - if (object.debug_query != null) - message.debug_query = Boolean(object.debug_query); - if (object.format != null) - message.format = String(object.format); + let message = new $root.tabletmanagerdata.DeleteVReplicationWorkflowResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".tabletmanagerdata.DeleteVReplicationWorkflowResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } return message; }; /** - * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. + * Creates a plain object from a DeleteVReplicationWorkflowResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.VDiffReportOptions} message VDiffReportOptions + * @param {tabletmanagerdata.DeleteVReplicationWorkflowResponse} message DeleteVReplicationWorkflowResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffReportOptions.toObject = function toObject(message, options) { + DeleteVReplicationWorkflowResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.only_pks = false; - object.debug_query = false; - object.format = ""; - } - if (message.only_pks != null && message.hasOwnProperty("only_pks")) - object.only_pks = message.only_pks; - if (message.debug_query != null && message.hasOwnProperty("debug_query")) - object.debug_query = message.debug_query; - if (message.format != null && message.hasOwnProperty("format")) - object.format = message.format; + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this VDiffReportOptions to JSON. + * Converts this DeleteVReplicationWorkflowResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @instance * @returns {Object.} JSON object */ - VDiffReportOptions.prototype.toJSON = function toJSON() { + DeleteVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffReportOptions + * Gets the default type url for DeleteVReplicationWorkflowResponse * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.DeleteVReplicationWorkflowResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffReportOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffReportOptions"; + return typeUrlPrefix + "/tabletmanagerdata.DeleteVReplicationWorkflowResponse"; }; - return VDiffReportOptions; + return DeleteVReplicationWorkflowResponse; })(); - tabletmanagerdata.VDiffCoreOptions = (function() { + tabletmanagerdata.ReadVReplicationWorkflowRequest = (function() { /** - * Properties of a VDiffCoreOptions. + * Properties of a ReadVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @interface IVDiffCoreOptions - * @property {string|null} [tables] VDiffCoreOptions tables - * @property {boolean|null} [auto_retry] VDiffCoreOptions auto_retry - * @property {number|Long|null} [max_rows] VDiffCoreOptions max_rows - * @property {boolean|null} [checksum] VDiffCoreOptions checksum - * @property {number|Long|null} [sample_pct] VDiffCoreOptions sample_pct - * @property {number|Long|null} [timeout_seconds] VDiffCoreOptions timeout_seconds - * @property {number|Long|null} [max_extra_rows_to_compare] VDiffCoreOptions max_extra_rows_to_compare - * @property {boolean|null} [update_table_stats] VDiffCoreOptions update_table_stats + * @interface IReadVReplicationWorkflowRequest + * @property {string|null} [workflow] ReadVReplicationWorkflowRequest workflow */ /** - * Constructs a new VDiffCoreOptions. + * Constructs a new ReadVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffCoreOptions. - * @implements IVDiffCoreOptions + * @classdesc Represents a ReadVReplicationWorkflowRequest. + * @implements IReadVReplicationWorkflowRequest * @constructor - * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set */ - function VDiffCoreOptions(properties) { + function ReadVReplicationWorkflowRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -61335,173 +61490,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffCoreOptions tables. - * @member {string} tables - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.tables = ""; - - /** - * VDiffCoreOptions auto_retry. - * @member {boolean} auto_retry - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.auto_retry = false; - - /** - * VDiffCoreOptions max_rows. - * @member {number|Long} max_rows - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * VDiffCoreOptions checksum. - * @member {boolean} checksum - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.checksum = false; - - /** - * VDiffCoreOptions sample_pct. - * @member {number|Long} sample_pct - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.sample_pct = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * VDiffCoreOptions timeout_seconds. - * @member {number|Long} timeout_seconds - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.timeout_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * VDiffCoreOptions max_extra_rows_to_compare. - * @member {number|Long} max_extra_rows_to_compare - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.max_extra_rows_to_compare = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * VDiffCoreOptions update_table_stats. - * @member {boolean} update_table_stats - * @memberof tabletmanagerdata.VDiffCoreOptions + * ReadVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.update_table_stats = false; + ReadVReplicationWorkflowRequest.prototype.workflow = ""; /** - * Creates a new VDiffCoreOptions instance using the specified properties. + * Creates a new ReadVReplicationWorkflowRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions instance + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest instance */ - VDiffCoreOptions.create = function create(properties) { - return new VDiffCoreOptions(properties); + ReadVReplicationWorkflowRequest.create = function create(properties) { + return new ReadVReplicationWorkflowRequest(properties); }; /** - * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * Encodes the specified ReadVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffCoreOptions.encode = function encode(message, writer) { + ReadVReplicationWorkflowRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables); - if (message.auto_retry != null && Object.hasOwnProperty.call(message, "auto_retry")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.auto_retry); - if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); - if (message.checksum != null && Object.hasOwnProperty.call(message, "checksum")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.checksum); - if (message.sample_pct != null && Object.hasOwnProperty.call(message, "sample_pct")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.sample_pct); - if (message.timeout_seconds != null && Object.hasOwnProperty.call(message, "timeout_seconds")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.timeout_seconds); - if (message.max_extra_rows_to_compare != null && Object.hasOwnProperty.call(message, "max_extra_rows_to_compare")) - writer.uint32(/* id 7, wireType 0 =*/56).int64(message.max_extra_rows_to_compare); - if (message.update_table_stats != null && Object.hasOwnProperty.call(message, "update_table_stats")) - writer.uint32(/* id 8, wireType 0 =*/64).bool(message.update_table_stats); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); return writer; }; /** - * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * Encodes the specified ReadVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffCoreOptions.encodeDelimited = function encodeDelimited(message, writer) { + ReadVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer. + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffCoreOptions.decode = function decode(reader, length) { + ReadVReplicationWorkflowRequest.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.tabletmanagerdata.VDiffCoreOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tables = reader.string(); - break; - } - case 2: { - message.auto_retry = reader.bool(); - break; - } - case 3: { - message.max_rows = reader.int64(); - break; - } - case 4: { - message.checksum = reader.bool(); - break; - } - case 5: { - message.sample_pct = reader.int64(); - break; - } - case 6: { - message.timeout_seconds = reader.int64(); - break; - } - case 7: { - message.max_extra_rows_to_compare = reader.int64(); - break; - } - case 8: { - message.update_table_stats = reader.bool(); + message.workflow = reader.string(); break; } default: @@ -61513,237 +61570,133 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffCoreOptions.decodeDelimited = function decodeDelimited(reader) { + ReadVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffCoreOptions message. + * Verifies a ReadVReplicationWorkflowRequest message. * @function verify - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffCoreOptions.verify = function verify(message) { + ReadVReplicationWorkflowRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tables != null && message.hasOwnProperty("tables")) - if (!$util.isString(message.tables)) - return "tables: string expected"; - if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) - if (typeof message.auto_retry !== "boolean") - return "auto_retry: boolean expected"; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) - return "max_rows: integer|Long expected"; - if (message.checksum != null && message.hasOwnProperty("checksum")) - if (typeof message.checksum !== "boolean") - return "checksum: boolean expected"; - if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) - if (!$util.isInteger(message.sample_pct) && !(message.sample_pct && $util.isInteger(message.sample_pct.low) && $util.isInteger(message.sample_pct.high))) - return "sample_pct: integer|Long expected"; - if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) - if (!$util.isInteger(message.timeout_seconds) && !(message.timeout_seconds && $util.isInteger(message.timeout_seconds.low) && $util.isInteger(message.timeout_seconds.high))) - return "timeout_seconds: integer|Long expected"; - if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) - if (!$util.isInteger(message.max_extra_rows_to_compare) && !(message.max_extra_rows_to_compare && $util.isInteger(message.max_extra_rows_to_compare.low) && $util.isInteger(message.max_extra_rows_to_compare.high))) - return "max_extra_rows_to_compare: integer|Long expected"; - if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) - if (typeof message.update_table_stats !== "boolean") - return "update_table_stats: boolean expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; return null; }; /** - * Creates a VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. + * Creates a ReadVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest */ - VDiffCoreOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffCoreOptions) + ReadVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowRequest) return object; - let message = new $root.tabletmanagerdata.VDiffCoreOptions(); - if (object.tables != null) - message.tables = String(object.tables); - if (object.auto_retry != null) - message.auto_retry = Boolean(object.auto_retry); - if (object.max_rows != null) - if ($util.Long) - (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; - else if (typeof object.max_rows === "string") - message.max_rows = parseInt(object.max_rows, 10); - else if (typeof object.max_rows === "number") - message.max_rows = object.max_rows; - else if (typeof object.max_rows === "object") - message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); - if (object.checksum != null) - message.checksum = Boolean(object.checksum); - if (object.sample_pct != null) - if ($util.Long) - (message.sample_pct = $util.Long.fromValue(object.sample_pct)).unsigned = false; - else if (typeof object.sample_pct === "string") - message.sample_pct = parseInt(object.sample_pct, 10); - else if (typeof object.sample_pct === "number") - message.sample_pct = object.sample_pct; - else if (typeof object.sample_pct === "object") - message.sample_pct = new $util.LongBits(object.sample_pct.low >>> 0, object.sample_pct.high >>> 0).toNumber(); - if (object.timeout_seconds != null) - if ($util.Long) - (message.timeout_seconds = $util.Long.fromValue(object.timeout_seconds)).unsigned = false; - else if (typeof object.timeout_seconds === "string") - message.timeout_seconds = parseInt(object.timeout_seconds, 10); - else if (typeof object.timeout_seconds === "number") - message.timeout_seconds = object.timeout_seconds; - else if (typeof object.timeout_seconds === "object") - message.timeout_seconds = new $util.LongBits(object.timeout_seconds.low >>> 0, object.timeout_seconds.high >>> 0).toNumber(); - if (object.max_extra_rows_to_compare != null) - if ($util.Long) - (message.max_extra_rows_to_compare = $util.Long.fromValue(object.max_extra_rows_to_compare)).unsigned = false; - else if (typeof object.max_extra_rows_to_compare === "string") - message.max_extra_rows_to_compare = parseInt(object.max_extra_rows_to_compare, 10); - else if (typeof object.max_extra_rows_to_compare === "number") - message.max_extra_rows_to_compare = object.max_extra_rows_to_compare; - else if (typeof object.max_extra_rows_to_compare === "object") - message.max_extra_rows_to_compare = new $util.LongBits(object.max_extra_rows_to_compare.low >>> 0, object.max_extra_rows_to_compare.high >>> 0).toNumber(); - if (object.update_table_stats != null) - message.update_table_stats = Boolean(object.update_table_stats); + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); return message; }; /** - * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. + * Creates a plain object from a ReadVReplicationWorkflowRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.VDiffCoreOptions} message VDiffCoreOptions + * @param {tabletmanagerdata.ReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffCoreOptions.toObject = function toObject(message, options) { + ReadVReplicationWorkflowRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.tables = ""; - object.auto_retry = false; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_rows = options.longs === String ? "0" : 0; - object.checksum = false; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.sample_pct = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.sample_pct = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.timeout_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.timeout_seconds = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_extra_rows_to_compare = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_extra_rows_to_compare = options.longs === String ? "0" : 0; - object.update_table_stats = false; - } - if (message.tables != null && message.hasOwnProperty("tables")) - object.tables = message.tables; - if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) - object.auto_retry = message.auto_retry; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (typeof message.max_rows === "number") - object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; - else - object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; - if (message.checksum != null && message.hasOwnProperty("checksum")) - object.checksum = message.checksum; - if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) - if (typeof message.sample_pct === "number") - object.sample_pct = options.longs === String ? String(message.sample_pct) : message.sample_pct; - else - object.sample_pct = options.longs === String ? $util.Long.prototype.toString.call(message.sample_pct) : options.longs === Number ? new $util.LongBits(message.sample_pct.low >>> 0, message.sample_pct.high >>> 0).toNumber() : message.sample_pct; - if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) - if (typeof message.timeout_seconds === "number") - object.timeout_seconds = options.longs === String ? String(message.timeout_seconds) : message.timeout_seconds; - else - object.timeout_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.timeout_seconds) : options.longs === Number ? new $util.LongBits(message.timeout_seconds.low >>> 0, message.timeout_seconds.high >>> 0).toNumber() : message.timeout_seconds; - if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) - if (typeof message.max_extra_rows_to_compare === "number") - object.max_extra_rows_to_compare = options.longs === String ? String(message.max_extra_rows_to_compare) : message.max_extra_rows_to_compare; - else - object.max_extra_rows_to_compare = options.longs === String ? $util.Long.prototype.toString.call(message.max_extra_rows_to_compare) : options.longs === Number ? new $util.LongBits(message.max_extra_rows_to_compare.low >>> 0, message.max_extra_rows_to_compare.high >>> 0).toNumber() : message.max_extra_rows_to_compare; - if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) - object.update_table_stats = message.update_table_stats; + if (options.defaults) + object.workflow = ""; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; return object; }; /** - * Converts this VDiffCoreOptions to JSON. + * Converts this ReadVReplicationWorkflowRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @instance * @returns {Object.} JSON object */ - VDiffCoreOptions.prototype.toJSON = function toJSON() { + ReadVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffCoreOptions + * Gets the default type url for ReadVReplicationWorkflowRequest * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffCoreOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReadVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffCoreOptions"; + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowRequest"; }; - return VDiffCoreOptions; + return ReadVReplicationWorkflowRequest; })(); - tabletmanagerdata.VDiffOptions = (function() { + tabletmanagerdata.ReadVReplicationWorkflowResponse = (function() { /** - * Properties of a VDiffOptions. + * Properties of a ReadVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @interface IVDiffOptions - * @property {tabletmanagerdata.IVDiffPickerOptions|null} [picker_options] VDiffOptions picker_options - * @property {tabletmanagerdata.IVDiffCoreOptions|null} [core_options] VDiffOptions core_options - * @property {tabletmanagerdata.IVDiffReportOptions|null} [report_options] VDiffOptions report_options + * @interface IReadVReplicationWorkflowResponse + * @property {string|null} [workflow] ReadVReplicationWorkflowResponse workflow + * @property {string|null} [cells] ReadVReplicationWorkflowResponse cells + * @property {Array.|null} [tablet_types] ReadVReplicationWorkflowResponse tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] ReadVReplicationWorkflowResponse tablet_selection_preference + * @property {string|null} [db_name] ReadVReplicationWorkflowResponse db_name + * @property {string|null} [tags] ReadVReplicationWorkflowResponse tags + * @property {binlogdata.VReplicationWorkflowType|null} [workflow_type] ReadVReplicationWorkflowResponse workflow_type + * @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 */ /** - * Constructs a new VDiffOptions. + * Constructs a new ReadVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffOptions. - * @implements IVDiffOptions + * @classdesc Represents a ReadVReplicationWorkflowResponse. + * @implements IReadVReplicationWorkflowResponse * @constructor - * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set */ - function VDiffOptions(properties) { + function ReadVReplicationWorkflowResponse(properties) { + this.tablet_types = []; + this.streams = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -61751,391 +61704,215 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffOptions picker_options. - * @member {tabletmanagerdata.IVDiffPickerOptions|null|undefined} picker_options - * @memberof tabletmanagerdata.VDiffOptions + * ReadVReplicationWorkflowResponse workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - VDiffOptions.prototype.picker_options = null; + ReadVReplicationWorkflowResponse.prototype.workflow = ""; /** - * VDiffOptions core_options. - * @member {tabletmanagerdata.IVDiffCoreOptions|null|undefined} core_options - * @memberof tabletmanagerdata.VDiffOptions + * ReadVReplicationWorkflowResponse cells. + * @member {string} cells + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - VDiffOptions.prototype.core_options = null; + ReadVReplicationWorkflowResponse.prototype.cells = ""; /** - * VDiffOptions report_options. - * @member {tabletmanagerdata.IVDiffReportOptions|null|undefined} report_options - * @memberof tabletmanagerdata.VDiffOptions + * ReadVReplicationWorkflowResponse tablet_types. + * @member {Array.} tablet_types + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - VDiffOptions.prototype.report_options = null; - - /** - * Creates a new VDiffOptions instance using the specified properties. - * @function create - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions instance - */ - VDiffOptions.create = function create(properties) { - return new VDiffOptions(properties); - }; - - /** - * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. - * @function encode - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - VDiffOptions.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.picker_options != null && Object.hasOwnProperty.call(message, "picker_options")) - $root.tabletmanagerdata.VDiffPickerOptions.encode(message.picker_options, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.core_options != null && Object.hasOwnProperty.call(message, "core_options")) - $root.tabletmanagerdata.VDiffCoreOptions.encode(message.core_options, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.report_options != null && Object.hasOwnProperty.call(message, "report_options")) - $root.tabletmanagerdata.VDiffReportOptions.encode(message.report_options, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - return writer; - }; - - /** - * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. - * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - VDiffOptions.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a VDiffOptions message from the specified reader or buffer. - * @function decode - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - VDiffOptions.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.tabletmanagerdata.VDiffOptions(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.decode(reader, reader.uint32()); - break; - } - case 2: { - message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.decode(reader, reader.uint32()); - break; - } - case 3: { - message.report_options = $root.tabletmanagerdata.VDiffReportOptions.decode(reader, reader.uint32()); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - VDiffOptions.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a VDiffOptions message. - * @function verify - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - VDiffOptions.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.picker_options != null && message.hasOwnProperty("picker_options")) { - let error = $root.tabletmanagerdata.VDiffPickerOptions.verify(message.picker_options); - if (error) - return "picker_options." + error; - } - if (message.core_options != null && message.hasOwnProperty("core_options")) { - let error = $root.tabletmanagerdata.VDiffCoreOptions.verify(message.core_options); - if (error) - return "core_options." + error; - } - if (message.report_options != null && message.hasOwnProperty("report_options")) { - let error = $root.tabletmanagerdata.VDiffReportOptions.verify(message.report_options); - if (error) - return "report_options." + error; - } - return null; - }; - - /** - * Creates a VDiffOptions message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions - */ - VDiffOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffOptions) - return object; - let message = new $root.tabletmanagerdata.VDiffOptions(); - if (object.picker_options != null) { - if (typeof object.picker_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.picker_options: object expected"); - message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.fromObject(object.picker_options); - } - if (object.core_options != null) { - if (typeof object.core_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.core_options: object expected"); - message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.fromObject(object.core_options); - } - if (object.report_options != null) { - if (typeof object.report_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.report_options: object expected"); - message.report_options = $root.tabletmanagerdata.VDiffReportOptions.fromObject(object.report_options); - } - return message; - }; - - /** - * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. - * @function toObject - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {tabletmanagerdata.VDiffOptions} message VDiffOptions - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - VDiffOptions.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.picker_options = null; - object.core_options = null; - object.report_options = null; - } - if (message.picker_options != null && message.hasOwnProperty("picker_options")) - object.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.toObject(message.picker_options, options); - if (message.core_options != null && message.hasOwnProperty("core_options")) - object.core_options = $root.tabletmanagerdata.VDiffCoreOptions.toObject(message.core_options, options); - if (message.report_options != null && message.hasOwnProperty("report_options")) - object.report_options = $root.tabletmanagerdata.VDiffReportOptions.toObject(message.report_options, options); - return object; - }; + ReadVReplicationWorkflowResponse.prototype.tablet_types = $util.emptyArray; /** - * Converts this VDiffOptions to JSON. - * @function toJSON - * @memberof tabletmanagerdata.VDiffOptions + * ReadVReplicationWorkflowResponse tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance - * @returns {Object.} JSON object - */ - VDiffOptions.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for VDiffOptions - * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffOptions - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url */ - VDiffOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/tabletmanagerdata.VDiffOptions"; - }; - - return VDiffOptions; - })(); - - tabletmanagerdata.UpdateVRWorkflowRequest = (function() { + ReadVReplicationWorkflowResponse.prototype.tablet_selection_preference = 0; /** - * Properties of an UpdateVRWorkflowRequest. - * @memberof tabletmanagerdata - * @interface IUpdateVRWorkflowRequest - * @property {string|null} [workflow] UpdateVRWorkflowRequest workflow - * @property {Array.|null} [cells] UpdateVRWorkflowRequest cells - * @property {Array.|null} [tablet_types] UpdateVRWorkflowRequest tablet_types - * @property {binlogdata.OnDDLAction|null} [on_ddl] UpdateVRWorkflowRequest on_ddl + * ReadVReplicationWorkflowResponse db_name. + * @member {string} db_name + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance */ + ReadVReplicationWorkflowResponse.prototype.db_name = ""; /** - * Constructs a new UpdateVRWorkflowRequest. - * @memberof tabletmanagerdata - * @classdesc Represents an UpdateVRWorkflowRequest. - * @implements IUpdateVRWorkflowRequest - * @constructor - * @param {tabletmanagerdata.IUpdateVRWorkflowRequest=} [properties] Properties to set + * ReadVReplicationWorkflowResponse tags. + * @member {string} tags + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance */ - function UpdateVRWorkflowRequest(properties) { - this.cells = []; - this.tablet_types = []; - 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]]; - } + ReadVReplicationWorkflowResponse.prototype.tags = ""; /** - * UpdateVRWorkflowRequest workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * ReadVReplicationWorkflowResponse workflow_type. + * @member {binlogdata.VReplicationWorkflowType} workflow_type + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - UpdateVRWorkflowRequest.prototype.workflow = ""; + ReadVReplicationWorkflowResponse.prototype.workflow_type = 0; /** - * UpdateVRWorkflowRequest cells. - * @member {Array.} cells - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * ReadVReplicationWorkflowResponse workflow_sub_type. + * @member {binlogdata.VReplicationWorkflowSubType} workflow_sub_type + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - UpdateVRWorkflowRequest.prototype.cells = $util.emptyArray; + ReadVReplicationWorkflowResponse.prototype.workflow_sub_type = 0; /** - * UpdateVRWorkflowRequest tablet_types. - * @member {Array.} tablet_types - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * ReadVReplicationWorkflowResponse defer_secondary_keys. + * @member {boolean} defer_secondary_keys + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - UpdateVRWorkflowRequest.prototype.tablet_types = $util.emptyArray; + ReadVReplicationWorkflowResponse.prototype.defer_secondary_keys = false; /** - * UpdateVRWorkflowRequest on_ddl. - * @member {binlogdata.OnDDLAction} on_ddl - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * ReadVReplicationWorkflowResponse streams. + * @member {Array.} streams + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance */ - UpdateVRWorkflowRequest.prototype.on_ddl = 0; + ReadVReplicationWorkflowResponse.prototype.streams = $util.emptyArray; /** - * Creates a new UpdateVRWorkflowRequest instance using the specified properties. + * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.UpdateVRWorkflowRequest} UpdateVRWorkflowRequest instance + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse instance */ - UpdateVRWorkflowRequest.create = function create(properties) { - return new UpdateVRWorkflowRequest(properties); + ReadVReplicationWorkflowResponse.create = function create(properties) { + return new ReadVReplicationWorkflowResponse(properties); }; /** - * Encodes the specified UpdateVRWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowRequest.verify|verify} messages. + * Encodes the specified ReadVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowRequest} message UpdateVRWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVRWorkflowRequest.encode = function encode(message, writer) { + ReadVReplicationWorkflowResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); - if (message.tablet_types != null && message.tablet_types.length) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.cells != null && Object.hasOwnProperty.call(message, "cells")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 4, wireType 2 =*/34).fork(); for (let i = 0; i < message.tablet_types.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.tablet_types[i]); - if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) - writer.uint32(/* id 4, wireType 0 =*/32).int32(message.on_ddl); + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 5, wireType 0 =*/40).int32(message.tablet_selection_preference); + if (message.db_name != null && Object.hasOwnProperty.call(message, "db_name")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.db_name); + if (message.tags != null && Object.hasOwnProperty.call(message, "tags")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.tags); + if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) + writer.uint32(/* id 8, wireType 0 =*/64).int32(message.workflow_type); + if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.workflow_sub_type); + if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) + writer.uint32(/* id 10, wireType 0 =*/80).bool(message.defer_secondary_keys); + 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(); return writer; }; /** - * Encodes the specified UpdateVRWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowRequest.verify|verify} messages. + * Encodes the specified ReadVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowRequest} message UpdateVRWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVRWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReadVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateVRWorkflowRequest message from the specified reader or buffer. + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.UpdateVRWorkflowRequest} UpdateVRWorkflowRequest + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVRWorkflowRequest.decode = function decode(reader, length) { + ReadVReplicationWorkflowResponse.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.tabletmanagerdata.UpdateVRWorkflowRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { + case 2: { message.workflow = reader.string(); break; } - case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + case 3: { + message.cells = reader.string(); break; } - case 3: { + case 4: { if (!(message.tablet_types && message.tablet_types.length)) message.tablet_types = []; - message.tablet_types.push(reader.string()); + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); break; } - case 4: { - message.on_ddl = reader.int32(); + case 5: { + message.tablet_selection_preference = reader.int32(); + break; + } + case 6: { + message.db_name = reader.string(); + break; + } + case 7: { + message.tags = reader.string(); + break; + } + case 8: { + message.workflow_type = reader.int32(); + break; + } + case 9: { + message.workflow_sub_type = reader.int32(); + break; + } + case 10: { + message.defer_secondary_keys = reader.bool(); + break; + } + case 11: { + if (!(message.streams && message.streams.length)) + message.streams = []; + message.streams.push($root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.decode(reader, reader.uint32())); break; } default: @@ -62147,278 +61924,1137 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes an UpdateVRWorkflowRequest message from the specified reader or buffer, length delimited. + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.UpdateVRWorkflowRequest} UpdateVRWorkflowRequest + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVRWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { + ReadVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateVRWorkflowRequest message. + * Verifies a ReadVReplicationWorkflowResponse message. * @function verify - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateVRWorkflowRequest.verify = function verify(message) { + ReadVReplicationWorkflowResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.workflow != null && message.hasOwnProperty("workflow")) if (!$util.isString(message.workflow)) return "workflow: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } + if (message.cells != null && message.hasOwnProperty("cells")) + if (!$util.isString(message.cells)) + return "cells: string expected"; if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { if (!Array.isArray(message.tablet_types)) return "tablet_types: array expected"; for (let i = 0; i < message.tablet_types.length; ++i) - if (!$util.isString(message.tablet_types[i])) - return "tablet_types: string[] expected"; + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - switch (message.on_ddl) { + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { default: - return "on_ddl: enum value expected"; + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + if (message.db_name != null && message.hasOwnProperty("db_name")) + if (!$util.isString(message.db_name)) + return "db_name: string expected"; + if (message.tags != null && message.hasOwnProperty("tags")) + if (!$util.isString(message.tags)) + return "tags: string expected"; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + switch (message.workflow_type) { + default: + return "workflow_type: enum value expected"; case 0: case 1: case 2: case 3: + case 4: + case 5: + break; + } + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + switch (message.workflow_sub_type) { + default: + return "workflow_sub_type: enum value expected"; + case 0: + case 1: break; } + 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.streams != null && message.hasOwnProperty("streams")) { + if (!Array.isArray(message.streams)) + return "streams: array expected"; + for (let i = 0; i < message.streams.length; ++i) { + let error = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify(message.streams[i]); + if (error) + return "streams." + error; + } + } return null; }; /** - * Creates an UpdateVRWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReadVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.UpdateVRWorkflowRequest} UpdateVRWorkflowRequest + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse */ - UpdateVRWorkflowRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.UpdateVRWorkflowRequest) + ReadVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse) return object; - let message = new $root.tabletmanagerdata.UpdateVRWorkflowRequest(); + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); if (object.workflow != null) message.workflow = String(object.workflow); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".tabletmanagerdata.UpdateVRWorkflowRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } + if (object.cells != null) + message.cells = String(object.cells); if (object.tablet_types) { if (!Array.isArray(object.tablet_types)) - throw TypeError(".tabletmanagerdata.UpdateVRWorkflowRequest.tablet_types: array expected"); + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types: array expected"); message.tablet_types = []; for (let i = 0; i < object.tablet_types.length; ++i) - message.tablet_types[i] = String(object.tablet_types[i]); + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } } - switch (object.on_ddl) { + switch (object.tablet_selection_preference) { default: - if (typeof object.on_ddl === "number") { - message.on_ddl = object.on_ddl; + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; break; } break; - case "IGNORE": + case "ANY": case 0: - message.on_ddl = 0; + message.tablet_selection_preference = 0; break; - case "STOP": + case "INORDER": case 1: - message.on_ddl = 1; + message.tablet_selection_preference = 1; break; - case "EXEC": + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + if (object.db_name != null) + message.db_name = String(object.db_name); + if (object.tags != null) + message.tags = String(object.tags); + switch (object.workflow_type) { + default: + if (typeof object.workflow_type === "number") { + message.workflow_type = object.workflow_type; + break; + } + break; + case "Materialize": + case 0: + message.workflow_type = 0; + break; + case "MoveTables": + case 1: + message.workflow_type = 1; + break; + case "CreateLookupIndex": case 2: - message.on_ddl = 2; + message.workflow_type = 2; break; - case "EXEC_IGNORE": + case "Migrate": case 3: - message.on_ddl = 3; + message.workflow_type = 3; + break; + case "Reshard": + case 4: + message.workflow_type = 4; + break; + case "OnlineDDL": + case 5: + message.workflow_type = 5; + break; + } + switch (object.workflow_sub_type) { + default: + if (typeof object.workflow_sub_type === "number") { + message.workflow_sub_type = object.workflow_sub_type; + break; + } + break; + case "None": + case 0: + message.workflow_sub_type = 0; + break; + case "Partial": + case 1: + message.workflow_sub_type = 1; break; } + if (object.defer_secondary_keys != null) + message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + if (object.streams) { + if (!Array.isArray(object.streams)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: array expected"); + message.streams = []; + for (let i = 0; i < object.streams.length; ++i) { + if (typeof object.streams[i] !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: object expected"); + message.streams[i] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.fromObject(object.streams[i]); + } + } return message; }; /** - * Creates a plain object from an UpdateVRWorkflowRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReadVReplicationWorkflowResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.UpdateVRWorkflowRequest} message UpdateVRWorkflowRequest + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateVRWorkflowRequest.toObject = function toObject(message, options) { + ReadVReplicationWorkflowResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) { - object.cells = []; object.tablet_types = []; + object.streams = []; } if (options.defaults) { object.workflow = ""; - object.on_ddl = options.enums === String ? "IGNORE" : 0; + object.cells = ""; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.db_name = ""; + object.tags = ""; + object.workflow_type = options.enums === String ? "Materialize" : 0; + object.workflow_sub_type = options.enums === String ? "None" : 0; + object.defer_secondary_keys = false; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } + if (message.cells != null && message.hasOwnProperty("cells")) + object.cells = message.cells; if (message.tablet_types && message.tablet_types.length) { object.tablet_types = []; for (let j = 0; j < message.tablet_types.length; ++j) - object.tablet_types[j] = message.tablet_types[j]; + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.db_name != null && message.hasOwnProperty("db_name")) + object.db_name = message.db_name; + if (message.tags != null && message.hasOwnProperty("tags")) + object.tags = message.tags; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + object.workflow_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowType[message.workflow_type] === undefined ? message.workflow_type : $root.binlogdata.VReplicationWorkflowType[message.workflow_type] : message.workflow_type; + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + object.workflow_sub_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] === undefined ? message.workflow_sub_type : $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] : message.workflow_sub_type; + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + object.defer_secondary_keys = message.defer_secondary_keys; + if (message.streams && message.streams.length) { + object.streams = []; + for (let j = 0; j < message.streams.length; ++j) + object.streams[j] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.toObject(message.streams[j], options); } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; return object; }; /** - * Converts this UpdateVRWorkflowRequest to JSON. + * Converts this ReadVReplicationWorkflowResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @instance * @returns {Object.} JSON object */ - UpdateVRWorkflowRequest.prototype.toJSON = function toJSON() { + ReadVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateVRWorkflowRequest + * Gets the default type url for ReadVReplicationWorkflowResponse * @function getTypeUrl - * @memberof tabletmanagerdata.UpdateVRWorkflowRequest + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateVRWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReadVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.UpdateVRWorkflowRequest"; + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse"; }; - return UpdateVRWorkflowRequest; - })(); - - tabletmanagerdata.UpdateVRWorkflowResponse = (function() { + ReadVReplicationWorkflowResponse.Stream = (function() { - /** - * Properties of an UpdateVRWorkflowResponse. - * @memberof tabletmanagerdata - * @interface IUpdateVRWorkflowResponse - * @property {query.IQueryResult|null} [result] UpdateVRWorkflowResponse result - */ + /** + * Properties of a Stream. + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @interface IStream + * @property {number|null} [id] Stream id + * @property {binlogdata.IBinlogSource|null} [bls] Stream bls + * @property {string|null} [pos] Stream pos + * @property {string|null} [stop_pos] Stream stop_pos + * @property {number|Long|null} [max_tps] Stream max_tps + * @property {number|Long|null} [max_replication_lag] Stream max_replication_lag + * @property {vttime.ITime|null} [time_updated] Stream time_updated + * @property {vttime.ITime|null} [transaction_timestamp] Stream transaction_timestamp + * @property {binlogdata.VReplicationWorkflowState|null} [state] Stream state + * @property {string|null} [message] Stream message + * @property {number|Long|null} [rows_copied] Stream rows_copied + * @property {vttime.ITime|null} [time_heartbeat] Stream time_heartbeat + * @property {vttime.ITime|null} [time_throttled] Stream time_throttled + * @property {string|null} [component_throttled] Stream component_throttled + */ - /** - * Constructs a new UpdateVRWorkflowResponse. - * @memberof tabletmanagerdata - * @classdesc Represents an UpdateVRWorkflowResponse. - * @implements IUpdateVRWorkflowResponse - * @constructor - * @param {tabletmanagerdata.IUpdateVRWorkflowResponse=} [properties] Properties to set - */ - function UpdateVRWorkflowResponse(properties) { - if (properties) - for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + /** + * Constructs a new Stream. + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @classdesc Represents a Stream. + * @implements IStream + * @constructor + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set + */ + function Stream(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]]; + } + + /** + * Stream id. + * @member {number} id + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.id = 0; + + /** + * Stream bls. + * @member {binlogdata.IBinlogSource|null|undefined} bls + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.bls = null; + + /** + * Stream pos. + * @member {string} pos + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.pos = ""; + + /** + * Stream stop_pos. + * @member {string} stop_pos + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.stop_pos = ""; + + /** + * Stream max_tps. + * @member {number|Long} max_tps + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.max_tps = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream max_replication_lag. + * @member {number|Long} max_replication_lag + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.max_replication_lag = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream time_updated. + * @member {vttime.ITime|null|undefined} time_updated + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_updated = null; + + /** + * Stream transaction_timestamp. + * @member {vttime.ITime|null|undefined} transaction_timestamp + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.transaction_timestamp = null; + + /** + * Stream state. + * @member {binlogdata.VReplicationWorkflowState} state + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.state = 0; + + /** + * Stream message. + * @member {string} message + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.message = ""; + + /** + * Stream rows_copied. + * @member {number|Long} rows_copied + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.rows_copied = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream time_heartbeat. + * @member {vttime.ITime|null|undefined} time_heartbeat + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_heartbeat = null; + + /** + * Stream time_throttled. + * @member {vttime.ITime|null|undefined} time_throttled + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_throttled = null; + + /** + * Stream component_throttled. + * @member {string} component_throttled + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.component_throttled = ""; + + /** + * Creates a new Stream instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream instance + */ + Stream.create = function create(properties) { + return new Stream(properties); + }; + + /** + * Encodes the specified Stream message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.id); + if (message.bls != null && Object.hasOwnProperty.call(message, "bls")) + $root.binlogdata.BinlogSource.encode(message.bls, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.pos != null && Object.hasOwnProperty.call(message, "pos")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.pos); + if (message.stop_pos != null && Object.hasOwnProperty.call(message, "stop_pos")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.stop_pos); + if (message.max_tps != null && Object.hasOwnProperty.call(message, "max_tps")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.max_tps); + if (message.max_replication_lag != null && Object.hasOwnProperty.call(message, "max_replication_lag")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.max_replication_lag); + if (message.time_updated != null && Object.hasOwnProperty.call(message, "time_updated")) + $root.vttime.Time.encode(message.time_updated, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); + if (message.transaction_timestamp != null && Object.hasOwnProperty.call(message, "transaction_timestamp")) + $root.vttime.Time.encode(message.transaction_timestamp, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.state); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.message); + if (message.rows_copied != null && Object.hasOwnProperty.call(message, "rows_copied")) + writer.uint32(/* id 11, wireType 0 =*/88).int64(message.rows_copied); + if (message.time_heartbeat != null && Object.hasOwnProperty.call(message, "time_heartbeat")) + $root.vttime.Time.encode(message.time_heartbeat, writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim(); + if (message.time_throttled != null && Object.hasOwnProperty.call(message, "time_throttled")) + $root.vttime.Time.encode(message.time_throttled, writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim(); + if (message.component_throttled != null && Object.hasOwnProperty.call(message, "component_throttled")) + writer.uint32(/* id 14, wireType 2 =*/114).string(message.component_throttled); + return writer; + }; + + /** + * Encodes the specified Stream message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Stream message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Stream.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.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int32(); + break; + } + case 2: { + message.bls = $root.binlogdata.BinlogSource.decode(reader, reader.uint32()); + break; + } + case 3: { + message.pos = reader.string(); + break; + } + case 4: { + message.stop_pos = reader.string(); + break; + } + case 5: { + message.max_tps = reader.int64(); + break; + } + case 6: { + message.max_replication_lag = reader.int64(); + break; + } + case 7: { + message.time_updated = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 8: { + message.transaction_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 9: { + message.state = reader.int32(); + break; + } + case 10: { + message.message = reader.string(); + break; + } + case 11: { + message.rows_copied = reader.int64(); + break; + } + case 12: { + message.time_heartbeat = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 13: { + message.time_throttled = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 14: { + message.component_throttled = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Stream message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Stream.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Stream message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Stream.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id)) + return "id: integer expected"; + if (message.bls != null && message.hasOwnProperty("bls")) { + let error = $root.binlogdata.BinlogSource.verify(message.bls); + if (error) + return "bls." + error; + } + if (message.pos != null && message.hasOwnProperty("pos")) + if (!$util.isString(message.pos)) + return "pos: string expected"; + if (message.stop_pos != null && message.hasOwnProperty("stop_pos")) + if (!$util.isString(message.stop_pos)) + return "stop_pos: string expected"; + if (message.max_tps != null && message.hasOwnProperty("max_tps")) + if (!$util.isInteger(message.max_tps) && !(message.max_tps && $util.isInteger(message.max_tps.low) && $util.isInteger(message.max_tps.high))) + return "max_tps: integer|Long expected"; + if (message.max_replication_lag != null && message.hasOwnProperty("max_replication_lag")) + if (!$util.isInteger(message.max_replication_lag) && !(message.max_replication_lag && $util.isInteger(message.max_replication_lag.low) && $util.isInteger(message.max_replication_lag.high))) + return "max_replication_lag: integer|Long expected"; + if (message.time_updated != null && message.hasOwnProperty("time_updated")) { + let error = $root.vttime.Time.verify(message.time_updated); + if (error) + return "time_updated." + error; + } + if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) { + let error = $root.vttime.Time.verify(message.transaction_timestamp); + if (error) + return "transaction_timestamp." + error; + } + if (message.state != null && message.hasOwnProperty("state")) + switch (message.state) { + default: + return "state: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) + if (!$util.isInteger(message.rows_copied) && !(message.rows_copied && $util.isInteger(message.rows_copied.low) && $util.isInteger(message.rows_copied.high))) + return "rows_copied: integer|Long expected"; + if (message.time_heartbeat != null && message.hasOwnProperty("time_heartbeat")) { + let error = $root.vttime.Time.verify(message.time_heartbeat); + if (error) + return "time_heartbeat." + error; + } + if (message.time_throttled != null && message.hasOwnProperty("time_throttled")) { + let error = $root.vttime.Time.verify(message.time_throttled); + if (error) + return "time_throttled." + error; + } + if (message.component_throttled != null && message.hasOwnProperty("component_throttled")) + if (!$util.isString(message.component_throttled)) + return "component_throttled: string expected"; + return null; + }; + + /** + * Creates a Stream message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + */ + Stream.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); + if (object.id != null) + message.id = object.id | 0; + if (object.bls != null) { + if (typeof object.bls !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls: object expected"); + message.bls = $root.binlogdata.BinlogSource.fromObject(object.bls); + } + if (object.pos != null) + message.pos = String(object.pos); + if (object.stop_pos != null) + message.stop_pos = String(object.stop_pos); + if (object.max_tps != null) + if ($util.Long) + (message.max_tps = $util.Long.fromValue(object.max_tps)).unsigned = false; + else if (typeof object.max_tps === "string") + message.max_tps = parseInt(object.max_tps, 10); + else if (typeof object.max_tps === "number") + message.max_tps = object.max_tps; + else if (typeof object.max_tps === "object") + message.max_tps = new $util.LongBits(object.max_tps.low >>> 0, object.max_tps.high >>> 0).toNumber(); + if (object.max_replication_lag != null) + if ($util.Long) + (message.max_replication_lag = $util.Long.fromValue(object.max_replication_lag)).unsigned = false; + else if (typeof object.max_replication_lag === "string") + message.max_replication_lag = parseInt(object.max_replication_lag, 10); + else if (typeof object.max_replication_lag === "number") + message.max_replication_lag = object.max_replication_lag; + else if (typeof object.max_replication_lag === "object") + message.max_replication_lag = new $util.LongBits(object.max_replication_lag.low >>> 0, object.max_replication_lag.high >>> 0).toNumber(); + if (object.time_updated != null) { + if (typeof object.time_updated !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated: object expected"); + message.time_updated = $root.vttime.Time.fromObject(object.time_updated); + } + if (object.transaction_timestamp != null) { + if (typeof object.transaction_timestamp !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp: object expected"); + message.transaction_timestamp = $root.vttime.Time.fromObject(object.transaction_timestamp); + } + switch (object.state) { + default: + if (typeof object.state === "number") { + message.state = object.state; + break; + } + break; + case "Unknown": + case 0: + message.state = 0; + break; + case "Init": + case 1: + message.state = 1; + break; + case "Stopped": + case 2: + message.state = 2; + break; + case "Copying": + case 3: + message.state = 3; + break; + case "Running": + case 4: + message.state = 4; + break; + case "Error": + case 5: + message.state = 5; + break; + case "Lagging": + case 6: + message.state = 6; + break; + } + if (object.message != null) + message.message = String(object.message); + if (object.rows_copied != null) + if ($util.Long) + (message.rows_copied = $util.Long.fromValue(object.rows_copied)).unsigned = false; + else if (typeof object.rows_copied === "string") + message.rows_copied = parseInt(object.rows_copied, 10); + else if (typeof object.rows_copied === "number") + message.rows_copied = object.rows_copied; + else if (typeof object.rows_copied === "object") + message.rows_copied = new $util.LongBits(object.rows_copied.low >>> 0, object.rows_copied.high >>> 0).toNumber(); + if (object.time_heartbeat != null) { + if (typeof object.time_heartbeat !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat: object expected"); + message.time_heartbeat = $root.vttime.Time.fromObject(object.time_heartbeat); + } + if (object.time_throttled != null) { + if (typeof object.time_throttled !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled: object expected"); + message.time_throttled = $root.vttime.Time.fromObject(object.time_throttled); + } + if (object.component_throttled != null) + message.component_throttled = String(object.component_throttled); + return message; + }; + + /** + * Creates a plain object from a Stream message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} message Stream + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Stream.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.id = 0; + object.bls = null; + object.pos = ""; + object.stop_pos = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_tps = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_tps = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_replication_lag = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_replication_lag = options.longs === String ? "0" : 0; + object.time_updated = null; + object.transaction_timestamp = null; + object.state = options.enums === String ? "Unknown" : 0; + object.message = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.rows_copied = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.rows_copied = options.longs === String ? "0" : 0; + object.time_heartbeat = null; + object.time_throttled = null; + object.component_throttled = ""; + } + if (message.id != null && message.hasOwnProperty("id")) + object.id = message.id; + if (message.bls != null && message.hasOwnProperty("bls")) + object.bls = $root.binlogdata.BinlogSource.toObject(message.bls, options); + if (message.pos != null && message.hasOwnProperty("pos")) + object.pos = message.pos; + if (message.stop_pos != null && message.hasOwnProperty("stop_pos")) + object.stop_pos = message.stop_pos; + if (message.max_tps != null && message.hasOwnProperty("max_tps")) + if (typeof message.max_tps === "number") + object.max_tps = options.longs === String ? String(message.max_tps) : message.max_tps; + else + object.max_tps = options.longs === String ? $util.Long.prototype.toString.call(message.max_tps) : options.longs === Number ? new $util.LongBits(message.max_tps.low >>> 0, message.max_tps.high >>> 0).toNumber() : message.max_tps; + if (message.max_replication_lag != null && message.hasOwnProperty("max_replication_lag")) + if (typeof message.max_replication_lag === "number") + object.max_replication_lag = options.longs === String ? String(message.max_replication_lag) : message.max_replication_lag; + else + object.max_replication_lag = options.longs === String ? $util.Long.prototype.toString.call(message.max_replication_lag) : options.longs === Number ? new $util.LongBits(message.max_replication_lag.low >>> 0, message.max_replication_lag.high >>> 0).toNumber() : message.max_replication_lag; + if (message.time_updated != null && message.hasOwnProperty("time_updated")) + object.time_updated = $root.vttime.Time.toObject(message.time_updated, options); + if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) + object.transaction_timestamp = $root.vttime.Time.toObject(message.transaction_timestamp, options); + if (message.state != null && message.hasOwnProperty("state")) + object.state = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.state] === undefined ? message.state : $root.binlogdata.VReplicationWorkflowState[message.state] : message.state; + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; + if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) + if (typeof message.rows_copied === "number") + object.rows_copied = options.longs === String ? String(message.rows_copied) : message.rows_copied; + else + object.rows_copied = options.longs === String ? $util.Long.prototype.toString.call(message.rows_copied) : options.longs === Number ? new $util.LongBits(message.rows_copied.low >>> 0, message.rows_copied.high >>> 0).toNumber() : message.rows_copied; + if (message.time_heartbeat != null && message.hasOwnProperty("time_heartbeat")) + object.time_heartbeat = $root.vttime.Time.toObject(message.time_heartbeat, options); + if (message.time_throttled != null && message.hasOwnProperty("time_throttled")) + object.time_throttled = $root.vttime.Time.toObject(message.time_throttled, options); + if (message.component_throttled != null && message.hasOwnProperty("component_throttled")) + object.component_throttled = message.component_throttled; + return object; + }; + + /** + * Converts this Stream to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + * @returns {Object.} JSON object + */ + Stream.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Stream + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Stream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream"; + }; + + return Stream; + })(); + + return ReadVReplicationWorkflowResponse; + })(); + + tabletmanagerdata.VDiffRequest = (function() { + + /** + * Properties of a VDiffRequest. + * @memberof tabletmanagerdata + * @interface IVDiffRequest + * @property {string|null} [keyspace] VDiffRequest keyspace + * @property {string|null} [workflow] VDiffRequest workflow + * @property {string|null} [action] VDiffRequest action + * @property {string|null} [action_arg] VDiffRequest action_arg + * @property {string|null} [vdiff_uuid] VDiffRequest vdiff_uuid + * @property {tabletmanagerdata.IVDiffOptions|null} [options] VDiffRequest options + */ + + /** + * Constructs a new VDiffRequest. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffRequest. + * @implements IVDiffRequest + * @constructor + * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + */ + function VDiffRequest(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]]; } /** - * UpdateVRWorkflowResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * VDiffRequest keyspace. + * @member {string} keyspace + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.keyspace = ""; + + /** + * VDiffRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.workflow = ""; + + /** + * VDiffRequest action. + * @member {string} action + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.action = ""; + + /** + * VDiffRequest action_arg. + * @member {string} action_arg + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.action_arg = ""; + + /** + * VDiffRequest vdiff_uuid. + * @member {string} vdiff_uuid + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.vdiff_uuid = ""; + + /** + * VDiffRequest options. + * @member {tabletmanagerdata.IVDiffOptions|null|undefined} options + * @memberof tabletmanagerdata.VDiffRequest * @instance */ - UpdateVRWorkflowResponse.prototype.result = null; + VDiffRequest.prototype.options = null; /** - * Creates a new UpdateVRWorkflowResponse instance using the specified properties. + * Creates a new VDiffRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.UpdateVRWorkflowResponse} UpdateVRWorkflowResponse instance + * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest instance */ - UpdateVRWorkflowResponse.create = function create(properties) { - return new UpdateVRWorkflowResponse(properties); + VDiffRequest.create = function create(properties) { + return new VDiffRequest(properties); }; /** - * Encodes the specified UpdateVRWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowResponse.verify|verify} messages. + * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowResponse} message UpdateVRWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVRWorkflowResponse.encode = function encode(message, writer) { + VDiffRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.action != null && Object.hasOwnProperty.call(message, "action")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.action); + if (message.action_arg != null && Object.hasOwnProperty.call(message, "action_arg")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.action_arg); + if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.vdiff_uuid); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.tabletmanagerdata.VDiffOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); return writer; }; /** - * Encodes the specified UpdateVRWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVRWorkflowResponse.verify|verify} messages. + * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static - * @param {tabletmanagerdata.IUpdateVRWorkflowResponse} message UpdateVRWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVRWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { + VDiffRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateVRWorkflowResponse message from the specified reader or buffer. + * Decodes a VDiffRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.UpdateVRWorkflowResponse} UpdateVRWorkflowResponse + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVRWorkflowResponse.decode = function decode(reader, length) { + VDiffRequest.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.tabletmanagerdata.UpdateVRWorkflowResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.workflow = reader.string(); + break; + } + case 3: { + message.action = reader.string(); + break; + } + case 4: { + message.action_arg = reader.string(); + break; + } + case 5: { + message.vdiff_uuid = reader.string(); + break; + } + case 6: { + message.options = $root.tabletmanagerdata.VDiffOptions.decode(reader, reader.uint32()); break; } default: @@ -62430,128 +63066,170 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes an UpdateVRWorkflowResponse message from the specified reader or buffer, length delimited. + * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.UpdateVRWorkflowResponse} UpdateVRWorkflowResponse + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVRWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { + VDiffRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateVRWorkflowResponse message. + * Verifies a VDiffRequest message. * @function verify - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateVRWorkflowResponse.verify = function verify(message) { + VDiffRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.action != null && message.hasOwnProperty("action")) + if (!$util.isString(message.action)) + return "action: string expected"; + if (message.action_arg != null && message.hasOwnProperty("action_arg")) + if (!$util.isString(message.action_arg)) + return "action_arg: string expected"; + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + if (!$util.isString(message.vdiff_uuid)) + return "vdiff_uuid: string expected"; + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.tabletmanagerdata.VDiffOptions.verify(message.options); if (error) - return "result." + error; + return "options." + error; } return null; }; /** - * Creates an UpdateVRWorkflowResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.UpdateVRWorkflowResponse} UpdateVRWorkflowResponse + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest */ - UpdateVRWorkflowResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.UpdateVRWorkflowResponse) + VDiffRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffRequest) return object; - let message = new $root.tabletmanagerdata.UpdateVRWorkflowResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".tabletmanagerdata.UpdateVRWorkflowResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.tabletmanagerdata.VDiffRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.action != null) + message.action = String(object.action); + if (object.action_arg != null) + message.action_arg = String(object.action_arg); + if (object.vdiff_uuid != null) + message.vdiff_uuid = String(object.vdiff_uuid); + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".tabletmanagerdata.VDiffRequest.options: object expected"); + message.options = $root.tabletmanagerdata.VDiffOptions.fromObject(object.options); } return message; }; /** - * Creates a plain object from an UpdateVRWorkflowResponse message. Also converts values to other types if specified. + * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static - * @param {tabletmanagerdata.UpdateVRWorkflowResponse} message UpdateVRWorkflowResponse + * @param {tabletmanagerdata.VDiffRequest} message VDiffRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateVRWorkflowResponse.toObject = function toObject(message, options) { + VDiffRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); + if (options.defaults) { + object.keyspace = ""; + object.workflow = ""; + object.action = ""; + object.action_arg = ""; + object.vdiff_uuid = ""; + object.options = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.action != null && message.hasOwnProperty("action")) + object.action = message.action; + if (message.action_arg != null && message.hasOwnProperty("action_arg")) + object.action_arg = message.action_arg; + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + object.vdiff_uuid = message.vdiff_uuid; + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.tabletmanagerdata.VDiffOptions.toObject(message.options, options); return object; }; /** - * Converts this UpdateVRWorkflowResponse to JSON. + * Converts this VDiffRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @instance * @returns {Object.} JSON object */ - UpdateVRWorkflowResponse.prototype.toJSON = function toJSON() { + VDiffRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateVRWorkflowResponse + * Gets the default type url for VDiffRequest * @function getTypeUrl - * @memberof tabletmanagerdata.UpdateVRWorkflowResponse + * @memberof tabletmanagerdata.VDiffRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateVRWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.UpdateVRWorkflowResponse"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffRequest"; }; - return UpdateVRWorkflowResponse; + return VDiffRequest; })(); - tabletmanagerdata.ResetSequencesRequest = (function() { + tabletmanagerdata.VDiffResponse = (function() { /** - * Properties of a ResetSequencesRequest. + * Properties of a VDiffResponse. * @memberof tabletmanagerdata - * @interface IResetSequencesRequest - * @property {Array.|null} [tables] ResetSequencesRequest tables + * @interface IVDiffResponse + * @property {number|Long|null} [id] VDiffResponse id + * @property {query.IQueryResult|null} [output] VDiffResponse output + * @property {string|null} [vdiff_uuid] VDiffResponse vdiff_uuid */ /** - * Constructs a new ResetSequencesRequest. + * Constructs a new VDiffResponse. * @memberof tabletmanagerdata - * @classdesc Represents a ResetSequencesRequest. - * @implements IResetSequencesRequest + * @classdesc Represents a VDiffResponse. + * @implements IVDiffResponse * @constructor - * @param {tabletmanagerdata.IResetSequencesRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set */ - function ResetSequencesRequest(properties) { - this.tables = []; + function VDiffResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -62559,78 +63237,103 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * ResetSequencesRequest tables. - * @member {Array.} tables - * @memberof tabletmanagerdata.ResetSequencesRequest + * VDiffResponse id. + * @member {number|Long} id + * @memberof tabletmanagerdata.VDiffResponse * @instance */ - ResetSequencesRequest.prototype.tables = $util.emptyArray; + VDiffResponse.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new ResetSequencesRequest instance using the specified properties. + * VDiffResponse output. + * @member {query.IQueryResult|null|undefined} output + * @memberof tabletmanagerdata.VDiffResponse + * @instance + */ + VDiffResponse.prototype.output = null; + + /** + * VDiffResponse vdiff_uuid. + * @member {string} vdiff_uuid + * @memberof tabletmanagerdata.VDiffResponse + * @instance + */ + VDiffResponse.prototype.vdiff_uuid = ""; + + /** + * Creates a new VDiffResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static - * @param {tabletmanagerdata.IResetSequencesRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest instance + * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse instance */ - ResetSequencesRequest.create = function create(properties) { - return new ResetSequencesRequest(properties); + VDiffResponse.create = function create(properties) { + return new VDiffResponse(properties); }; /** - * Encodes the specified ResetSequencesRequest message. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. + * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static - * @param {tabletmanagerdata.IResetSequencesRequest} message ResetSequencesRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResetSequencesRequest.encode = function encode(message, writer) { + VDiffResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables[i]); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); + if (message.output != null && Object.hasOwnProperty.call(message, "output")) + $root.query.QueryResult.encode(message.output, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.vdiff_uuid); return writer; }; /** - * Encodes the specified ResetSequencesRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. + * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static - * @param {tabletmanagerdata.IResetSequencesRequest} message ResetSequencesRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResetSequencesRequest.encodeDelimited = function encodeDelimited(message, writer) { + VDiffResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ResetSequencesRequest message from the specified reader or buffer. + * Decodes a VDiffResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResetSequencesRequest.decode = function decode(reader, length) { + VDiffResponse.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.tabletmanagerdata.ResetSequencesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); + message.id = reader.int64(); + break; + } + case 2: { + message.output = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } + case 3: { + message.vdiff_uuid = reader.string(); break; } default: @@ -62642,133 +63345,160 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a ResetSequencesRequest message from the specified reader or buffer, length delimited. + * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResetSequencesRequest.decodeDelimited = function decodeDelimited(reader) { + VDiffResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ResetSequencesRequest message. + * Verifies a VDiffResponse message. * @function verify - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ResetSequencesRequest.verify = function verify(message) { + VDiffResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) + return "id: integer|Long expected"; + if (message.output != null && message.hasOwnProperty("output")) { + let error = $root.query.QueryResult.verify(message.output); + if (error) + return "output." + error; } + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + if (!$util.isString(message.vdiff_uuid)) + return "vdiff_uuid: string expected"; return null; }; /** - * Creates a ResetSequencesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse */ - ResetSequencesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.ResetSequencesRequest) + VDiffResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffResponse) return object; - let message = new $root.tabletmanagerdata.ResetSequencesRequest(); - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".tabletmanagerdata.ResetSequencesRequest.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); + let message = new $root.tabletmanagerdata.VDiffResponse(); + if (object.id != null) + if ($util.Long) + (message.id = $util.Long.fromValue(object.id)).unsigned = false; + else if (typeof object.id === "string") + message.id = parseInt(object.id, 10); + else if (typeof object.id === "number") + message.id = object.id; + else if (typeof object.id === "object") + message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); + if (object.output != null) { + if (typeof object.output !== "object") + throw TypeError(".tabletmanagerdata.VDiffResponse.output: object expected"); + message.output = $root.query.QueryResult.fromObject(object.output); } + if (object.vdiff_uuid != null) + message.vdiff_uuid = String(object.vdiff_uuid); return message; }; /** - * Creates a plain object from a ResetSequencesRequest message. Also converts values to other types if specified. + * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static - * @param {tabletmanagerdata.ResetSequencesRequest} message ResetSequencesRequest + * @param {tabletmanagerdata.VDiffResponse} message VDiffResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ResetSequencesRequest.toObject = function toObject(message, options) { + VDiffResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tables = []; - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.id = options.longs === String ? "0" : 0; + object.output = null; + object.vdiff_uuid = ""; } + if (message.id != null && message.hasOwnProperty("id")) + if (typeof message.id === "number") + object.id = options.longs === String ? String(message.id) : message.id; + else + object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; + if (message.output != null && message.hasOwnProperty("output")) + object.output = $root.query.QueryResult.toObject(message.output, options); + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + object.vdiff_uuid = message.vdiff_uuid; return object; }; /** - * Converts this ResetSequencesRequest to JSON. + * Converts this VDiffResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @instance * @returns {Object.} JSON object */ - ResetSequencesRequest.prototype.toJSON = function toJSON() { + VDiffResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ResetSequencesRequest + * Gets the default type url for VDiffResponse * @function getTypeUrl - * @memberof tabletmanagerdata.ResetSequencesRequest + * @memberof tabletmanagerdata.VDiffResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ResetSequencesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.ResetSequencesRequest"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffResponse"; }; - return ResetSequencesRequest; + return VDiffResponse; })(); - tabletmanagerdata.ResetSequencesResponse = (function() { + tabletmanagerdata.VDiffPickerOptions = (function() { /** - * Properties of a ResetSequencesResponse. + * Properties of a VDiffPickerOptions. * @memberof tabletmanagerdata - * @interface IResetSequencesResponse + * @interface IVDiffPickerOptions + * @property {string|null} [tablet_types] VDiffPickerOptions tablet_types + * @property {string|null} [source_cell] VDiffPickerOptions source_cell + * @property {string|null} [target_cell] VDiffPickerOptions target_cell */ /** - * Constructs a new ResetSequencesResponse. + * Constructs a new VDiffPickerOptions. * @memberof tabletmanagerdata - * @classdesc Represents a ResetSequencesResponse. - * @implements IResetSequencesResponse + * @classdesc Represents a VDiffPickerOptions. + * @implements IVDiffPickerOptions * @constructor - * @param {tabletmanagerdata.IResetSequencesResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set */ - function ResetSequencesResponse(properties) { + function VDiffPickerOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -62776,63 +63506,105 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * Creates a new ResetSequencesResponse instance using the specified properties. + * VDiffPickerOptions tablet_types. + * @member {string} tablet_types + * @memberof tabletmanagerdata.VDiffPickerOptions + * @instance + */ + VDiffPickerOptions.prototype.tablet_types = ""; + + /** + * VDiffPickerOptions source_cell. + * @member {string} source_cell + * @memberof tabletmanagerdata.VDiffPickerOptions + * @instance + */ + VDiffPickerOptions.prototype.source_cell = ""; + + /** + * VDiffPickerOptions target_cell. + * @member {string} target_cell + * @memberof tabletmanagerdata.VDiffPickerOptions + * @instance + */ + VDiffPickerOptions.prototype.target_cell = ""; + + /** + * Creates a new VDiffPickerOptions instance using the specified properties. * @function create - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IResetSequencesResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse instance + * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions instance */ - ResetSequencesResponse.create = function create(properties) { - return new ResetSequencesResponse(properties); + VDiffPickerOptions.create = function create(properties) { + return new VDiffPickerOptions(properties); }; /** - * Encodes the specified ResetSequencesResponse message. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. + * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IResetSequencesResponse} message ResetSequencesResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResetSequencesResponse.encode = function encode(message, writer) { + VDiffPickerOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tablet_types); + if (message.source_cell != null && Object.hasOwnProperty.call(message, "source_cell")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_cell); + if (message.target_cell != null && Object.hasOwnProperty.call(message, "target_cell")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_cell); return writer; }; /** - * Encodes the specified ResetSequencesResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. + * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IResetSequencesResponse} message ResetSequencesResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResetSequencesResponse.encodeDelimited = function encodeDelimited(message, writer) { + VDiffPickerOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ResetSequencesResponse message from the specified reader or buffer. + * Decodes a VDiffPickerOptions message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResetSequencesResponse.decode = function decode(reader, length) { + VDiffPickerOptions.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.tabletmanagerdata.ResetSequencesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffPickerOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_types = reader.string(); + break; + } + case 2: { + message.source_cell = reader.string(); + break; + } + case 3: { + message.target_cell = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -62842,123 +63614,141 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a ResetSequencesResponse message from the specified reader or buffer, length delimited. + * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResetSequencesResponse.decodeDelimited = function decodeDelimited(reader) { + VDiffPickerOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ResetSequencesResponse message. + * Verifies a VDiffPickerOptions message. * @function verify - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ResetSequencesResponse.verify = function verify(message) { + VDiffPickerOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + if (!$util.isString(message.tablet_types)) + return "tablet_types: string expected"; + if (message.source_cell != null && message.hasOwnProperty("source_cell")) + if (!$util.isString(message.source_cell)) + return "source_cell: string expected"; + if (message.target_cell != null && message.hasOwnProperty("target_cell")) + if (!$util.isString(message.target_cell)) + return "target_cell: string expected"; return null; }; /** - * Creates a ResetSequencesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions */ - ResetSequencesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.ResetSequencesResponse) + VDiffPickerOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffPickerOptions) return object; - return new $root.tabletmanagerdata.ResetSequencesResponse(); + let message = new $root.tabletmanagerdata.VDiffPickerOptions(); + if (object.tablet_types != null) + message.tablet_types = String(object.tablet_types); + if (object.source_cell != null) + message.source_cell = String(object.source_cell); + if (object.target_cell != null) + message.target_cell = String(object.target_cell); + return message; }; /** - * Creates a plain object from a ResetSequencesResponse message. Also converts values to other types if specified. + * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.ResetSequencesResponse} message ResetSequencesResponse + * @param {tabletmanagerdata.VDiffPickerOptions} message VDiffPickerOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ResetSequencesResponse.toObject = function toObject() { - return {}; + VDiffPickerOptions.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet_types = ""; + object.source_cell = ""; + object.target_cell = ""; + } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + object.tablet_types = message.tablet_types; + if (message.source_cell != null && message.hasOwnProperty("source_cell")) + object.source_cell = message.source_cell; + if (message.target_cell != null && message.hasOwnProperty("target_cell")) + object.target_cell = message.target_cell; + return object; }; /** - * Converts this ResetSequencesResponse to JSON. + * Converts this VDiffPickerOptions to JSON. * @function toJSON - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @instance * @returns {Object.} JSON object */ - ResetSequencesResponse.prototype.toJSON = function toJSON() { + VDiffPickerOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ResetSequencesResponse + * Gets the default type url for VDiffPickerOptions * @function getTypeUrl - * @memberof tabletmanagerdata.ResetSequencesResponse + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ResetSequencesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffPickerOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.ResetSequencesResponse"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffPickerOptions"; }; - return ResetSequencesResponse; + return VDiffPickerOptions; })(); - return tabletmanagerdata; -})(); - -export const binlogdata = $root.binlogdata = (() => { - - /** - * Namespace binlogdata. - * @exports binlogdata - * @namespace - */ - const binlogdata = {}; - - binlogdata.Charset = (function() { + tabletmanagerdata.VDiffReportOptions = (function() { /** - * Properties of a Charset. - * @memberof binlogdata - * @interface ICharset - * @property {number|null} [client] Charset client - * @property {number|null} [conn] Charset conn - * @property {number|null} [server] Charset server + * Properties of a VDiffReportOptions. + * @memberof tabletmanagerdata + * @interface IVDiffReportOptions + * @property {boolean|null} [only_pks] VDiffReportOptions only_pks + * @property {boolean|null} [debug_query] VDiffReportOptions debug_query + * @property {string|null} [format] VDiffReportOptions format */ /** - * Constructs a new Charset. - * @memberof binlogdata - * @classdesc Represents a Charset. - * @implements ICharset + * Constructs a new VDiffReportOptions. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffReportOptions. + * @implements IVDiffReportOptions * @constructor - * @param {binlogdata.ICharset=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set */ - function Charset(properties) { + function VDiffReportOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -62966,103 +63756,103 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * Charset client. - * @member {number} client - * @memberof binlogdata.Charset + * VDiffReportOptions only_pks. + * @member {boolean} only_pks + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - Charset.prototype.client = 0; + VDiffReportOptions.prototype.only_pks = false; /** - * Charset conn. - * @member {number} conn - * @memberof binlogdata.Charset + * VDiffReportOptions debug_query. + * @member {boolean} debug_query + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - Charset.prototype.conn = 0; + VDiffReportOptions.prototype.debug_query = false; /** - * Charset server. - * @member {number} server - * @memberof binlogdata.Charset + * VDiffReportOptions format. + * @member {string} format + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - Charset.prototype.server = 0; + VDiffReportOptions.prototype.format = ""; /** - * Creates a new Charset instance using the specified properties. + * Creates a new VDiffReportOptions instance using the specified properties. * @function create - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {binlogdata.ICharset=} [properties] Properties to set - * @returns {binlogdata.Charset} Charset instance + * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions instance */ - Charset.create = function create(properties) { - return new Charset(properties); + VDiffReportOptions.create = function create(properties) { + return new VDiffReportOptions(properties); }; /** - * Encodes the specified Charset message. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. + * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. * @function encode - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {binlogdata.ICharset} message Charset message or plain object to encode + * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Charset.encode = function encode(message, writer) { + VDiffReportOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.client != null && Object.hasOwnProperty.call(message, "client")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.client); - if (message.conn != null && Object.hasOwnProperty.call(message, "conn")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.conn); - if (message.server != null && Object.hasOwnProperty.call(message, "server")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.server); + if (message.only_pks != null && Object.hasOwnProperty.call(message, "only_pks")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.only_pks); + if (message.debug_query != null && Object.hasOwnProperty.call(message, "debug_query")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.debug_query); + if (message.format != null && Object.hasOwnProperty.call(message, "format")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.format); return writer; }; /** - * Encodes the specified Charset message, length delimited. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. + * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {binlogdata.ICharset} message Charset message or plain object to encode + * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Charset.encodeDelimited = function encodeDelimited(message, writer) { + VDiffReportOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Charset message from the specified reader or buffer. + * Decodes a VDiffReportOptions message from the specified reader or buffer. * @function decode - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.Charset} Charset + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Charset.decode = function decode(reader, length) { + VDiffReportOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Charset(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffReportOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.client = reader.int32(); + message.only_pks = reader.bool(); break; } case 2: { - message.conn = reader.int32(); + message.debug_query = reader.bool(); break; } case 3: { - message.server = reader.int32(); + message.format = reader.string(); break; } default: @@ -63074,141 +63864,146 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a Charset message from the specified reader or buffer, length delimited. + * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.Charset} Charset + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Charset.decodeDelimited = function decodeDelimited(reader) { + VDiffReportOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Charset message. + * Verifies a VDiffReportOptions message. * @function verify - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Charset.verify = function verify(message) { + VDiffReportOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.client != null && message.hasOwnProperty("client")) - if (!$util.isInteger(message.client)) - return "client: integer expected"; - if (message.conn != null && message.hasOwnProperty("conn")) - if (!$util.isInteger(message.conn)) - return "conn: integer expected"; - if (message.server != null && message.hasOwnProperty("server")) - if (!$util.isInteger(message.server)) - return "server: integer expected"; + if (message.only_pks != null && message.hasOwnProperty("only_pks")) + if (typeof message.only_pks !== "boolean") + return "only_pks: boolean expected"; + if (message.debug_query != null && message.hasOwnProperty("debug_query")) + if (typeof message.debug_query !== "boolean") + return "debug_query: boolean expected"; + if (message.format != null && message.hasOwnProperty("format")) + if (!$util.isString(message.format)) + return "format: string expected"; return null; }; /** - * Creates a Charset message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffReportOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {Object.} object Plain object - * @returns {binlogdata.Charset} Charset + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions */ - Charset.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.Charset) + VDiffReportOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffReportOptions) return object; - let message = new $root.binlogdata.Charset(); - if (object.client != null) - message.client = object.client | 0; - if (object.conn != null) - message.conn = object.conn | 0; - if (object.server != null) - message.server = object.server | 0; + let message = new $root.tabletmanagerdata.VDiffReportOptions(); + if (object.only_pks != null) + message.only_pks = Boolean(object.only_pks); + if (object.debug_query != null) + message.debug_query = Boolean(object.debug_query); + if (object.format != null) + message.format = String(object.format); return message; }; /** - * Creates a plain object from a Charset message. Also converts values to other types if specified. + * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {binlogdata.Charset} message Charset + * @param {tabletmanagerdata.VDiffReportOptions} message VDiffReportOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Charset.toObject = function toObject(message, options) { + VDiffReportOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.client = 0; - object.conn = 0; - object.server = 0; + object.only_pks = false; + object.debug_query = false; + object.format = ""; } - if (message.client != null && message.hasOwnProperty("client")) - object.client = message.client; - if (message.conn != null && message.hasOwnProperty("conn")) - object.conn = message.conn; - if (message.server != null && message.hasOwnProperty("server")) - object.server = message.server; + if (message.only_pks != null && message.hasOwnProperty("only_pks")) + object.only_pks = message.only_pks; + if (message.debug_query != null && message.hasOwnProperty("debug_query")) + object.debug_query = message.debug_query; + if (message.format != null && message.hasOwnProperty("format")) + object.format = message.format; return object; }; /** - * Converts this Charset to JSON. + * Converts this VDiffReportOptions to JSON. * @function toJSON - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @instance * @returns {Object.} JSON object */ - Charset.prototype.toJSON = function toJSON() { + VDiffReportOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Charset + * Gets the default type url for VDiffReportOptions * @function getTypeUrl - * @memberof binlogdata.Charset + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Charset.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffReportOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.Charset"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffReportOptions"; }; - return Charset; + return VDiffReportOptions; })(); - binlogdata.BinlogTransaction = (function() { + tabletmanagerdata.VDiffCoreOptions = (function() { /** - * Properties of a BinlogTransaction. - * @memberof binlogdata - * @interface IBinlogTransaction - * @property {Array.|null} [statements] BinlogTransaction statements - * @property {query.IEventToken|null} [event_token] BinlogTransaction event_token + * Properties of a VDiffCoreOptions. + * @memberof tabletmanagerdata + * @interface IVDiffCoreOptions + * @property {string|null} [tables] VDiffCoreOptions tables + * @property {boolean|null} [auto_retry] VDiffCoreOptions auto_retry + * @property {number|Long|null} [max_rows] VDiffCoreOptions max_rows + * @property {boolean|null} [checksum] VDiffCoreOptions checksum + * @property {number|Long|null} [sample_pct] VDiffCoreOptions sample_pct + * @property {number|Long|null} [timeout_seconds] VDiffCoreOptions timeout_seconds + * @property {number|Long|null} [max_extra_rows_to_compare] VDiffCoreOptions max_extra_rows_to_compare + * @property {boolean|null} [update_table_stats] VDiffCoreOptions update_table_stats */ /** - * Constructs a new BinlogTransaction. - * @memberof binlogdata - * @classdesc Represents a BinlogTransaction. - * @implements IBinlogTransaction + * Constructs a new VDiffCoreOptions. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffCoreOptions. + * @implements IVDiffCoreOptions * @constructor - * @param {binlogdata.IBinlogTransaction=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set */ - function BinlogTransaction(properties) { - this.statements = []; + function VDiffCoreOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -63216,92 +64011,173 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * BinlogTransaction statements. - * @member {Array.} statements - * @memberof binlogdata.BinlogTransaction + * VDiffCoreOptions tables. + * @member {string} tables + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance */ - BinlogTransaction.prototype.statements = $util.emptyArray; + VDiffCoreOptions.prototype.tables = ""; /** - * BinlogTransaction event_token. - * @member {query.IEventToken|null|undefined} event_token - * @memberof binlogdata.BinlogTransaction + * VDiffCoreOptions auto_retry. + * @member {boolean} auto_retry + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance */ - BinlogTransaction.prototype.event_token = null; + VDiffCoreOptions.prototype.auto_retry = false; /** - * Creates a new BinlogTransaction instance using the specified properties. + * VDiffCoreOptions max_rows. + * @member {number|Long} max_rows + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VDiffCoreOptions checksum. + * @member {boolean} checksum + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.checksum = false; + + /** + * VDiffCoreOptions sample_pct. + * @member {number|Long} sample_pct + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.sample_pct = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VDiffCoreOptions timeout_seconds. + * @member {number|Long} timeout_seconds + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.timeout_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VDiffCoreOptions max_extra_rows_to_compare. + * @member {number|Long} max_extra_rows_to_compare + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.max_extra_rows_to_compare = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VDiffCoreOptions update_table_stats. + * @member {boolean} update_table_stats + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.update_table_stats = false; + + /** + * Creates a new VDiffCoreOptions instance using the specified properties. * @function create - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {binlogdata.IBinlogTransaction=} [properties] Properties to set - * @returns {binlogdata.BinlogTransaction} BinlogTransaction instance + * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions instance */ - BinlogTransaction.create = function create(properties) { - return new BinlogTransaction(properties); + VDiffCoreOptions.create = function create(properties) { + return new VDiffCoreOptions(properties); }; /** - * Encodes the specified BinlogTransaction message. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. + * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. * @function encode - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {binlogdata.IBinlogTransaction} message BinlogTransaction message or plain object to encode + * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BinlogTransaction.encode = function encode(message, writer) { + VDiffCoreOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.statements != null && message.statements.length) - for (let i = 0; i < message.statements.length; ++i) - $root.binlogdata.BinlogTransaction.Statement.encode(message.statements[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.event_token != null && Object.hasOwnProperty.call(message, "event_token")) - $root.query.EventToken.encode(message.event_token, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables); + if (message.auto_retry != null && Object.hasOwnProperty.call(message, "auto_retry")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.auto_retry); + if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); + if (message.checksum != null && Object.hasOwnProperty.call(message, "checksum")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.checksum); + if (message.sample_pct != null && Object.hasOwnProperty.call(message, "sample_pct")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.sample_pct); + if (message.timeout_seconds != null && Object.hasOwnProperty.call(message, "timeout_seconds")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.timeout_seconds); + if (message.max_extra_rows_to_compare != null && Object.hasOwnProperty.call(message, "max_extra_rows_to_compare")) + writer.uint32(/* id 7, wireType 0 =*/56).int64(message.max_extra_rows_to_compare); + if (message.update_table_stats != null && Object.hasOwnProperty.call(message, "update_table_stats")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.update_table_stats); return writer; }; /** - * Encodes the specified BinlogTransaction message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. + * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {binlogdata.IBinlogTransaction} message BinlogTransaction message or plain object to encode + * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BinlogTransaction.encodeDelimited = function encodeDelimited(message, writer) { + VDiffCoreOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BinlogTransaction message from the specified reader or buffer. + * Decodes a VDiffCoreOptions message from the specified reader or buffer. * @function decode - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.BinlogTransaction} BinlogTransaction + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BinlogTransaction.decode = function decode(reader, length) { + VDiffCoreOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogTransaction(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffCoreOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.statements && message.statements.length)) - message.statements = []; - message.statements.push($root.binlogdata.BinlogTransaction.Statement.decode(reader, reader.uint32())); + message.tables = reader.string(); + break; + } + case 2: { + message.auto_retry = reader.bool(); + break; + } + case 3: { + message.max_rows = reader.int64(); break; } case 4: { - message.event_token = $root.query.EventToken.decode(reader, reader.uint32()); + message.checksum = reader.bool(); + break; + } + case 5: { + message.sample_pct = reader.int64(); + break; + } + case 6: { + message.timeout_seconds = reader.int64(); + break; + } + case 7: { + message.max_extra_rows_to_compare = reader.int64(); + break; + } + case 8: { + message.update_table_stats = reader.bool(); break; } default: @@ -63313,508 +64189,237 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a BinlogTransaction message from the specified reader or buffer, length delimited. + * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.BinlogTransaction} BinlogTransaction + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BinlogTransaction.decodeDelimited = function decodeDelimited(reader) { + VDiffCoreOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BinlogTransaction message. + * Verifies a VDiffCoreOptions message. * @function verify - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BinlogTransaction.verify = function verify(message) { + VDiffCoreOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.statements != null && message.hasOwnProperty("statements")) { - if (!Array.isArray(message.statements)) - return "statements: array expected"; - for (let i = 0; i < message.statements.length; ++i) { - let error = $root.binlogdata.BinlogTransaction.Statement.verify(message.statements[i]); - if (error) - return "statements." + error; - } - } - if (message.event_token != null && message.hasOwnProperty("event_token")) { - let error = $root.query.EventToken.verify(message.event_token); - if (error) - return "event_token." + error; - } + if (message.tables != null && message.hasOwnProperty("tables")) + if (!$util.isString(message.tables)) + return "tables: string expected"; + if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) + if (typeof message.auto_retry !== "boolean") + return "auto_retry: boolean expected"; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) + return "max_rows: integer|Long expected"; + if (message.checksum != null && message.hasOwnProperty("checksum")) + if (typeof message.checksum !== "boolean") + return "checksum: boolean expected"; + if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) + if (!$util.isInteger(message.sample_pct) && !(message.sample_pct && $util.isInteger(message.sample_pct.low) && $util.isInteger(message.sample_pct.high))) + return "sample_pct: integer|Long expected"; + if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) + if (!$util.isInteger(message.timeout_seconds) && !(message.timeout_seconds && $util.isInteger(message.timeout_seconds.low) && $util.isInteger(message.timeout_seconds.high))) + return "timeout_seconds: integer|Long expected"; + if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) + if (!$util.isInteger(message.max_extra_rows_to_compare) && !(message.max_extra_rows_to_compare && $util.isInteger(message.max_extra_rows_to_compare.low) && $util.isInteger(message.max_extra_rows_to_compare.high))) + return "max_extra_rows_to_compare: integer|Long expected"; + if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) + if (typeof message.update_table_stats !== "boolean") + return "update_table_stats: boolean expected"; return null; }; /** - * Creates a BinlogTransaction message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {Object.} object Plain object - * @returns {binlogdata.BinlogTransaction} BinlogTransaction + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions */ - BinlogTransaction.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.BinlogTransaction) + VDiffCoreOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffCoreOptions) return object; - let message = new $root.binlogdata.BinlogTransaction(); - if (object.statements) { - if (!Array.isArray(object.statements)) - throw TypeError(".binlogdata.BinlogTransaction.statements: array expected"); - message.statements = []; - for (let i = 0; i < object.statements.length; ++i) { - if (typeof object.statements[i] !== "object") - throw TypeError(".binlogdata.BinlogTransaction.statements: object expected"); - message.statements[i] = $root.binlogdata.BinlogTransaction.Statement.fromObject(object.statements[i]); - } - } - if (object.event_token != null) { - if (typeof object.event_token !== "object") - throw TypeError(".binlogdata.BinlogTransaction.event_token: object expected"); - message.event_token = $root.query.EventToken.fromObject(object.event_token); - } + let message = new $root.tabletmanagerdata.VDiffCoreOptions(); + if (object.tables != null) + message.tables = String(object.tables); + if (object.auto_retry != null) + message.auto_retry = Boolean(object.auto_retry); + if (object.max_rows != null) + if ($util.Long) + (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; + else if (typeof object.max_rows === "string") + message.max_rows = parseInt(object.max_rows, 10); + else if (typeof object.max_rows === "number") + message.max_rows = object.max_rows; + else if (typeof object.max_rows === "object") + message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); + if (object.checksum != null) + message.checksum = Boolean(object.checksum); + if (object.sample_pct != null) + if ($util.Long) + (message.sample_pct = $util.Long.fromValue(object.sample_pct)).unsigned = false; + else if (typeof object.sample_pct === "string") + message.sample_pct = parseInt(object.sample_pct, 10); + else if (typeof object.sample_pct === "number") + message.sample_pct = object.sample_pct; + else if (typeof object.sample_pct === "object") + message.sample_pct = new $util.LongBits(object.sample_pct.low >>> 0, object.sample_pct.high >>> 0).toNumber(); + if (object.timeout_seconds != null) + if ($util.Long) + (message.timeout_seconds = $util.Long.fromValue(object.timeout_seconds)).unsigned = false; + else if (typeof object.timeout_seconds === "string") + message.timeout_seconds = parseInt(object.timeout_seconds, 10); + else if (typeof object.timeout_seconds === "number") + message.timeout_seconds = object.timeout_seconds; + else if (typeof object.timeout_seconds === "object") + message.timeout_seconds = new $util.LongBits(object.timeout_seconds.low >>> 0, object.timeout_seconds.high >>> 0).toNumber(); + if (object.max_extra_rows_to_compare != null) + if ($util.Long) + (message.max_extra_rows_to_compare = $util.Long.fromValue(object.max_extra_rows_to_compare)).unsigned = false; + else if (typeof object.max_extra_rows_to_compare === "string") + message.max_extra_rows_to_compare = parseInt(object.max_extra_rows_to_compare, 10); + else if (typeof object.max_extra_rows_to_compare === "number") + message.max_extra_rows_to_compare = object.max_extra_rows_to_compare; + else if (typeof object.max_extra_rows_to_compare === "object") + message.max_extra_rows_to_compare = new $util.LongBits(object.max_extra_rows_to_compare.low >>> 0, object.max_extra_rows_to_compare.high >>> 0).toNumber(); + if (object.update_table_stats != null) + message.update_table_stats = Boolean(object.update_table_stats); return message; }; /** - * Creates a plain object from a BinlogTransaction message. Also converts values to other types if specified. + * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {binlogdata.BinlogTransaction} message BinlogTransaction + * @param {tabletmanagerdata.VDiffCoreOptions} message VDiffCoreOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BinlogTransaction.toObject = function toObject(message, options) { + VDiffCoreOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.statements = []; - if (options.defaults) - object.event_token = null; - if (message.statements && message.statements.length) { - object.statements = []; - for (let j = 0; j < message.statements.length; ++j) - object.statements[j] = $root.binlogdata.BinlogTransaction.Statement.toObject(message.statements[j], options); + if (options.defaults) { + object.tables = ""; + object.auto_retry = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_rows = options.longs === String ? "0" : 0; + object.checksum = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.sample_pct = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.sample_pct = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.timeout_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.timeout_seconds = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_extra_rows_to_compare = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_extra_rows_to_compare = options.longs === String ? "0" : 0; + object.update_table_stats = false; } - if (message.event_token != null && message.hasOwnProperty("event_token")) - object.event_token = $root.query.EventToken.toObject(message.event_token, options); - return object; - }; - - /** - * Converts this BinlogTransaction to JSON. - * @function toJSON - * @memberof binlogdata.BinlogTransaction - * @instance - * @returns {Object.} JSON object - */ - BinlogTransaction.prototype.toJSON = function toJSON() { + if (message.tables != null && message.hasOwnProperty("tables")) + object.tables = message.tables; + if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) + object.auto_retry = message.auto_retry; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (typeof message.max_rows === "number") + object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; + else + object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; + if (message.checksum != null && message.hasOwnProperty("checksum")) + object.checksum = message.checksum; + if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) + if (typeof message.sample_pct === "number") + object.sample_pct = options.longs === String ? String(message.sample_pct) : message.sample_pct; + else + object.sample_pct = options.longs === String ? $util.Long.prototype.toString.call(message.sample_pct) : options.longs === Number ? new $util.LongBits(message.sample_pct.low >>> 0, message.sample_pct.high >>> 0).toNumber() : message.sample_pct; + if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) + if (typeof message.timeout_seconds === "number") + object.timeout_seconds = options.longs === String ? String(message.timeout_seconds) : message.timeout_seconds; + else + object.timeout_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.timeout_seconds) : options.longs === Number ? new $util.LongBits(message.timeout_seconds.low >>> 0, message.timeout_seconds.high >>> 0).toNumber() : message.timeout_seconds; + if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) + if (typeof message.max_extra_rows_to_compare === "number") + object.max_extra_rows_to_compare = options.longs === String ? String(message.max_extra_rows_to_compare) : message.max_extra_rows_to_compare; + else + object.max_extra_rows_to_compare = options.longs === String ? $util.Long.prototype.toString.call(message.max_extra_rows_to_compare) : options.longs === Number ? new $util.LongBits(message.max_extra_rows_to_compare.low >>> 0, message.max_extra_rows_to_compare.high >>> 0).toNumber() : message.max_extra_rows_to_compare; + if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) + object.update_table_stats = message.update_table_stats; + return object; + }; + + /** + * Converts this VDiffCoreOptions to JSON. + * @function toJSON + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + * @returns {Object.} JSON object + */ + VDiffCoreOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BinlogTransaction + * Gets the default type url for VDiffCoreOptions * @function getTypeUrl - * @memberof binlogdata.BinlogTransaction + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BinlogTransaction.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffCoreOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.BinlogTransaction"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffCoreOptions"; }; - BinlogTransaction.Statement = (function() { - - /** - * Properties of a Statement. - * @memberof binlogdata.BinlogTransaction - * @interface IStatement - * @property {binlogdata.BinlogTransaction.Statement.Category|null} [category] Statement category - * @property {binlogdata.ICharset|null} [charset] Statement charset - * @property {Uint8Array|null} [sql] Statement sql - */ - - /** - * Constructs a new Statement. - * @memberof binlogdata.BinlogTransaction - * @classdesc Represents a Statement. - * @implements IStatement - * @constructor - * @param {binlogdata.BinlogTransaction.IStatement=} [properties] Properties to set - */ - function Statement(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]]; - } - - /** - * Statement category. - * @member {binlogdata.BinlogTransaction.Statement.Category} category - * @memberof binlogdata.BinlogTransaction.Statement - * @instance - */ - Statement.prototype.category = 0; - - /** - * Statement charset. - * @member {binlogdata.ICharset|null|undefined} charset - * @memberof binlogdata.BinlogTransaction.Statement - * @instance - */ - Statement.prototype.charset = null; - - /** - * Statement sql. - * @member {Uint8Array} sql - * @memberof binlogdata.BinlogTransaction.Statement - * @instance - */ - Statement.prototype.sql = $util.newBuffer([]); - - /** - * Creates a new Statement instance using the specified properties. - * @function create - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {binlogdata.BinlogTransaction.IStatement=} [properties] Properties to set - * @returns {binlogdata.BinlogTransaction.Statement} Statement instance - */ - Statement.create = function create(properties) { - return new Statement(properties); - }; - - /** - * Encodes the specified Statement message. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. - * @function encode - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {binlogdata.BinlogTransaction.IStatement} message Statement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Statement.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.category != null && Object.hasOwnProperty.call(message, "category")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.category); - if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) - $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) - writer.uint32(/* id 3, wireType 2 =*/26).bytes(message.sql); - return writer; - }; - - /** - * Encodes the specified Statement message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. - * @function encodeDelimited - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {binlogdata.BinlogTransaction.IStatement} message Statement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Statement.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a Statement message from the specified reader or buffer. - * @function decode - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.BinlogTransaction.Statement} Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Statement.decode = function decode(reader, length) { - if (!(reader instanceof $Reader)) - reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogTransaction.Statement(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.category = reader.int32(); - break; - } - case 2: { - message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); - break; - } - case 3: { - message.sql = reader.bytes(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a Statement message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.BinlogTransaction.Statement} Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Statement.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a Statement message. - * @function verify - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Statement.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.category != null && message.hasOwnProperty("category")) - switch (message.category) { - default: - return "category: enum value expected"; - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - case 9: - break; - } - if (message.charset != null && message.hasOwnProperty("charset")) { - let error = $root.binlogdata.Charset.verify(message.charset); - if (error) - return "charset." + error; - } - if (message.sql != null && message.hasOwnProperty("sql")) - if (!(message.sql && typeof message.sql.length === "number" || $util.isString(message.sql))) - return "sql: buffer expected"; - return null; - }; - - /** - * Creates a Statement message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {Object.} object Plain object - * @returns {binlogdata.BinlogTransaction.Statement} Statement - */ - Statement.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.BinlogTransaction.Statement) - return object; - let message = new $root.binlogdata.BinlogTransaction.Statement(); - switch (object.category) { - default: - if (typeof object.category === "number") { - message.category = object.category; - break; - } - break; - case "BL_UNRECOGNIZED": - case 0: - message.category = 0; - break; - case "BL_BEGIN": - case 1: - message.category = 1; - break; - case "BL_COMMIT": - case 2: - message.category = 2; - break; - case "BL_ROLLBACK": - case 3: - message.category = 3; - break; - case "BL_DML_DEPRECATED": - case 4: - message.category = 4; - break; - case "BL_DDL": - case 5: - message.category = 5; - break; - case "BL_SET": - case 6: - message.category = 6; - break; - case "BL_INSERT": - case 7: - message.category = 7; - break; - case "BL_UPDATE": - case 8: - message.category = 8; - break; - case "BL_DELETE": - case 9: - message.category = 9; - break; - } - if (object.charset != null) { - if (typeof object.charset !== "object") - throw TypeError(".binlogdata.BinlogTransaction.Statement.charset: object expected"); - message.charset = $root.binlogdata.Charset.fromObject(object.charset); - } - if (object.sql != null) - if (typeof object.sql === "string") - $util.base64.decode(object.sql, message.sql = $util.newBuffer($util.base64.length(object.sql)), 0); - else if (object.sql.length >= 0) - message.sql = object.sql; - return message; - }; - - /** - * Creates a plain object from a Statement message. Also converts values to other types if specified. - * @function toObject - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {binlogdata.BinlogTransaction.Statement} message Statement - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Statement.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.category = options.enums === String ? "BL_UNRECOGNIZED" : 0; - object.charset = null; - if (options.bytes === String) - object.sql = ""; - else { - object.sql = []; - if (options.bytes !== Array) - object.sql = $util.newBuffer(object.sql); - } - } - if (message.category != null && message.hasOwnProperty("category")) - object.category = options.enums === String ? $root.binlogdata.BinlogTransaction.Statement.Category[message.category] === undefined ? message.category : $root.binlogdata.BinlogTransaction.Statement.Category[message.category] : message.category; - if (message.charset != null && message.hasOwnProperty("charset")) - object.charset = $root.binlogdata.Charset.toObject(message.charset, options); - if (message.sql != null && message.hasOwnProperty("sql")) - object.sql = options.bytes === String ? $util.base64.encode(message.sql, 0, message.sql.length) : options.bytes === Array ? Array.prototype.slice.call(message.sql) : message.sql; - return object; - }; - - /** - * Converts this Statement to JSON. - * @function toJSON - * @memberof binlogdata.BinlogTransaction.Statement - * @instance - * @returns {Object.} JSON object - */ - Statement.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for Statement - * @function getTypeUrl - * @memberof binlogdata.BinlogTransaction.Statement - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Statement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/binlogdata.BinlogTransaction.Statement"; - }; - - /** - * Category enum. - * @name binlogdata.BinlogTransaction.Statement.Category - * @enum {number} - * @property {number} BL_UNRECOGNIZED=0 BL_UNRECOGNIZED value - * @property {number} BL_BEGIN=1 BL_BEGIN value - * @property {number} BL_COMMIT=2 BL_COMMIT value - * @property {number} BL_ROLLBACK=3 BL_ROLLBACK value - * @property {number} BL_DML_DEPRECATED=4 BL_DML_DEPRECATED value - * @property {number} BL_DDL=5 BL_DDL value - * @property {number} BL_SET=6 BL_SET value - * @property {number} BL_INSERT=7 BL_INSERT value - * @property {number} BL_UPDATE=8 BL_UPDATE value - * @property {number} BL_DELETE=9 BL_DELETE value - */ - Statement.Category = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "BL_UNRECOGNIZED"] = 0; - values[valuesById[1] = "BL_BEGIN"] = 1; - values[valuesById[2] = "BL_COMMIT"] = 2; - values[valuesById[3] = "BL_ROLLBACK"] = 3; - values[valuesById[4] = "BL_DML_DEPRECATED"] = 4; - values[valuesById[5] = "BL_DDL"] = 5; - values[valuesById[6] = "BL_SET"] = 6; - values[valuesById[7] = "BL_INSERT"] = 7; - values[valuesById[8] = "BL_UPDATE"] = 8; - values[valuesById[9] = "BL_DELETE"] = 9; - return values; - })(); - - return Statement; - })(); - - return BinlogTransaction; + return VDiffCoreOptions; })(); - binlogdata.StreamKeyRangeRequest = (function() { + tabletmanagerdata.VDiffOptions = (function() { /** - * Properties of a StreamKeyRangeRequest. - * @memberof binlogdata - * @interface IStreamKeyRangeRequest - * @property {string|null} [position] StreamKeyRangeRequest position - * @property {topodata.IKeyRange|null} [key_range] StreamKeyRangeRequest key_range - * @property {binlogdata.ICharset|null} [charset] StreamKeyRangeRequest charset + * Properties of a VDiffOptions. + * @memberof tabletmanagerdata + * @interface IVDiffOptions + * @property {tabletmanagerdata.IVDiffPickerOptions|null} [picker_options] VDiffOptions picker_options + * @property {tabletmanagerdata.IVDiffCoreOptions|null} [core_options] VDiffOptions core_options + * @property {tabletmanagerdata.IVDiffReportOptions|null} [report_options] VDiffOptions report_options */ /** - * Constructs a new StreamKeyRangeRequest. - * @memberof binlogdata - * @classdesc Represents a StreamKeyRangeRequest. - * @implements IStreamKeyRangeRequest + * Constructs a new VDiffOptions. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffOptions. + * @implements IVDiffOptions * @constructor - * @param {binlogdata.IStreamKeyRangeRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set */ - function StreamKeyRangeRequest(properties) { + function VDiffOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -63822,103 +64427,103 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * StreamKeyRangeRequest position. - * @member {string} position - * @memberof binlogdata.StreamKeyRangeRequest + * VDiffOptions picker_options. + * @member {tabletmanagerdata.IVDiffPickerOptions|null|undefined} picker_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - StreamKeyRangeRequest.prototype.position = ""; + VDiffOptions.prototype.picker_options = null; /** - * StreamKeyRangeRequest key_range. - * @member {topodata.IKeyRange|null|undefined} key_range - * @memberof binlogdata.StreamKeyRangeRequest + * VDiffOptions core_options. + * @member {tabletmanagerdata.IVDiffCoreOptions|null|undefined} core_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - StreamKeyRangeRequest.prototype.key_range = null; + VDiffOptions.prototype.core_options = null; /** - * StreamKeyRangeRequest charset. - * @member {binlogdata.ICharset|null|undefined} charset - * @memberof binlogdata.StreamKeyRangeRequest + * VDiffOptions report_options. + * @member {tabletmanagerdata.IVDiffReportOptions|null|undefined} report_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - StreamKeyRangeRequest.prototype.charset = null; + VDiffOptions.prototype.report_options = null; /** - * Creates a new StreamKeyRangeRequest instance using the specified properties. + * Creates a new VDiffOptions instance using the specified properties. * @function create - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {binlogdata.IStreamKeyRangeRequest=} [properties] Properties to set - * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest instance + * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions instance */ - StreamKeyRangeRequest.create = function create(properties) { - return new StreamKeyRangeRequest(properties); + VDiffOptions.create = function create(properties) { + return new VDiffOptions(properties); }; /** - * Encodes the specified StreamKeyRangeRequest message. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. + * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. * @function encode - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {binlogdata.IStreamKeyRangeRequest} message StreamKeyRangeRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamKeyRangeRequest.encode = function encode(message, writer) { + VDiffOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); - if (message.key_range != null && Object.hasOwnProperty.call(message, "key_range")) - $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) - $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.picker_options != null && Object.hasOwnProperty.call(message, "picker_options")) + $root.tabletmanagerdata.VDiffPickerOptions.encode(message.picker_options, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.core_options != null && Object.hasOwnProperty.call(message, "core_options")) + $root.tabletmanagerdata.VDiffCoreOptions.encode(message.core_options, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.report_options != null && Object.hasOwnProperty.call(message, "report_options")) + $root.tabletmanagerdata.VDiffReportOptions.encode(message.report_options, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified StreamKeyRangeRequest message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. + * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {binlogdata.IStreamKeyRangeRequest} message StreamKeyRangeRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamKeyRangeRequest.encodeDelimited = function encodeDelimited(message, writer) { + VDiffOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamKeyRangeRequest message from the specified reader or buffer. + * Decodes a VDiffOptions message from the specified reader or buffer. * @function decode - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamKeyRangeRequest.decode = function decode(reader, length) { + VDiffOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamKeyRangeRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.position = reader.string(); + message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.decode(reader, reader.uint32()); break; } case 2: { - message.key_range = $root.topodata.KeyRange.decode(reader, reader.uint32()); + message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.decode(reader, reader.uint32()); break; } case 3: { - message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); + message.report_options = $root.tabletmanagerdata.VDiffReportOptions.decode(reader, reader.uint32()); break; } default: @@ -63930,149 +64535,161 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a StreamKeyRangeRequest message from the specified reader or buffer, length delimited. + * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamKeyRangeRequest.decodeDelimited = function decodeDelimited(reader) { + VDiffOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamKeyRangeRequest message. + * Verifies a VDiffOptions message. * @function verify - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamKeyRangeRequest.verify = function verify(message) { + VDiffOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; - if (message.key_range != null && message.hasOwnProperty("key_range")) { - let error = $root.topodata.KeyRange.verify(message.key_range); + if (message.picker_options != null && message.hasOwnProperty("picker_options")) { + let error = $root.tabletmanagerdata.VDiffPickerOptions.verify(message.picker_options); if (error) - return "key_range." + error; + return "picker_options." + error; } - if (message.charset != null && message.hasOwnProperty("charset")) { - let error = $root.binlogdata.Charset.verify(message.charset); + if (message.core_options != null && message.hasOwnProperty("core_options")) { + let error = $root.tabletmanagerdata.VDiffCoreOptions.verify(message.core_options); if (error) - return "charset." + error; + return "core_options." + error; + } + if (message.report_options != null && message.hasOwnProperty("report_options")) { + let error = $root.tabletmanagerdata.VDiffReportOptions.verify(message.report_options); + if (error) + return "report_options." + error; } return null; }; /** - * Creates a StreamKeyRangeRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {Object.} object Plain object - * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions */ - StreamKeyRangeRequest.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.StreamKeyRangeRequest) + VDiffOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffOptions) return object; - let message = new $root.binlogdata.StreamKeyRangeRequest(); - if (object.position != null) - message.position = String(object.position); - if (object.key_range != null) { - if (typeof object.key_range !== "object") - throw TypeError(".binlogdata.StreamKeyRangeRequest.key_range: object expected"); - message.key_range = $root.topodata.KeyRange.fromObject(object.key_range); + let message = new $root.tabletmanagerdata.VDiffOptions(); + if (object.picker_options != null) { + if (typeof object.picker_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.picker_options: object expected"); + message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.fromObject(object.picker_options); } - if (object.charset != null) { - if (typeof object.charset !== "object") - throw TypeError(".binlogdata.StreamKeyRangeRequest.charset: object expected"); - message.charset = $root.binlogdata.Charset.fromObject(object.charset); + if (object.core_options != null) { + if (typeof object.core_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.core_options: object expected"); + message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.fromObject(object.core_options); + } + if (object.report_options != null) { + if (typeof object.report_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.report_options: object expected"); + message.report_options = $root.tabletmanagerdata.VDiffReportOptions.fromObject(object.report_options); } return message; }; /** - * Creates a plain object from a StreamKeyRangeRequest message. Also converts values to other types if specified. + * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {binlogdata.StreamKeyRangeRequest} message StreamKeyRangeRequest + * @param {tabletmanagerdata.VDiffOptions} message VDiffOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamKeyRangeRequest.toObject = function toObject(message, options) { + VDiffOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.position = ""; - object.key_range = null; - object.charset = null; + object.picker_options = null; + object.core_options = null; + object.report_options = null; } - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; - if (message.key_range != null && message.hasOwnProperty("key_range")) - object.key_range = $root.topodata.KeyRange.toObject(message.key_range, options); - if (message.charset != null && message.hasOwnProperty("charset")) - object.charset = $root.binlogdata.Charset.toObject(message.charset, options); + if (message.picker_options != null && message.hasOwnProperty("picker_options")) + object.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.toObject(message.picker_options, options); + if (message.core_options != null && message.hasOwnProperty("core_options")) + object.core_options = $root.tabletmanagerdata.VDiffCoreOptions.toObject(message.core_options, options); + if (message.report_options != null && message.hasOwnProperty("report_options")) + object.report_options = $root.tabletmanagerdata.VDiffReportOptions.toObject(message.report_options, options); return object; }; /** - * Converts this StreamKeyRangeRequest to JSON. + * Converts this VDiffOptions to JSON. * @function toJSON - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @instance * @returns {Object.} JSON object */ - StreamKeyRangeRequest.prototype.toJSON = function toJSON() { + VDiffOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamKeyRangeRequest + * Gets the default type url for VDiffOptions * @function getTypeUrl - * @memberof binlogdata.StreamKeyRangeRequest + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamKeyRangeRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.StreamKeyRangeRequest"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffOptions"; }; - return StreamKeyRangeRequest; + return VDiffOptions; })(); - binlogdata.StreamKeyRangeResponse = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowRequest = (function() { /** - * Properties of a StreamKeyRangeResponse. - * @memberof binlogdata - * @interface IStreamKeyRangeResponse - * @property {binlogdata.IBinlogTransaction|null} [binlog_transaction] StreamKeyRangeResponse binlog_transaction + * Properties of an UpdateVReplicationWorkflowRequest. + * @memberof tabletmanagerdata + * @interface IUpdateVReplicationWorkflowRequest + * @property {string|null} [workflow] UpdateVReplicationWorkflowRequest workflow + * @property {Array.|null} [cells] UpdateVReplicationWorkflowRequest cells + * @property {Array.|null} [tablet_types] UpdateVReplicationWorkflowRequest tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] UpdateVReplicationWorkflowRequest tablet_selection_preference + * @property {binlogdata.OnDDLAction|null} [on_ddl] UpdateVReplicationWorkflowRequest on_ddl + * @property {binlogdata.VReplicationWorkflowState|null} [state] UpdateVReplicationWorkflowRequest state */ /** - * Constructs a new StreamKeyRangeResponse. - * @memberof binlogdata - * @classdesc Represents a StreamKeyRangeResponse. - * @implements IStreamKeyRangeResponse + * Constructs a new UpdateVReplicationWorkflowRequest. + * @memberof tabletmanagerdata + * @classdesc Represents an UpdateVReplicationWorkflowRequest. + * @implements IUpdateVReplicationWorkflowRequest * @constructor - * @param {binlogdata.IStreamKeyRangeResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set */ - function StreamKeyRangeResponse(properties) { + function UpdateVReplicationWorkflowRequest(properties) { + this.cells = []; + this.tablet_types = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -64080,75 +64697,159 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * StreamKeyRangeResponse binlog_transaction. - * @member {binlogdata.IBinlogTransaction|null|undefined} binlog_transaction - * @memberof binlogdata.StreamKeyRangeResponse + * UpdateVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - StreamKeyRangeResponse.prototype.binlog_transaction = null; + UpdateVReplicationWorkflowRequest.prototype.workflow = ""; /** - * Creates a new StreamKeyRangeResponse instance using the specified properties. + * UpdateVReplicationWorkflowRequest cells. + * @member {Array.} cells + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @instance + */ + UpdateVReplicationWorkflowRequest.prototype.cells = $util.emptyArray; + + /** + * UpdateVReplicationWorkflowRequest tablet_types. + * @member {Array.} tablet_types + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @instance + */ + UpdateVReplicationWorkflowRequest.prototype.tablet_types = $util.emptyArray; + + /** + * UpdateVReplicationWorkflowRequest tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @instance + */ + UpdateVReplicationWorkflowRequest.prototype.tablet_selection_preference = 0; + + /** + * UpdateVReplicationWorkflowRequest on_ddl. + * @member {binlogdata.OnDDLAction} on_ddl + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @instance + */ + UpdateVReplicationWorkflowRequest.prototype.on_ddl = 0; + + /** + * UpdateVReplicationWorkflowRequest state. + * @member {binlogdata.VReplicationWorkflowState} state + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @instance + */ + UpdateVReplicationWorkflowRequest.prototype.state = 0; + + /** + * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. * @function create - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {binlogdata.IStreamKeyRangeResponse=} [properties] Properties to set - * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest instance */ - StreamKeyRangeResponse.create = function create(properties) { - return new StreamKeyRangeResponse(properties); + UpdateVReplicationWorkflowRequest.create = function create(properties) { + return new UpdateVReplicationWorkflowRequest(properties); }; /** - * Encodes the specified StreamKeyRangeResponse message. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. * @function encode - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {binlogdata.IStreamKeyRangeResponse} message StreamKeyRangeResponse message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamKeyRangeResponse.encode = function encode(message, writer) { + UpdateVReplicationWorkflowRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.binlog_transaction != null && Object.hasOwnProperty.call(message, "binlog_transaction")) - $root.binlogdata.BinlogTransaction.encode(message.binlog_transaction, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 3, wireType 2 =*/26).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 4, wireType 0 =*/32).int32(message.tablet_selection_preference); + if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) + writer.uint32(/* id 5, wireType 0 =*/40).int32(message.on_ddl); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.state); return writer; }; /** - * Encodes the specified StreamKeyRangeResponse message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {binlogdata.IStreamKeyRangeResponse} message StreamKeyRangeResponse message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamKeyRangeResponse.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamKeyRangeResponse message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer. * @function decode - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamKeyRangeResponse.decode = function decode(reader, length) { + UpdateVReplicationWorkflowRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamKeyRangeResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.binlog_transaction = $root.binlogdata.BinlogTransaction.decode(reader, reader.uint32()); + message.workflow = reader.string(); + break; + } + case 2: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 3: { + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); + break; + } + case 4: { + message.tablet_selection_preference = reader.int32(); + break; + } + case 5: { + message.on_ddl = reader.int32(); + break; + } + case 6: { + message.state = reader.int32(); break; } default: @@ -64160,130 +64861,350 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a StreamKeyRangeResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamKeyRangeResponse.decodeDelimited = function decodeDelimited(reader) { + UpdateVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamKeyRangeResponse message. + * Verifies an UpdateVReplicationWorkflowRequest message. * @function verify - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamKeyRangeResponse.verify = function verify(message) { + UpdateVReplicationWorkflowRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) { - let error = $root.binlogdata.BinlogTransaction.verify(message.binlog_transaction); - if (error) - return "binlog_transaction." + error; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + switch (message.on_ddl) { + default: + return "on_ddl: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.state != null && message.hasOwnProperty("state")) + switch (message.state) { + default: + return "state: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } return null; }; /** - * Creates a StreamKeyRangeResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {Object.} object Plain object - * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest */ - StreamKeyRangeResponse.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.StreamKeyRangeResponse) + UpdateVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest) return object; - let message = new $root.binlogdata.StreamKeyRangeResponse(); - if (object.binlog_transaction != null) { - if (typeof object.binlog_transaction !== "object") - throw TypeError(".binlogdata.StreamKeyRangeResponse.binlog_transaction: object expected"); - message.binlog_transaction = $root.binlogdata.BinlogTransaction.fromObject(object.binlog_transaction); + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } + } + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; + break; + } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + switch (object.on_ddl) { + default: + if (typeof object.on_ddl === "number") { + message.on_ddl = object.on_ddl; + break; + } + break; + case "IGNORE": + case 0: + message.on_ddl = 0; + break; + case "STOP": + case 1: + message.on_ddl = 1; + break; + case "EXEC": + case 2: + message.on_ddl = 2; + break; + case "EXEC_IGNORE": + case 3: + message.on_ddl = 3; + break; + } + switch (object.state) { + default: + if (typeof object.state === "number") { + message.state = object.state; + break; + } + break; + case "Unknown": + case 0: + message.state = 0; + break; + case "Init": + case 1: + message.state = 1; + break; + case "Stopped": + case 2: + message.state = 2; + break; + case "Copying": + case 3: + message.state = 3; + break; + case "Running": + case 4: + message.state = 4; + break; + case "Error": + case 5: + message.state = 5; + break; + case "Lagging": + case 6: + message.state = 6; + break; } return message; }; /** - * Creates a plain object from a StreamKeyRangeResponse message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowRequest message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {binlogdata.StreamKeyRangeResponse} message StreamKeyRangeResponse + * @param {tabletmanagerdata.UpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamKeyRangeResponse.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.binlog_transaction = null; - if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) - object.binlog_transaction = $root.binlogdata.BinlogTransaction.toObject(message.binlog_transaction, options); + if (options.arrays || options.defaults) { + object.cells = []; + object.tablet_types = []; + } + if (options.defaults) { + object.workflow = ""; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.on_ddl = options.enums === String ? "IGNORE" : 0; + object.state = options.enums === String ? "Unknown" : 0; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; + if (message.state != null && message.hasOwnProperty("state")) + object.state = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.state] === undefined ? message.state : $root.binlogdata.VReplicationWorkflowState[message.state] : message.state; return object; }; /** - * Converts this StreamKeyRangeResponse to JSON. + * Converts this UpdateVReplicationWorkflowRequest to JSON. * @function toJSON - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance * @returns {Object.} JSON object */ - StreamKeyRangeResponse.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamKeyRangeResponse + * Gets the default type url for UpdateVReplicationWorkflowRequest * @function getTypeUrl - * @memberof binlogdata.StreamKeyRangeResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamKeyRangeResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.StreamKeyRangeResponse"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowRequest"; }; - return StreamKeyRangeResponse; + return UpdateVReplicationWorkflowRequest; })(); - binlogdata.StreamTablesRequest = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowResponse = (function() { /** - * Properties of a StreamTablesRequest. - * @memberof binlogdata - * @interface IStreamTablesRequest - * @property {string|null} [position] StreamTablesRequest position - * @property {Array.|null} [tables] StreamTablesRequest tables - * @property {binlogdata.ICharset|null} [charset] StreamTablesRequest charset + * Properties of an UpdateVReplicationWorkflowResponse. + * @memberof tabletmanagerdata + * @interface IUpdateVReplicationWorkflowResponse + * @property {query.IQueryResult|null} [result] UpdateVReplicationWorkflowResponse result */ /** - * Constructs a new StreamTablesRequest. - * @memberof binlogdata - * @classdesc Represents a StreamTablesRequest. - * @implements IStreamTablesRequest + * Constructs a new UpdateVReplicationWorkflowResponse. + * @memberof tabletmanagerdata + * @classdesc Represents an UpdateVReplicationWorkflowResponse. + * @implements IUpdateVReplicationWorkflowResponse * @constructor - * @param {binlogdata.IStreamTablesRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set */ - function StreamTablesRequest(properties) { - this.tables = []; + function UpdateVReplicationWorkflowResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -64291,106 +65212,75 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * StreamTablesRequest position. - * @member {string} position - * @memberof binlogdata.StreamTablesRequest - * @instance - */ - StreamTablesRequest.prototype.position = ""; - - /** - * StreamTablesRequest tables. - * @member {Array.} tables - * @memberof binlogdata.StreamTablesRequest - * @instance - */ - StreamTablesRequest.prototype.tables = $util.emptyArray; - - /** - * StreamTablesRequest charset. - * @member {binlogdata.ICharset|null|undefined} charset - * @memberof binlogdata.StreamTablesRequest + * UpdateVReplicationWorkflowResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @instance */ - StreamTablesRequest.prototype.charset = null; + UpdateVReplicationWorkflowResponse.prototype.result = null; /** - * Creates a new StreamTablesRequest instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowResponse instance using the specified properties. * @function create - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {binlogdata.IStreamTablesRequest=} [properties] Properties to set - * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse instance */ - StreamTablesRequest.create = function create(properties) { - return new StreamTablesRequest(properties); + UpdateVReplicationWorkflowResponse.create = function create(properties) { + return new UpdateVReplicationWorkflowResponse(properties); }; /** - * Encodes the specified StreamTablesRequest message. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. * @function encode - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {binlogdata.IStreamTablesRequest} message StreamTablesRequest message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamTablesRequest.encode = function encode(message, writer) { + UpdateVReplicationWorkflowResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.tables[i]); - if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) - $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified StreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {binlogdata.IStreamTablesRequest} message StreamTablesRequest message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamTablesRequest.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamTablesRequest message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer. * @function decode - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamTablesRequest.decode = function decode(reader, length) { + UpdateVReplicationWorkflowResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamTablesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.position = reader.string(); - break; - } - case 2: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); - break; - } - case 3: { - message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -64402,157 +65292,128 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a StreamTablesRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamTablesRequest.decodeDelimited = function decodeDelimited(reader) { + UpdateVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamTablesRequest message. + * Verifies an UpdateVReplicationWorkflowResponse message. * @function verify - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamTablesRequest.verify = function verify(message) { + UpdateVReplicationWorkflowResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; - } - if (message.charset != null && message.hasOwnProperty("charset")) { - let error = $root.binlogdata.Charset.verify(message.charset); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "charset." + error; + return "result." + error; } return null; }; /** - * Creates a StreamTablesRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {Object.} object Plain object - * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse */ - StreamTablesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.StreamTablesRequest) + UpdateVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse) return object; - let message = new $root.binlogdata.StreamTablesRequest(); - if (object.position != null) - message.position = String(object.position); - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".binlogdata.StreamTablesRequest.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); - } - if (object.charset != null) { - if (typeof object.charset !== "object") - throw TypeError(".binlogdata.StreamTablesRequest.charset: object expected"); - message.charset = $root.binlogdata.Charset.fromObject(object.charset); + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } return message; }; /** - * Creates a plain object from a StreamTablesRequest message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowResponse message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {binlogdata.StreamTablesRequest} message StreamTablesRequest + * @param {tabletmanagerdata.UpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamTablesRequest.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tables = []; - if (options.defaults) { - object.position = ""; - object.charset = null; - } - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; - } - if (message.charset != null && message.hasOwnProperty("charset")) - object.charset = $root.binlogdata.Charset.toObject(message.charset, options); + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this StreamTablesRequest to JSON. + * Converts this UpdateVReplicationWorkflowResponse to JSON. * @function toJSON - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @instance * @returns {Object.} JSON object */ - StreamTablesRequest.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamTablesRequest + * Gets the default type url for UpdateVReplicationWorkflowResponse * @function getTypeUrl - * @memberof binlogdata.StreamTablesRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamTablesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.StreamTablesRequest"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowResponse"; }; - return StreamTablesRequest; + return UpdateVReplicationWorkflowResponse; })(); - binlogdata.StreamTablesResponse = (function() { + tabletmanagerdata.ResetSequencesRequest = (function() { /** - * Properties of a StreamTablesResponse. - * @memberof binlogdata - * @interface IStreamTablesResponse - * @property {binlogdata.IBinlogTransaction|null} [binlog_transaction] StreamTablesResponse binlog_transaction + * Properties of a ResetSequencesRequest. + * @memberof tabletmanagerdata + * @interface IResetSequencesRequest + * @property {Array.|null} [tables] ResetSequencesRequest tables */ /** - * Constructs a new StreamTablesResponse. - * @memberof binlogdata - * @classdesc Represents a StreamTablesResponse. - * @implements IStreamTablesResponse + * Constructs a new ResetSequencesRequest. + * @memberof tabletmanagerdata + * @classdesc Represents a ResetSequencesRequest. + * @implements IResetSequencesRequest * @constructor - * @param {binlogdata.IStreamTablesResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IResetSequencesRequest=} [properties] Properties to set */ - function StreamTablesResponse(properties) { + function ResetSequencesRequest(properties) { + this.tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -64560,75 +65421,78 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * StreamTablesResponse binlog_transaction. - * @member {binlogdata.IBinlogTransaction|null|undefined} binlog_transaction - * @memberof binlogdata.StreamTablesResponse + * ResetSequencesRequest tables. + * @member {Array.} tables + * @memberof tabletmanagerdata.ResetSequencesRequest * @instance */ - StreamTablesResponse.prototype.binlog_transaction = null; + ResetSequencesRequest.prototype.tables = $util.emptyArray; /** - * Creates a new StreamTablesResponse instance using the specified properties. + * Creates a new ResetSequencesRequest instance using the specified properties. * @function create - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static - * @param {binlogdata.IStreamTablesResponse=} [properties] Properties to set - * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse instance + * @param {tabletmanagerdata.IResetSequencesRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest instance */ - StreamTablesResponse.create = function create(properties) { - return new StreamTablesResponse(properties); + ResetSequencesRequest.create = function create(properties) { + return new ResetSequencesRequest(properties); }; /** - * Encodes the specified StreamTablesResponse message. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. + * Encodes the specified ResetSequencesRequest message. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. * @function encode - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static - * @param {binlogdata.IStreamTablesResponse} message StreamTablesResponse message or plain object to encode + * @param {tabletmanagerdata.IResetSequencesRequest} message ResetSequencesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamTablesResponse.encode = function encode(message, writer) { + ResetSequencesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.binlog_transaction != null && Object.hasOwnProperty.call(message, "binlog_transaction")) - $root.binlogdata.BinlogTransaction.encode(message.binlog_transaction, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables[i]); return writer; }; /** - * Encodes the specified StreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. + * Encodes the specified ResetSequencesRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static - * @param {binlogdata.IStreamTablesResponse} message StreamTablesResponse message or plain object to encode + * @param {tabletmanagerdata.IResetSequencesRequest} message ResetSequencesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamTablesResponse.encodeDelimited = function encodeDelimited(message, writer) { + ResetSequencesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamTablesResponse message from the specified reader or buffer. + * Decodes a ResetSequencesRequest message from the specified reader or buffer. * @function decode - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse + * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamTablesResponse.decode = function decode(reader, length) { + ResetSequencesRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamTablesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ResetSequencesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.binlog_transaction = $root.binlogdata.BinlogTransaction.decode(reader, reader.uint32()); + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); break; } default: @@ -64640,128 +65504,133 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a StreamTablesResponse message from the specified reader or buffer, length delimited. + * Decodes a ResetSequencesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse + * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamTablesResponse.decodeDelimited = function decodeDelimited(reader) { + ResetSequencesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamTablesResponse message. + * Verifies a ResetSequencesRequest message. * @function verify - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamTablesResponse.verify = function verify(message) { + ResetSequencesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) { - let error = $root.binlogdata.BinlogTransaction.verify(message.binlog_transaction); - if (error) - return "binlog_transaction." + error; + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; } return null; }; /** - * Creates a StreamTablesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ResetSequencesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static * @param {Object.} object Plain object - * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse + * @returns {tabletmanagerdata.ResetSequencesRequest} ResetSequencesRequest */ - StreamTablesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.StreamTablesResponse) + ResetSequencesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ResetSequencesRequest) return object; - let message = new $root.binlogdata.StreamTablesResponse(); - if (object.binlog_transaction != null) { - if (typeof object.binlog_transaction !== "object") - throw TypeError(".binlogdata.StreamTablesResponse.binlog_transaction: object expected"); - message.binlog_transaction = $root.binlogdata.BinlogTransaction.fromObject(object.binlog_transaction); + let message = new $root.tabletmanagerdata.ResetSequencesRequest(); + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".tabletmanagerdata.ResetSequencesRequest.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); } return message; }; /** - * Creates a plain object from a StreamTablesResponse message. Also converts values to other types if specified. + * Creates a plain object from a ResetSequencesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static - * @param {binlogdata.StreamTablesResponse} message StreamTablesResponse + * @param {tabletmanagerdata.ResetSequencesRequest} message ResetSequencesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamTablesResponse.toObject = function toObject(message, options) { + ResetSequencesRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.binlog_transaction = null; - if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) - object.binlog_transaction = $root.binlogdata.BinlogTransaction.toObject(message.binlog_transaction, options); + if (options.arrays || options.defaults) + object.tables = []; + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } return object; }; /** - * Converts this StreamTablesResponse to JSON. + * Converts this ResetSequencesRequest to JSON. * @function toJSON - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @instance * @returns {Object.} JSON object */ - StreamTablesResponse.prototype.toJSON = function toJSON() { + ResetSequencesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamTablesResponse + * Gets the default type url for ResetSequencesRequest * @function getTypeUrl - * @memberof binlogdata.StreamTablesResponse + * @memberof tabletmanagerdata.ResetSequencesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamTablesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ResetSequencesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.StreamTablesResponse"; + return typeUrlPrefix + "/tabletmanagerdata.ResetSequencesRequest"; }; - return StreamTablesResponse; + return ResetSequencesRequest; })(); - binlogdata.CharsetConversion = (function() { + tabletmanagerdata.ResetSequencesResponse = (function() { /** - * Properties of a CharsetConversion. - * @memberof binlogdata - * @interface ICharsetConversion - * @property {string|null} [from_charset] CharsetConversion from_charset - * @property {string|null} [to_charset] CharsetConversion to_charset + * Properties of a ResetSequencesResponse. + * @memberof tabletmanagerdata + * @interface IResetSequencesResponse */ /** - * Constructs a new CharsetConversion. - * @memberof binlogdata - * @classdesc Represents a CharsetConversion. - * @implements ICharsetConversion + * Constructs a new ResetSequencesResponse. + * @memberof tabletmanagerdata + * @classdesc Represents a ResetSequencesResponse. + * @implements IResetSequencesResponse * @constructor - * @param {binlogdata.ICharsetConversion=} [properties] Properties to set + * @param {tabletmanagerdata.IResetSequencesResponse=} [properties] Properties to set */ - function CharsetConversion(properties) { + function ResetSequencesResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -64769,91 +65638,63 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * CharsetConversion from_charset. - * @member {string} from_charset - * @memberof binlogdata.CharsetConversion - * @instance - */ - CharsetConversion.prototype.from_charset = ""; - - /** - * CharsetConversion to_charset. - * @member {string} to_charset - * @memberof binlogdata.CharsetConversion - * @instance - */ - CharsetConversion.prototype.to_charset = ""; - - /** - * Creates a new CharsetConversion instance using the specified properties. + * Creates a new ResetSequencesResponse instance using the specified properties. * @function create - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static - * @param {binlogdata.ICharsetConversion=} [properties] Properties to set - * @returns {binlogdata.CharsetConversion} CharsetConversion instance + * @param {tabletmanagerdata.IResetSequencesResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse instance */ - CharsetConversion.create = function create(properties) { - return new CharsetConversion(properties); + ResetSequencesResponse.create = function create(properties) { + return new ResetSequencesResponse(properties); }; /** - * Encodes the specified CharsetConversion message. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. + * Encodes the specified ResetSequencesResponse message. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. * @function encode - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static - * @param {binlogdata.ICharsetConversion} message CharsetConversion message or plain object to encode + * @param {tabletmanagerdata.IResetSequencesResponse} message ResetSequencesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CharsetConversion.encode = function encode(message, writer) { + ResetSequencesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.from_charset != null && Object.hasOwnProperty.call(message, "from_charset")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.from_charset); - if (message.to_charset != null && Object.hasOwnProperty.call(message, "to_charset")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.to_charset); return writer; }; /** - * Encodes the specified CharsetConversion message, length delimited. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. + * Encodes the specified ResetSequencesResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ResetSequencesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static - * @param {binlogdata.ICharsetConversion} message CharsetConversion message or plain object to encode + * @param {tabletmanagerdata.IResetSequencesResponse} message ResetSequencesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CharsetConversion.encodeDelimited = function encodeDelimited(message, writer) { + ResetSequencesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CharsetConversion message from the specified reader or buffer. + * Decodes a ResetSequencesResponse message from the specified reader or buffer. * @function decode - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.CharsetConversion} CharsetConversion + * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CharsetConversion.decode = function decode(reader, length) { + ResetSequencesResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.CharsetConversion(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ResetSequencesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.from_charset = reader.string(); - break; - } - case 2: { - message.to_charset = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -64863,141 +65704,123 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a CharsetConversion message from the specified reader or buffer, length delimited. + * Decodes a ResetSequencesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.CharsetConversion} CharsetConversion + * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CharsetConversion.decodeDelimited = function decodeDelimited(reader) { + ResetSequencesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CharsetConversion message. + * Verifies a ResetSequencesResponse message. * @function verify - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CharsetConversion.verify = function verify(message) { + ResetSequencesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.from_charset != null && message.hasOwnProperty("from_charset")) - if (!$util.isString(message.from_charset)) - return "from_charset: string expected"; - if (message.to_charset != null && message.hasOwnProperty("to_charset")) - if (!$util.isString(message.to_charset)) - return "to_charset: string expected"; return null; }; /** - * Creates a CharsetConversion message from a plain object. Also converts values to their respective internal types. + * Creates a ResetSequencesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static * @param {Object.} object Plain object - * @returns {binlogdata.CharsetConversion} CharsetConversion + * @returns {tabletmanagerdata.ResetSequencesResponse} ResetSequencesResponse */ - CharsetConversion.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.CharsetConversion) + ResetSequencesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ResetSequencesResponse) return object; - let message = new $root.binlogdata.CharsetConversion(); - if (object.from_charset != null) - message.from_charset = String(object.from_charset); - if (object.to_charset != null) - message.to_charset = String(object.to_charset); - return message; + return new $root.tabletmanagerdata.ResetSequencesResponse(); }; /** - * Creates a plain object from a CharsetConversion message. Also converts values to other types if specified. + * Creates a plain object from a ResetSequencesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static - * @param {binlogdata.CharsetConversion} message CharsetConversion + * @param {tabletmanagerdata.ResetSequencesResponse} message ResetSequencesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CharsetConversion.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.from_charset = ""; - object.to_charset = ""; - } - if (message.from_charset != null && message.hasOwnProperty("from_charset")) - object.from_charset = message.from_charset; - if (message.to_charset != null && message.hasOwnProperty("to_charset")) - object.to_charset = message.to_charset; - return object; + ResetSequencesResponse.toObject = function toObject() { + return {}; }; /** - * Converts this CharsetConversion to JSON. + * Converts this ResetSequencesResponse to JSON. * @function toJSON - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @instance * @returns {Object.} JSON object */ - CharsetConversion.prototype.toJSON = function toJSON() { + ResetSequencesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CharsetConversion + * Gets the default type url for ResetSequencesResponse * @function getTypeUrl - * @memberof binlogdata.CharsetConversion + * @memberof tabletmanagerdata.ResetSequencesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CharsetConversion.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ResetSequencesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.CharsetConversion"; + return typeUrlPrefix + "/tabletmanagerdata.ResetSequencesResponse"; }; - return CharsetConversion; + return ResetSequencesResponse; })(); - binlogdata.Rule = (function() { + return tabletmanagerdata; +})(); + +export const binlogdata = $root.binlogdata = (() => { + + /** + * Namespace binlogdata. + * @exports binlogdata + * @namespace + */ + const binlogdata = {}; + + binlogdata.Charset = (function() { /** - * Properties of a Rule. + * Properties of a Charset. * @memberof binlogdata - * @interface IRule - * @property {string|null} [match] Rule match - * @property {string|null} [filter] Rule filter - * @property {Object.|null} [convert_enum_to_text] Rule convert_enum_to_text - * @property {Object.|null} [convert_charset] Rule convert_charset - * @property {string|null} [source_unique_key_columns] Rule source_unique_key_columns - * @property {string|null} [target_unique_key_columns] Rule target_unique_key_columns - * @property {string|null} [source_unique_key_target_columns] Rule source_unique_key_target_columns - * @property {Object.|null} [convert_int_to_enum] Rule convert_int_to_enum + * @interface ICharset + * @property {number|null} [client] Charset client + * @property {number|null} [conn] Charset conn + * @property {number|null} [server] Charset server */ /** - * Constructs a new Rule. + * Constructs a new Charset. * @memberof binlogdata - * @classdesc Represents a Rule. - * @implements IRule + * @classdesc Represents a Charset. + * @implements ICharset * @constructor - * @param {binlogdata.IRule=} [properties] Properties to set + * @param {binlogdata.ICharset=} [properties] Properties to set */ - function Rule(properties) { - this.convert_enum_to_text = {}; - this.convert_charset = {}; - this.convert_int_to_enum = {}; + function Charset(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -65005,235 +65828,103 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * Rule match. - * @member {string} match - * @memberof binlogdata.Rule - * @instance - */ - Rule.prototype.match = ""; - - /** - * Rule filter. - * @member {string} filter - * @memberof binlogdata.Rule - * @instance - */ - Rule.prototype.filter = ""; - - /** - * Rule convert_enum_to_text. - * @member {Object.} convert_enum_to_text - * @memberof binlogdata.Rule - * @instance - */ - Rule.prototype.convert_enum_to_text = $util.emptyObject; - - /** - * Rule convert_charset. - * @member {Object.} convert_charset - * @memberof binlogdata.Rule - * @instance - */ - Rule.prototype.convert_charset = $util.emptyObject; - - /** - * Rule source_unique_key_columns. - * @member {string} source_unique_key_columns - * @memberof binlogdata.Rule - * @instance - */ - Rule.prototype.source_unique_key_columns = ""; - - /** - * Rule target_unique_key_columns. - * @member {string} target_unique_key_columns - * @memberof binlogdata.Rule + * Charset client. + * @member {number} client + * @memberof binlogdata.Charset * @instance */ - Rule.prototype.target_unique_key_columns = ""; + Charset.prototype.client = 0; /** - * Rule source_unique_key_target_columns. - * @member {string} source_unique_key_target_columns - * @memberof binlogdata.Rule + * Charset conn. + * @member {number} conn + * @memberof binlogdata.Charset * @instance */ - Rule.prototype.source_unique_key_target_columns = ""; + Charset.prototype.conn = 0; /** - * Rule convert_int_to_enum. - * @member {Object.} convert_int_to_enum - * @memberof binlogdata.Rule + * Charset server. + * @member {number} server + * @memberof binlogdata.Charset * @instance */ - Rule.prototype.convert_int_to_enum = $util.emptyObject; + Charset.prototype.server = 0; /** - * Creates a new Rule instance using the specified properties. + * Creates a new Charset instance using the specified properties. * @function create - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static - * @param {binlogdata.IRule=} [properties] Properties to set - * @returns {binlogdata.Rule} Rule instance + * @param {binlogdata.ICharset=} [properties] Properties to set + * @returns {binlogdata.Charset} Charset instance */ - Rule.create = function create(properties) { - return new Rule(properties); + Charset.create = function create(properties) { + return new Charset(properties); }; /** - * Encodes the specified Rule message. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. + * Encodes the specified Charset message. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. * @function encode - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static - * @param {binlogdata.IRule} message Rule message or plain object to encode + * @param {binlogdata.ICharset} message Charset message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Rule.encode = function encode(message, writer) { + Charset.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.match != null && Object.hasOwnProperty.call(message, "match")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.match); - if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.filter); - if (message.convert_enum_to_text != null && Object.hasOwnProperty.call(message, "convert_enum_to_text")) - for (let keys = Object.keys(message.convert_enum_to_text), i = 0; i < keys.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.convert_enum_to_text[keys[i]]).ldelim(); - if (message.convert_charset != null && Object.hasOwnProperty.call(message, "convert_charset")) - for (let keys = Object.keys(message.convert_charset), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 4, wireType 2 =*/34).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.binlogdata.CharsetConversion.encode(message.convert_charset[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.source_unique_key_columns != null && Object.hasOwnProperty.call(message, "source_unique_key_columns")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_unique_key_columns); - if (message.target_unique_key_columns != null && Object.hasOwnProperty.call(message, "target_unique_key_columns")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.target_unique_key_columns); - if (message.source_unique_key_target_columns != null && Object.hasOwnProperty.call(message, "source_unique_key_target_columns")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.source_unique_key_target_columns); - if (message.convert_int_to_enum != null && Object.hasOwnProperty.call(message, "convert_int_to_enum")) - for (let keys = Object.keys(message.convert_int_to_enum), i = 0; i < keys.length; ++i) - writer.uint32(/* id 8, wireType 2 =*/66).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 0 =*/16).bool(message.convert_int_to_enum[keys[i]]).ldelim(); + if (message.client != null && Object.hasOwnProperty.call(message, "client")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.client); + if (message.conn != null && Object.hasOwnProperty.call(message, "conn")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.conn); + if (message.server != null && Object.hasOwnProperty.call(message, "server")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.server); return writer; }; /** - * Encodes the specified Rule message, length delimited. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. + * Encodes the specified Charset message, length delimited. Does not implicitly {@link binlogdata.Charset.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static - * @param {binlogdata.IRule} message Rule message or plain object to encode + * @param {binlogdata.ICharset} message Charset message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Rule.encodeDelimited = function encodeDelimited(message, writer) { + Charset.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Rule message from the specified reader or buffer. + * Decodes a Charset message from the specified reader or buffer. * @function decode - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.Rule} Rule + * @returns {binlogdata.Charset} Charset * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Rule.decode = function decode(reader, length) { + Charset.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Rule(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Charset(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.match = reader.string(); + message.client = reader.int32(); break; } case 2: { - message.filter = reader.string(); + message.conn = reader.int32(); break; } case 3: { - if (message.convert_enum_to_text === $util.emptyObject) - message.convert_enum_to_text = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = ""; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = reader.string(); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.convert_enum_to_text[key] = value; - break; - } - case 4: { - if (message.convert_charset === $util.emptyObject) - message.convert_charset = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.binlogdata.CharsetConversion.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.convert_charset[key] = value; - break; - } - case 5: { - message.source_unique_key_columns = reader.string(); - break; - } - case 6: { - message.target_unique_key_columns = reader.string(); - break; - } - case 7: { - message.source_unique_key_target_columns = reader.string(); - break; - } - case 8: { - if (message.convert_int_to_enum === $util.emptyObject) - message.convert_int_to_enum = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = false; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = reader.bool(); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.convert_int_to_enum[key] = value; + message.server = reader.int32(); break; } default: @@ -65245,230 +65936,141 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a Rule message from the specified reader or buffer, length delimited. + * Decodes a Charset message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.Rule} Rule + * @returns {binlogdata.Charset} Charset * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Rule.decodeDelimited = function decodeDelimited(reader) { + Charset.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Rule message. + * Verifies a Charset message. * @function verify - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Rule.verify = function verify(message) { + Charset.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.match != null && message.hasOwnProperty("match")) - if (!$util.isString(message.match)) - return "match: string expected"; - if (message.filter != null && message.hasOwnProperty("filter")) - if (!$util.isString(message.filter)) - return "filter: string expected"; - if (message.convert_enum_to_text != null && message.hasOwnProperty("convert_enum_to_text")) { - if (!$util.isObject(message.convert_enum_to_text)) - return "convert_enum_to_text: object expected"; - let key = Object.keys(message.convert_enum_to_text); - for (let i = 0; i < key.length; ++i) - if (!$util.isString(message.convert_enum_to_text[key[i]])) - return "convert_enum_to_text: string{k:string} expected"; - } - if (message.convert_charset != null && message.hasOwnProperty("convert_charset")) { - if (!$util.isObject(message.convert_charset)) - return "convert_charset: object expected"; - let key = Object.keys(message.convert_charset); - for (let i = 0; i < key.length; ++i) { - let error = $root.binlogdata.CharsetConversion.verify(message.convert_charset[key[i]]); - if (error) - return "convert_charset." + error; - } - } - if (message.source_unique_key_columns != null && message.hasOwnProperty("source_unique_key_columns")) - if (!$util.isString(message.source_unique_key_columns)) - return "source_unique_key_columns: string expected"; - if (message.target_unique_key_columns != null && message.hasOwnProperty("target_unique_key_columns")) - if (!$util.isString(message.target_unique_key_columns)) - return "target_unique_key_columns: string expected"; - if (message.source_unique_key_target_columns != null && message.hasOwnProperty("source_unique_key_target_columns")) - if (!$util.isString(message.source_unique_key_target_columns)) - return "source_unique_key_target_columns: string expected"; - if (message.convert_int_to_enum != null && message.hasOwnProperty("convert_int_to_enum")) { - if (!$util.isObject(message.convert_int_to_enum)) - return "convert_int_to_enum: object expected"; - let key = Object.keys(message.convert_int_to_enum); - for (let i = 0; i < key.length; ++i) - if (typeof message.convert_int_to_enum[key[i]] !== "boolean") - return "convert_int_to_enum: boolean{k:string} expected"; - } + if (message.client != null && message.hasOwnProperty("client")) + if (!$util.isInteger(message.client)) + return "client: integer expected"; + if (message.conn != null && message.hasOwnProperty("conn")) + if (!$util.isInteger(message.conn)) + return "conn: integer expected"; + if (message.server != null && message.hasOwnProperty("server")) + if (!$util.isInteger(message.server)) + return "server: integer expected"; return null; }; /** - * Creates a Rule message from a plain object. Also converts values to their respective internal types. + * Creates a Charset message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static * @param {Object.} object Plain object - * @returns {binlogdata.Rule} Rule + * @returns {binlogdata.Charset} Charset */ - Rule.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.Rule) + Charset.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.Charset) return object; - let message = new $root.binlogdata.Rule(); - if (object.match != null) - message.match = String(object.match); - if (object.filter != null) - message.filter = String(object.filter); - if (object.convert_enum_to_text) { - if (typeof object.convert_enum_to_text !== "object") - throw TypeError(".binlogdata.Rule.convert_enum_to_text: object expected"); - message.convert_enum_to_text = {}; - for (let keys = Object.keys(object.convert_enum_to_text), i = 0; i < keys.length; ++i) - message.convert_enum_to_text[keys[i]] = String(object.convert_enum_to_text[keys[i]]); - } - if (object.convert_charset) { - if (typeof object.convert_charset !== "object") - throw TypeError(".binlogdata.Rule.convert_charset: object expected"); - message.convert_charset = {}; - for (let keys = Object.keys(object.convert_charset), i = 0; i < keys.length; ++i) { - if (typeof object.convert_charset[keys[i]] !== "object") - throw TypeError(".binlogdata.Rule.convert_charset: object expected"); - message.convert_charset[keys[i]] = $root.binlogdata.CharsetConversion.fromObject(object.convert_charset[keys[i]]); - } - } - if (object.source_unique_key_columns != null) - message.source_unique_key_columns = String(object.source_unique_key_columns); - if (object.target_unique_key_columns != null) - message.target_unique_key_columns = String(object.target_unique_key_columns); - if (object.source_unique_key_target_columns != null) - message.source_unique_key_target_columns = String(object.source_unique_key_target_columns); - if (object.convert_int_to_enum) { - if (typeof object.convert_int_to_enum !== "object") - throw TypeError(".binlogdata.Rule.convert_int_to_enum: object expected"); - message.convert_int_to_enum = {}; - for (let keys = Object.keys(object.convert_int_to_enum), i = 0; i < keys.length; ++i) - message.convert_int_to_enum[keys[i]] = Boolean(object.convert_int_to_enum[keys[i]]); - } + let message = new $root.binlogdata.Charset(); + if (object.client != null) + message.client = object.client | 0; + if (object.conn != null) + message.conn = object.conn | 0; + if (object.server != null) + message.server = object.server | 0; return message; }; /** - * Creates a plain object from a Rule message. Also converts values to other types if specified. + * Creates a plain object from a Charset message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static - * @param {binlogdata.Rule} message Rule + * @param {binlogdata.Charset} message Charset * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Rule.toObject = function toObject(message, options) { + Charset.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) { - object.convert_enum_to_text = {}; - object.convert_charset = {}; - object.convert_int_to_enum = {}; - } if (options.defaults) { - object.match = ""; - object.filter = ""; - object.source_unique_key_columns = ""; - object.target_unique_key_columns = ""; - object.source_unique_key_target_columns = ""; - } - if (message.match != null && message.hasOwnProperty("match")) - object.match = message.match; - if (message.filter != null && message.hasOwnProperty("filter")) - object.filter = message.filter; - let keys2; - if (message.convert_enum_to_text && (keys2 = Object.keys(message.convert_enum_to_text)).length) { - object.convert_enum_to_text = {}; - for (let j = 0; j < keys2.length; ++j) - object.convert_enum_to_text[keys2[j]] = message.convert_enum_to_text[keys2[j]]; - } - if (message.convert_charset && (keys2 = Object.keys(message.convert_charset)).length) { - object.convert_charset = {}; - for (let j = 0; j < keys2.length; ++j) - object.convert_charset[keys2[j]] = $root.binlogdata.CharsetConversion.toObject(message.convert_charset[keys2[j]], options); - } - if (message.source_unique_key_columns != null && message.hasOwnProperty("source_unique_key_columns")) - object.source_unique_key_columns = message.source_unique_key_columns; - if (message.target_unique_key_columns != null && message.hasOwnProperty("target_unique_key_columns")) - object.target_unique_key_columns = message.target_unique_key_columns; - if (message.source_unique_key_target_columns != null && message.hasOwnProperty("source_unique_key_target_columns")) - object.source_unique_key_target_columns = message.source_unique_key_target_columns; - if (message.convert_int_to_enum && (keys2 = Object.keys(message.convert_int_to_enum)).length) { - object.convert_int_to_enum = {}; - for (let j = 0; j < keys2.length; ++j) - object.convert_int_to_enum[keys2[j]] = message.convert_int_to_enum[keys2[j]]; + object.client = 0; + object.conn = 0; + object.server = 0; } + if (message.client != null && message.hasOwnProperty("client")) + object.client = message.client; + if (message.conn != null && message.hasOwnProperty("conn")) + object.conn = message.conn; + if (message.server != null && message.hasOwnProperty("server")) + object.server = message.server; return object; }; /** - * Converts this Rule to JSON. + * Converts this Charset to JSON. * @function toJSON - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @instance * @returns {Object.} JSON object */ - Rule.prototype.toJSON = function toJSON() { + Charset.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Rule + * Gets the default type url for Charset * @function getTypeUrl - * @memberof binlogdata.Rule + * @memberof binlogdata.Charset * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Rule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Charset.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.Rule"; + return typeUrlPrefix + "/binlogdata.Charset"; }; - return Rule; + return Charset; })(); - binlogdata.Filter = (function() { + binlogdata.BinlogTransaction = (function() { /** - * Properties of a Filter. + * Properties of a BinlogTransaction. * @memberof binlogdata - * @interface IFilter - * @property {Array.|null} [rules] Filter rules - * @property {binlogdata.Filter.FieldEventMode|null} [field_event_mode] Filter field_event_mode - * @property {number|Long|null} [workflow_type] Filter workflow_type - * @property {string|null} [workflow_name] Filter workflow_name + * @interface IBinlogTransaction + * @property {Array.|null} [statements] BinlogTransaction statements + * @property {query.IEventToken|null} [event_token] BinlogTransaction event_token */ /** - * Constructs a new Filter. + * Constructs a new BinlogTransaction. * @memberof binlogdata - * @classdesc Represents a Filter. - * @implements IFilter + * @classdesc Represents a BinlogTransaction. + * @implements IBinlogTransaction * @constructor - * @param {binlogdata.IFilter=} [properties] Properties to set + * @param {binlogdata.IBinlogTransaction=} [properties] Properties to set */ - function Filter(properties) { - this.rules = []; + function BinlogTransaction(properties) { + this.statements = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -65476,120 +66078,92 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * Filter rules. - * @member {Array.} rules - * @memberof binlogdata.Filter - * @instance - */ - Filter.prototype.rules = $util.emptyArray; - - /** - * Filter field_event_mode. - * @member {binlogdata.Filter.FieldEventMode} field_event_mode - * @memberof binlogdata.Filter - * @instance - */ - Filter.prototype.field_event_mode = 0; - - /** - * Filter workflow_type. - * @member {number|Long} workflow_type - * @memberof binlogdata.Filter + * BinlogTransaction statements. + * @member {Array.} statements + * @memberof binlogdata.BinlogTransaction * @instance */ - Filter.prototype.workflow_type = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + BinlogTransaction.prototype.statements = $util.emptyArray; /** - * Filter workflow_name. - * @member {string} workflow_name - * @memberof binlogdata.Filter + * BinlogTransaction event_token. + * @member {query.IEventToken|null|undefined} event_token + * @memberof binlogdata.BinlogTransaction * @instance */ - Filter.prototype.workflow_name = ""; + BinlogTransaction.prototype.event_token = null; /** - * Creates a new Filter instance using the specified properties. + * Creates a new BinlogTransaction instance using the specified properties. * @function create - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static - * @param {binlogdata.IFilter=} [properties] Properties to set - * @returns {binlogdata.Filter} Filter instance + * @param {binlogdata.IBinlogTransaction=} [properties] Properties to set + * @returns {binlogdata.BinlogTransaction} BinlogTransaction instance */ - Filter.create = function create(properties) { - return new Filter(properties); + BinlogTransaction.create = function create(properties) { + return new BinlogTransaction(properties); }; /** - * Encodes the specified Filter message. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. + * Encodes the specified BinlogTransaction message. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. * @function encode - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static - * @param {binlogdata.IFilter} message Filter message or plain object to encode + * @param {binlogdata.IBinlogTransaction} message BinlogTransaction message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Filter.encode = function encode(message, writer) { + BinlogTransaction.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.binlogdata.Rule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.field_event_mode != null && Object.hasOwnProperty.call(message, "field_event_mode")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.field_event_mode); - if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.workflow_type); - if (message.workflow_name != null && Object.hasOwnProperty.call(message, "workflow_name")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.workflow_name); + if (message.statements != null && message.statements.length) + for (let i = 0; i < message.statements.length; ++i) + $root.binlogdata.BinlogTransaction.Statement.encode(message.statements[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.event_token != null && Object.hasOwnProperty.call(message, "event_token")) + $root.query.EventToken.encode(message.event_token, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified Filter message, length delimited. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. + * Encodes the specified BinlogTransaction message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static - * @param {binlogdata.IFilter} message Filter message or plain object to encode + * @param {binlogdata.IBinlogTransaction} message BinlogTransaction message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Filter.encodeDelimited = function encodeDelimited(message, writer) { + BinlogTransaction.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Filter message from the specified reader or buffer. + * Decodes a BinlogTransaction message from the specified reader or buffer. * @function decode - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.Filter} Filter + * @returns {binlogdata.BinlogTransaction} BinlogTransaction * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Filter.decode = function decode(reader, length) { + BinlogTransaction.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Filter(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogTransaction(); 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.binlogdata.Rule.decode(reader, reader.uint32())); - break; - } - case 2: { - message.field_event_mode = reader.int32(); - break; - } - case 3: { - message.workflow_type = reader.int64(); + if (!(message.statements && message.statements.length)) + message.statements = []; + message.statements.push($root.binlogdata.BinlogTransaction.Statement.decode(reader, reader.uint32())); break; } case 4: { - message.workflow_name = reader.string(); + message.event_token = $root.query.EventToken.decode(reader, reader.uint32()); break; } default: @@ -65601,511 +66175,612 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a Filter message from the specified reader or buffer, length delimited. + * Decodes a BinlogTransaction message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.Filter} Filter + * @returns {binlogdata.BinlogTransaction} BinlogTransaction * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Filter.decodeDelimited = function decodeDelimited(reader) { + BinlogTransaction.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Filter message. + * Verifies a BinlogTransaction message. * @function verify - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Filter.verify = function verify(message) { + BinlogTransaction.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.binlogdata.Rule.verify(message.rules[i]); + if (message.statements != null && message.hasOwnProperty("statements")) { + if (!Array.isArray(message.statements)) + return "statements: array expected"; + for (let i = 0; i < message.statements.length; ++i) { + let error = $root.binlogdata.BinlogTransaction.Statement.verify(message.statements[i]); if (error) - return "rules." + error; + return "statements." + error; } } - if (message.field_event_mode != null && message.hasOwnProperty("field_event_mode")) - switch (message.field_event_mode) { - default: - return "field_event_mode: enum value expected"; - case 0: - case 1: - break; - } - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - if (!$util.isInteger(message.workflow_type) && !(message.workflow_type && $util.isInteger(message.workflow_type.low) && $util.isInteger(message.workflow_type.high))) - return "workflow_type: integer|Long expected"; - if (message.workflow_name != null && message.hasOwnProperty("workflow_name")) - if (!$util.isString(message.workflow_name)) - return "workflow_name: string expected"; + if (message.event_token != null && message.hasOwnProperty("event_token")) { + let error = $root.query.EventToken.verify(message.event_token); + if (error) + return "event_token." + error; + } return null; }; /** - * Creates a Filter message from a plain object. Also converts values to their respective internal types. + * Creates a BinlogTransaction message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static * @param {Object.} object Plain object - * @returns {binlogdata.Filter} Filter + * @returns {binlogdata.BinlogTransaction} BinlogTransaction */ - Filter.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.Filter) + BinlogTransaction.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.BinlogTransaction) return object; - let message = new $root.binlogdata.Filter(); - if (object.rules) { - if (!Array.isArray(object.rules)) - throw TypeError(".binlogdata.Filter.rules: array expected"); - message.rules = []; - for (let i = 0; i < object.rules.length; ++i) { - if (typeof object.rules[i] !== "object") - throw TypeError(".binlogdata.Filter.rules: object expected"); - message.rules[i] = $root.binlogdata.Rule.fromObject(object.rules[i]); + let message = new $root.binlogdata.BinlogTransaction(); + if (object.statements) { + if (!Array.isArray(object.statements)) + throw TypeError(".binlogdata.BinlogTransaction.statements: array expected"); + message.statements = []; + for (let i = 0; i < object.statements.length; ++i) { + if (typeof object.statements[i] !== "object") + throw TypeError(".binlogdata.BinlogTransaction.statements: object expected"); + message.statements[i] = $root.binlogdata.BinlogTransaction.Statement.fromObject(object.statements[i]); } } - switch (object.field_event_mode) { - default: - if (typeof object.field_event_mode === "number") { - message.field_event_mode = object.field_event_mode; - break; - } - break; - case "ERR_ON_MISMATCH": - case 0: - message.field_event_mode = 0; - break; - case "BEST_EFFORT": - case 1: - message.field_event_mode = 1; - break; + if (object.event_token != null) { + if (typeof object.event_token !== "object") + throw TypeError(".binlogdata.BinlogTransaction.event_token: object expected"); + message.event_token = $root.query.EventToken.fromObject(object.event_token); } - if (object.workflow_type != null) - if ($util.Long) - (message.workflow_type = $util.Long.fromValue(object.workflow_type)).unsigned = false; - else if (typeof object.workflow_type === "string") - message.workflow_type = parseInt(object.workflow_type, 10); - else if (typeof object.workflow_type === "number") - message.workflow_type = object.workflow_type; - else if (typeof object.workflow_type === "object") - message.workflow_type = new $util.LongBits(object.workflow_type.low >>> 0, object.workflow_type.high >>> 0).toNumber(); - if (object.workflow_name != null) - message.workflow_name = String(object.workflow_name); return message; }; /** - * Creates a plain object from a Filter message. Also converts values to other types if specified. + * Creates a plain object from a BinlogTransaction message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static - * @param {binlogdata.Filter} message Filter + * @param {binlogdata.BinlogTransaction} message BinlogTransaction * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Filter.toObject = function toObject(message, options) { + BinlogTransaction.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.rules = []; - if (options.defaults) { - object.field_event_mode = options.enums === String ? "ERR_ON_MISMATCH" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.workflow_type = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.workflow_type = options.longs === String ? "0" : 0; - object.workflow_name = ""; - } - if (message.rules && message.rules.length) { - object.rules = []; - for (let j = 0; j < message.rules.length; ++j) - object.rules[j] = $root.binlogdata.Rule.toObject(message.rules[j], options); + object.statements = []; + if (options.defaults) + object.event_token = null; + if (message.statements && message.statements.length) { + object.statements = []; + for (let j = 0; j < message.statements.length; ++j) + object.statements[j] = $root.binlogdata.BinlogTransaction.Statement.toObject(message.statements[j], options); } - if (message.field_event_mode != null && message.hasOwnProperty("field_event_mode")) - object.field_event_mode = options.enums === String ? $root.binlogdata.Filter.FieldEventMode[message.field_event_mode] === undefined ? message.field_event_mode : $root.binlogdata.Filter.FieldEventMode[message.field_event_mode] : message.field_event_mode; - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - if (typeof message.workflow_type === "number") - object.workflow_type = options.longs === String ? String(message.workflow_type) : message.workflow_type; - else - object.workflow_type = options.longs === String ? $util.Long.prototype.toString.call(message.workflow_type) : options.longs === Number ? new $util.LongBits(message.workflow_type.low >>> 0, message.workflow_type.high >>> 0).toNumber() : message.workflow_type; - if (message.workflow_name != null && message.hasOwnProperty("workflow_name")) - object.workflow_name = message.workflow_name; + if (message.event_token != null && message.hasOwnProperty("event_token")) + object.event_token = $root.query.EventToken.toObject(message.event_token, options); return object; }; /** - * Converts this Filter to JSON. + * Converts this BinlogTransaction to JSON. * @function toJSON - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @instance * @returns {Object.} JSON object */ - Filter.prototype.toJSON = function toJSON() { + BinlogTransaction.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Filter + * Gets the default type url for BinlogTransaction * @function getTypeUrl - * @memberof binlogdata.Filter + * @memberof binlogdata.BinlogTransaction * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Filter.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BinlogTransaction.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.Filter"; + return typeUrlPrefix + "/binlogdata.BinlogTransaction"; }; - /** - * FieldEventMode enum. - * @name binlogdata.Filter.FieldEventMode - * @enum {number} - * @property {number} ERR_ON_MISMATCH=0 ERR_ON_MISMATCH value - * @property {number} BEST_EFFORT=1 BEST_EFFORT value - */ - Filter.FieldEventMode = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "ERR_ON_MISMATCH"] = 0; - values[valuesById[1] = "BEST_EFFORT"] = 1; - return values; - })(); + BinlogTransaction.Statement = (function() { - return Filter; - })(); + /** + * Properties of a Statement. + * @memberof binlogdata.BinlogTransaction + * @interface IStatement + * @property {binlogdata.BinlogTransaction.Statement.Category|null} [category] Statement category + * @property {binlogdata.ICharset|null} [charset] Statement charset + * @property {Uint8Array|null} [sql] Statement sql + */ - /** - * OnDDLAction enum. - * @name binlogdata.OnDDLAction - * @enum {number} - * @property {number} IGNORE=0 IGNORE value - * @property {number} STOP=1 STOP value - * @property {number} EXEC=2 EXEC value - * @property {number} EXEC_IGNORE=3 EXEC_IGNORE value - */ - binlogdata.OnDDLAction = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "IGNORE"] = 0; - values[valuesById[1] = "STOP"] = 1; - values[valuesById[2] = "EXEC"] = 2; - values[valuesById[3] = "EXEC_IGNORE"] = 3; - return values; - })(); + /** + * Constructs a new Statement. + * @memberof binlogdata.BinlogTransaction + * @classdesc Represents a Statement. + * @implements IStatement + * @constructor + * @param {binlogdata.BinlogTransaction.IStatement=} [properties] Properties to set + */ + function Statement(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]]; + } - /** - * VReplicationWorkflowType enum. - * @name binlogdata.VReplicationWorkflowType - * @enum {number} - * @property {number} Materialize=0 Materialize value - * @property {number} MoveTables=1 MoveTables value - * @property {number} CreateLookupIndex=2 CreateLookupIndex value - * @property {number} Migrate=3 Migrate value - * @property {number} Reshard=4 Reshard value - * @property {number} OnlineDDL=5 OnlineDDL value - */ - binlogdata.VReplicationWorkflowType = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "Materialize"] = 0; - values[valuesById[1] = "MoveTables"] = 1; - values[valuesById[2] = "CreateLookupIndex"] = 2; - values[valuesById[3] = "Migrate"] = 3; - values[valuesById[4] = "Reshard"] = 4; - values[valuesById[5] = "OnlineDDL"] = 5; - return values; - })(); + /** + * Statement category. + * @member {binlogdata.BinlogTransaction.Statement.Category} category + * @memberof binlogdata.BinlogTransaction.Statement + * @instance + */ + Statement.prototype.category = 0; - /** - * VReplicationWorkflowSubType enum. - * @name binlogdata.VReplicationWorkflowSubType - * @enum {number} - * @property {number} None=0 None value - * @property {number} Partial=1 Partial value - */ - binlogdata.VReplicationWorkflowSubType = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "None"] = 0; - values[valuesById[1] = "Partial"] = 1; - return values; - })(); + /** + * Statement charset. + * @member {binlogdata.ICharset|null|undefined} charset + * @memberof binlogdata.BinlogTransaction.Statement + * @instance + */ + Statement.prototype.charset = null; - binlogdata.BinlogSource = (function() { + /** + * Statement sql. + * @member {Uint8Array} sql + * @memberof binlogdata.BinlogTransaction.Statement + * @instance + */ + Statement.prototype.sql = $util.newBuffer([]); - /** - * Properties of a BinlogSource. - * @memberof binlogdata - * @interface IBinlogSource - * @property {string|null} [keyspace] BinlogSource keyspace - * @property {string|null} [shard] BinlogSource shard - * @property {topodata.TabletType|null} [tablet_type] BinlogSource tablet_type - * @property {topodata.IKeyRange|null} [key_range] BinlogSource key_range - * @property {Array.|null} [tables] BinlogSource tables - * @property {binlogdata.IFilter|null} [filter] BinlogSource filter - * @property {binlogdata.OnDDLAction|null} [on_ddl] BinlogSource on_ddl - * @property {string|null} [external_mysql] BinlogSource external_mysql - * @property {boolean|null} [stop_after_copy] BinlogSource stop_after_copy - * @property {string|null} [external_cluster] BinlogSource external_cluster - * @property {string|null} [source_time_zone] BinlogSource source_time_zone - * @property {string|null} [target_time_zone] BinlogSource target_time_zone - */ + /** + * Creates a new Statement instance using the specified properties. + * @function create + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {binlogdata.BinlogTransaction.IStatement=} [properties] Properties to set + * @returns {binlogdata.BinlogTransaction.Statement} Statement instance + */ + Statement.create = function create(properties) { + return new Statement(properties); + }; - /** - * Constructs a new BinlogSource. - * @memberof binlogdata - * @classdesc Represents a BinlogSource. - * @implements IBinlogSource - * @constructor - * @param {binlogdata.IBinlogSource=} [properties] Properties to set - */ - function BinlogSource(properties) { - this.tables = []; - 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]]; - } + /** + * Encodes the specified Statement message. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. + * @function encode + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {binlogdata.BinlogTransaction.IStatement} message Statement message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Statement.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.category != null && Object.hasOwnProperty.call(message, "category")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.category); + if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) + $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) + writer.uint32(/* id 3, wireType 2 =*/26).bytes(message.sql); + return writer; + }; - /** - * BinlogSource keyspace. - * @member {string} keyspace - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.keyspace = ""; + /** + * Encodes the specified Statement message, length delimited. Does not implicitly {@link binlogdata.BinlogTransaction.Statement.verify|verify} messages. + * @function encodeDelimited + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {binlogdata.BinlogTransaction.IStatement} message Statement message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Statement.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * BinlogSource shard. - * @member {string} shard - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.shard = ""; + /** + * Decodes a Statement message from the specified reader or buffer. + * @function decode + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {binlogdata.BinlogTransaction.Statement} Statement + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Statement.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogTransaction.Statement(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.category = reader.int32(); + break; + } + case 2: { + message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); + break; + } + case 3: { + message.sql = reader.bytes(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; - /** - * BinlogSource tablet_type. - * @member {topodata.TabletType} tablet_type - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.tablet_type = 0; + /** + * Decodes a Statement message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {binlogdata.BinlogTransaction.Statement} Statement + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Statement.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * BinlogSource key_range. - * @member {topodata.IKeyRange|null|undefined} key_range - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.key_range = null; + /** + * Verifies a Statement message. + * @function verify + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Statement.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.category != null && message.hasOwnProperty("category")) + switch (message.category) { + default: + return "category: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + case 9: + break; + } + if (message.charset != null && message.hasOwnProperty("charset")) { + let error = $root.binlogdata.Charset.verify(message.charset); + if (error) + return "charset." + error; + } + if (message.sql != null && message.hasOwnProperty("sql")) + if (!(message.sql && typeof message.sql.length === "number" || $util.isString(message.sql))) + return "sql: buffer expected"; + return null; + }; - /** - * BinlogSource tables. - * @member {Array.} tables - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.tables = $util.emptyArray; + /** + * Creates a Statement message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {Object.} object Plain object + * @returns {binlogdata.BinlogTransaction.Statement} Statement + */ + Statement.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.BinlogTransaction.Statement) + return object; + let message = new $root.binlogdata.BinlogTransaction.Statement(); + switch (object.category) { + default: + if (typeof object.category === "number") { + message.category = object.category; + break; + } + break; + case "BL_UNRECOGNIZED": + case 0: + message.category = 0; + break; + case "BL_BEGIN": + case 1: + message.category = 1; + break; + case "BL_COMMIT": + case 2: + message.category = 2; + break; + case "BL_ROLLBACK": + case 3: + message.category = 3; + break; + case "BL_DML_DEPRECATED": + case 4: + message.category = 4; + break; + case "BL_DDL": + case 5: + message.category = 5; + break; + case "BL_SET": + case 6: + message.category = 6; + break; + case "BL_INSERT": + case 7: + message.category = 7; + break; + case "BL_UPDATE": + case 8: + message.category = 8; + break; + case "BL_DELETE": + case 9: + message.category = 9; + break; + } + if (object.charset != null) { + if (typeof object.charset !== "object") + throw TypeError(".binlogdata.BinlogTransaction.Statement.charset: object expected"); + message.charset = $root.binlogdata.Charset.fromObject(object.charset); + } + if (object.sql != null) + if (typeof object.sql === "string") + $util.base64.decode(object.sql, message.sql = $util.newBuffer($util.base64.length(object.sql)), 0); + else if (object.sql.length >= 0) + message.sql = object.sql; + return message; + }; - /** - * BinlogSource filter. - * @member {binlogdata.IFilter|null|undefined} filter - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.filter = null; + /** + * Creates a plain object from a Statement message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {binlogdata.BinlogTransaction.Statement} message Statement + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Statement.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.category = options.enums === String ? "BL_UNRECOGNIZED" : 0; + object.charset = null; + if (options.bytes === String) + object.sql = ""; + else { + object.sql = []; + if (options.bytes !== Array) + object.sql = $util.newBuffer(object.sql); + } + } + if (message.category != null && message.hasOwnProperty("category")) + object.category = options.enums === String ? $root.binlogdata.BinlogTransaction.Statement.Category[message.category] === undefined ? message.category : $root.binlogdata.BinlogTransaction.Statement.Category[message.category] : message.category; + if (message.charset != null && message.hasOwnProperty("charset")) + object.charset = $root.binlogdata.Charset.toObject(message.charset, options); + if (message.sql != null && message.hasOwnProperty("sql")) + object.sql = options.bytes === String ? $util.base64.encode(message.sql, 0, message.sql.length) : options.bytes === Array ? Array.prototype.slice.call(message.sql) : message.sql; + return object; + }; - /** - * BinlogSource on_ddl. - * @member {binlogdata.OnDDLAction} on_ddl - * @memberof binlogdata.BinlogSource - * @instance - */ - BinlogSource.prototype.on_ddl = 0; + /** + * Converts this Statement to JSON. + * @function toJSON + * @memberof binlogdata.BinlogTransaction.Statement + * @instance + * @returns {Object.} JSON object + */ + Statement.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Statement + * @function getTypeUrl + * @memberof binlogdata.BinlogTransaction.Statement + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Statement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/binlogdata.BinlogTransaction.Statement"; + }; + + /** + * Category enum. + * @name binlogdata.BinlogTransaction.Statement.Category + * @enum {number} + * @property {number} BL_UNRECOGNIZED=0 BL_UNRECOGNIZED value + * @property {number} BL_BEGIN=1 BL_BEGIN value + * @property {number} BL_COMMIT=2 BL_COMMIT value + * @property {number} BL_ROLLBACK=3 BL_ROLLBACK value + * @property {number} BL_DML_DEPRECATED=4 BL_DML_DEPRECATED value + * @property {number} BL_DDL=5 BL_DDL value + * @property {number} BL_SET=6 BL_SET value + * @property {number} BL_INSERT=7 BL_INSERT value + * @property {number} BL_UPDATE=8 BL_UPDATE value + * @property {number} BL_DELETE=9 BL_DELETE value + */ + Statement.Category = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "BL_UNRECOGNIZED"] = 0; + values[valuesById[1] = "BL_BEGIN"] = 1; + values[valuesById[2] = "BL_COMMIT"] = 2; + values[valuesById[3] = "BL_ROLLBACK"] = 3; + values[valuesById[4] = "BL_DML_DEPRECATED"] = 4; + values[valuesById[5] = "BL_DDL"] = 5; + values[valuesById[6] = "BL_SET"] = 6; + values[valuesById[7] = "BL_INSERT"] = 7; + values[valuesById[8] = "BL_UPDATE"] = 8; + values[valuesById[9] = "BL_DELETE"] = 9; + return values; + })(); + + return Statement; + })(); + + return BinlogTransaction; + })(); + + binlogdata.StreamKeyRangeRequest = (function() { /** - * BinlogSource external_mysql. - * @member {string} external_mysql - * @memberof binlogdata.BinlogSource - * @instance + * Properties of a StreamKeyRangeRequest. + * @memberof binlogdata + * @interface IStreamKeyRangeRequest + * @property {string|null} [position] StreamKeyRangeRequest position + * @property {topodata.IKeyRange|null} [key_range] StreamKeyRangeRequest key_range + * @property {binlogdata.ICharset|null} [charset] StreamKeyRangeRequest charset */ - BinlogSource.prototype.external_mysql = ""; /** - * BinlogSource stop_after_copy. - * @member {boolean} stop_after_copy - * @memberof binlogdata.BinlogSource - * @instance + * Constructs a new StreamKeyRangeRequest. + * @memberof binlogdata + * @classdesc Represents a StreamKeyRangeRequest. + * @implements IStreamKeyRangeRequest + * @constructor + * @param {binlogdata.IStreamKeyRangeRequest=} [properties] Properties to set */ - BinlogSource.prototype.stop_after_copy = false; + function StreamKeyRangeRequest(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]]; + } /** - * BinlogSource external_cluster. - * @member {string} external_cluster - * @memberof binlogdata.BinlogSource + * StreamKeyRangeRequest position. + * @member {string} position + * @memberof binlogdata.StreamKeyRangeRequest * @instance */ - BinlogSource.prototype.external_cluster = ""; + StreamKeyRangeRequest.prototype.position = ""; /** - * BinlogSource source_time_zone. - * @member {string} source_time_zone - * @memberof binlogdata.BinlogSource + * StreamKeyRangeRequest key_range. + * @member {topodata.IKeyRange|null|undefined} key_range + * @memberof binlogdata.StreamKeyRangeRequest * @instance */ - BinlogSource.prototype.source_time_zone = ""; + StreamKeyRangeRequest.prototype.key_range = null; /** - * BinlogSource target_time_zone. - * @member {string} target_time_zone - * @memberof binlogdata.BinlogSource + * StreamKeyRangeRequest charset. + * @member {binlogdata.ICharset|null|undefined} charset + * @memberof binlogdata.StreamKeyRangeRequest * @instance */ - BinlogSource.prototype.target_time_zone = ""; + StreamKeyRangeRequest.prototype.charset = null; /** - * Creates a new BinlogSource instance using the specified properties. + * Creates a new StreamKeyRangeRequest instance using the specified properties. * @function create - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static - * @param {binlogdata.IBinlogSource=} [properties] Properties to set - * @returns {binlogdata.BinlogSource} BinlogSource instance + * @param {binlogdata.IStreamKeyRangeRequest=} [properties] Properties to set + * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest instance */ - BinlogSource.create = function create(properties) { - return new BinlogSource(properties); + StreamKeyRangeRequest.create = function create(properties) { + return new StreamKeyRangeRequest(properties); }; /** - * Encodes the specified BinlogSource message. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. + * Encodes the specified StreamKeyRangeRequest message. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. * @function encode - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static - * @param {binlogdata.IBinlogSource} message BinlogSource message or plain object to encode + * @param {binlogdata.IStreamKeyRangeRequest} message StreamKeyRangeRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BinlogSource.encode = function encode(message, writer) { + StreamKeyRangeRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); if (message.key_range != null && Object.hasOwnProperty.call(message, "key_range")) - $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.tables[i]); - if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) - $root.binlogdata.Filter.encode(message.filter, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) - writer.uint32(/* id 7, wireType 0 =*/56).int32(message.on_ddl); - if (message.external_mysql != null && Object.hasOwnProperty.call(message, "external_mysql")) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.external_mysql); - if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) - writer.uint32(/* id 9, wireType 0 =*/72).bool(message.stop_after_copy); - if (message.external_cluster != null && Object.hasOwnProperty.call(message, "external_cluster")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.external_cluster); - if (message.source_time_zone != null && Object.hasOwnProperty.call(message, "source_time_zone")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.source_time_zone); - if (message.target_time_zone != null && Object.hasOwnProperty.call(message, "target_time_zone")) - writer.uint32(/* id 12, wireType 2 =*/98).string(message.target_time_zone); + $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) + $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified BinlogSource message, length delimited. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. + * Encodes the specified StreamKeyRangeRequest message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeRequest.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static - * @param {binlogdata.IBinlogSource} message BinlogSource message or plain object to encode + * @param {binlogdata.IStreamKeyRangeRequest} message StreamKeyRangeRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BinlogSource.encodeDelimited = function encodeDelimited(message, writer) { + StreamKeyRangeRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BinlogSource message from the specified reader or buffer. + * Decodes a StreamKeyRangeRequest message from the specified reader or buffer. * @function decode - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.BinlogSource} BinlogSource + * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BinlogSource.decode = function decode(reader, length) { + StreamKeyRangeRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogSource(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamKeyRangeRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.position = reader.string(); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.tablet_type = reader.int32(); - break; - } - case 4: { message.key_range = $root.topodata.KeyRange.decode(reader, reader.uint32()); break; } - case 5: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); - break; - } - case 6: { - message.filter = $root.binlogdata.Filter.decode(reader, reader.uint32()); - break; - } - case 7: { - message.on_ddl = reader.int32(); - break; - } - case 8: { - message.external_mysql = reader.string(); - break; - } - case 9: { - message.stop_after_copy = reader.bool(); - break; - } - case 10: { - message.external_cluster = reader.string(); - break; - } - case 11: { - message.source_time_zone = reader.string(); - break; - } - case 12: { - message.target_time_zone = reader.string(); + case 3: { + message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); break; } default: @@ -66117,381 +66792,360 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a BinlogSource message from the specified reader or buffer, length delimited. + * Decodes a StreamKeyRangeRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.BinlogSource} BinlogSource + * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BinlogSource.decodeDelimited = function decodeDelimited(reader) { + StreamKeyRangeRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BinlogSource message. + * Verifies a StreamKeyRangeRequest message. * @function verify - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BinlogSource.verify = function verify(message) { + StreamKeyRangeRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - switch (message.tablet_type) { - default: - return "tablet_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; if (message.key_range != null && message.hasOwnProperty("key_range")) { let error = $root.topodata.KeyRange.verify(message.key_range); if (error) return "key_range." + error; } - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; - } - if (message.filter != null && message.hasOwnProperty("filter")) { - let error = $root.binlogdata.Filter.verify(message.filter); + if (message.charset != null && message.hasOwnProperty("charset")) { + let error = $root.binlogdata.Charset.verify(message.charset); if (error) - return "filter." + error; + return "charset." + error; } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - switch (message.on_ddl) { - default: - return "on_ddl: enum value expected"; - case 0: - case 1: - case 2: - case 3: - break; - } - if (message.external_mysql != null && message.hasOwnProperty("external_mysql")) - if (!$util.isString(message.external_mysql)) - return "external_mysql: string expected"; - 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.external_cluster != null && message.hasOwnProperty("external_cluster")) - if (!$util.isString(message.external_cluster)) - return "external_cluster: string expected"; - if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) - if (!$util.isString(message.source_time_zone)) - return "source_time_zone: string expected"; - if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) - if (!$util.isString(message.target_time_zone)) - return "target_time_zone: string expected"; return null; }; /** - * Creates a BinlogSource message from a plain object. Also converts values to their respective internal types. + * Creates a StreamKeyRangeRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static * @param {Object.} object Plain object - * @returns {binlogdata.BinlogSource} BinlogSource + * @returns {binlogdata.StreamKeyRangeRequest} StreamKeyRangeRequest */ - BinlogSource.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.BinlogSource) + StreamKeyRangeRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.StreamKeyRangeRequest) return object; - let message = new $root.binlogdata.BinlogSource(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - switch (object.tablet_type) { - default: - if (typeof object.tablet_type === "number") { - message.tablet_type = object.tablet_type; - break; - } - break; - case "UNKNOWN": - case 0: - message.tablet_type = 0; - break; - case "PRIMARY": - case 1: - message.tablet_type = 1; - break; - case "MASTER": - case 1: - message.tablet_type = 1; - break; - case "REPLICA": - case 2: - message.tablet_type = 2; - break; - case "RDONLY": - case 3: - message.tablet_type = 3; - break; - case "BATCH": - case 3: - message.tablet_type = 3; - break; - case "SPARE": - case 4: - message.tablet_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_type = 5; - break; - case "BACKUP": - case 6: - message.tablet_type = 6; - break; - case "RESTORE": - case 7: - message.tablet_type = 7; - break; - case "DRAINED": - case 8: - message.tablet_type = 8; - break; - } + let message = new $root.binlogdata.StreamKeyRangeRequest(); + if (object.position != null) + message.position = String(object.position); if (object.key_range != null) { if (typeof object.key_range !== "object") - throw TypeError(".binlogdata.BinlogSource.key_range: object expected"); + throw TypeError(".binlogdata.StreamKeyRangeRequest.key_range: object expected"); message.key_range = $root.topodata.KeyRange.fromObject(object.key_range); } - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".binlogdata.BinlogSource.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); - } - if (object.filter != null) { - if (typeof object.filter !== "object") - throw TypeError(".binlogdata.BinlogSource.filter: object expected"); - message.filter = $root.binlogdata.Filter.fromObject(object.filter); - } - switch (object.on_ddl) { - default: - if (typeof object.on_ddl === "number") { - message.on_ddl = object.on_ddl; - break; - } - break; - case "IGNORE": - case 0: - message.on_ddl = 0; - break; - case "STOP": - case 1: - message.on_ddl = 1; - break; - case "EXEC": - case 2: - message.on_ddl = 2; - break; - case "EXEC_IGNORE": - case 3: - message.on_ddl = 3; - break; + if (object.charset != null) { + if (typeof object.charset !== "object") + throw TypeError(".binlogdata.StreamKeyRangeRequest.charset: object expected"); + message.charset = $root.binlogdata.Charset.fromObject(object.charset); } - if (object.external_mysql != null) - message.external_mysql = String(object.external_mysql); - if (object.stop_after_copy != null) - message.stop_after_copy = Boolean(object.stop_after_copy); - if (object.external_cluster != null) - message.external_cluster = String(object.external_cluster); - if (object.source_time_zone != null) - message.source_time_zone = String(object.source_time_zone); - if (object.target_time_zone != null) - message.target_time_zone = String(object.target_time_zone); return message; }; /** - * Creates a plain object from a BinlogSource message. Also converts values to other types if specified. + * Creates a plain object from a StreamKeyRangeRequest message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static - * @param {binlogdata.BinlogSource} message BinlogSource + * @param {binlogdata.StreamKeyRangeRequest} message StreamKeyRangeRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BinlogSource.toObject = function toObject(message, options) { + StreamKeyRangeRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tables = []; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + object.position = ""; object.key_range = null; - object.filter = null; - object.on_ddl = options.enums === String ? "IGNORE" : 0; - object.external_mysql = ""; - object.stop_after_copy = false; - object.external_cluster = ""; - object.source_time_zone = ""; - object.target_time_zone = ""; + object.charset = null; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; if (message.key_range != null && message.hasOwnProperty("key_range")) object.key_range = $root.topodata.KeyRange.toObject(message.key_range, options); - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; - } - if (message.filter != null && message.hasOwnProperty("filter")) - object.filter = $root.binlogdata.Filter.toObject(message.filter, options); - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; - if (message.external_mysql != null && message.hasOwnProperty("external_mysql")) - object.external_mysql = message.external_mysql; - if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) - object.stop_after_copy = message.stop_after_copy; - if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) - object.external_cluster = message.external_cluster; - if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) - object.source_time_zone = message.source_time_zone; - if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) - object.target_time_zone = message.target_time_zone; + if (message.charset != null && message.hasOwnProperty("charset")) + object.charset = $root.binlogdata.Charset.toObject(message.charset, options); return object; }; /** - * Converts this BinlogSource to JSON. + * Converts this StreamKeyRangeRequest to JSON. * @function toJSON - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @instance * @returns {Object.} JSON object */ - BinlogSource.prototype.toJSON = function toJSON() { + StreamKeyRangeRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BinlogSource + * Gets the default type url for StreamKeyRangeRequest * @function getTypeUrl - * @memberof binlogdata.BinlogSource + * @memberof binlogdata.StreamKeyRangeRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BinlogSource.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamKeyRangeRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.BinlogSource"; + return typeUrlPrefix + "/binlogdata.StreamKeyRangeRequest"; }; - return BinlogSource; + return StreamKeyRangeRequest; })(); - /** - * VEventType enum. - * @name binlogdata.VEventType - * @enum {number} - * @property {number} UNKNOWN=0 UNKNOWN value - * @property {number} GTID=1 GTID value - * @property {number} BEGIN=2 BEGIN value - * @property {number} COMMIT=3 COMMIT value - * @property {number} ROLLBACK=4 ROLLBACK value - * @property {number} DDL=5 DDL value - * @property {number} INSERT=6 INSERT value - * @property {number} REPLACE=7 REPLACE value - * @property {number} UPDATE=8 UPDATE value - * @property {number} DELETE=9 DELETE value - * @property {number} SET=10 SET value - * @property {number} OTHER=11 OTHER value - * @property {number} ROW=12 ROW value - * @property {number} FIELD=13 FIELD value - * @property {number} HEARTBEAT=14 HEARTBEAT value - * @property {number} VGTID=15 VGTID value - * @property {number} JOURNAL=16 JOURNAL value - * @property {number} VERSION=17 VERSION value - * @property {number} LASTPK=18 LASTPK value - * @property {number} SAVEPOINT=19 SAVEPOINT value - * @property {number} COPY_COMPLETED=20 COPY_COMPLETED value - */ - binlogdata.VEventType = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "UNKNOWN"] = 0; - values[valuesById[1] = "GTID"] = 1; - values[valuesById[2] = "BEGIN"] = 2; - values[valuesById[3] = "COMMIT"] = 3; - values[valuesById[4] = "ROLLBACK"] = 4; - values[valuesById[5] = "DDL"] = 5; - values[valuesById[6] = "INSERT"] = 6; - values[valuesById[7] = "REPLACE"] = 7; - values[valuesById[8] = "UPDATE"] = 8; - values[valuesById[9] = "DELETE"] = 9; - values[valuesById[10] = "SET"] = 10; - values[valuesById[11] = "OTHER"] = 11; - values[valuesById[12] = "ROW"] = 12; - values[valuesById[13] = "FIELD"] = 13; - values[valuesById[14] = "HEARTBEAT"] = 14; - values[valuesById[15] = "VGTID"] = 15; - values[valuesById[16] = "JOURNAL"] = 16; - values[valuesById[17] = "VERSION"] = 17; - values[valuesById[18] = "LASTPK"] = 18; - values[valuesById[19] = "SAVEPOINT"] = 19; - values[valuesById[20] = "COPY_COMPLETED"] = 20; - return values; + binlogdata.StreamKeyRangeResponse = (function() { + + /** + * Properties of a StreamKeyRangeResponse. + * @memberof binlogdata + * @interface IStreamKeyRangeResponse + * @property {binlogdata.IBinlogTransaction|null} [binlog_transaction] StreamKeyRangeResponse binlog_transaction + */ + + /** + * Constructs a new StreamKeyRangeResponse. + * @memberof binlogdata + * @classdesc Represents a StreamKeyRangeResponse. + * @implements IStreamKeyRangeResponse + * @constructor + * @param {binlogdata.IStreamKeyRangeResponse=} [properties] Properties to set + */ + function StreamKeyRangeResponse(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]]; + } + + /** + * StreamKeyRangeResponse binlog_transaction. + * @member {binlogdata.IBinlogTransaction|null|undefined} binlog_transaction + * @memberof binlogdata.StreamKeyRangeResponse + * @instance + */ + StreamKeyRangeResponse.prototype.binlog_transaction = null; + + /** + * Creates a new StreamKeyRangeResponse instance using the specified properties. + * @function create + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {binlogdata.IStreamKeyRangeResponse=} [properties] Properties to set + * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse instance + */ + StreamKeyRangeResponse.create = function create(properties) { + return new StreamKeyRangeResponse(properties); + }; + + /** + * Encodes the specified StreamKeyRangeResponse message. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * @function encode + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {binlogdata.IStreamKeyRangeResponse} message StreamKeyRangeResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + StreamKeyRangeResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.binlog_transaction != null && Object.hasOwnProperty.call(message, "binlog_transaction")) + $root.binlogdata.BinlogTransaction.encode(message.binlog_transaction, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified StreamKeyRangeResponse message, length delimited. Does not implicitly {@link binlogdata.StreamKeyRangeResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {binlogdata.IStreamKeyRangeResponse} message StreamKeyRangeResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + StreamKeyRangeResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a StreamKeyRangeResponse message from the specified reader or buffer. + * @function decode + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + StreamKeyRangeResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamKeyRangeResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.binlog_transaction = $root.binlogdata.BinlogTransaction.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a StreamKeyRangeResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + StreamKeyRangeResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a StreamKeyRangeResponse message. + * @function verify + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + StreamKeyRangeResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) { + let error = $root.binlogdata.BinlogTransaction.verify(message.binlog_transaction); + if (error) + return "binlog_transaction." + error; + } + return null; + }; + + /** + * Creates a StreamKeyRangeResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {Object.} object Plain object + * @returns {binlogdata.StreamKeyRangeResponse} StreamKeyRangeResponse + */ + StreamKeyRangeResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.StreamKeyRangeResponse) + return object; + let message = new $root.binlogdata.StreamKeyRangeResponse(); + if (object.binlog_transaction != null) { + if (typeof object.binlog_transaction !== "object") + throw TypeError(".binlogdata.StreamKeyRangeResponse.binlog_transaction: object expected"); + message.binlog_transaction = $root.binlogdata.BinlogTransaction.fromObject(object.binlog_transaction); + } + return message; + }; + + /** + * Creates a plain object from a StreamKeyRangeResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {binlogdata.StreamKeyRangeResponse} message StreamKeyRangeResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + StreamKeyRangeResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.binlog_transaction = null; + if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) + object.binlog_transaction = $root.binlogdata.BinlogTransaction.toObject(message.binlog_transaction, options); + return object; + }; + + /** + * Converts this StreamKeyRangeResponse to JSON. + * @function toJSON + * @memberof binlogdata.StreamKeyRangeResponse + * @instance + * @returns {Object.} JSON object + */ + StreamKeyRangeResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for StreamKeyRangeResponse + * @function getTypeUrl + * @memberof binlogdata.StreamKeyRangeResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + StreamKeyRangeResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/binlogdata.StreamKeyRangeResponse"; + }; + + return StreamKeyRangeResponse; })(); - binlogdata.RowChange = (function() { + binlogdata.StreamTablesRequest = (function() { /** - * Properties of a RowChange. + * Properties of a StreamTablesRequest. * @memberof binlogdata - * @interface IRowChange - * @property {query.IRow|null} [before] RowChange before - * @property {query.IRow|null} [after] RowChange after - * @property {binlogdata.RowChange.IBitmap|null} [data_columns] RowChange data_columns + * @interface IStreamTablesRequest + * @property {string|null} [position] StreamTablesRequest position + * @property {Array.|null} [tables] StreamTablesRequest tables + * @property {binlogdata.ICharset|null} [charset] StreamTablesRequest charset */ /** - * Constructs a new RowChange. + * Constructs a new StreamTablesRequest. * @memberof binlogdata - * @classdesc Represents a RowChange. - * @implements IRowChange + * @classdesc Represents a StreamTablesRequest. + * @implements IStreamTablesRequest * @constructor - * @param {binlogdata.IRowChange=} [properties] Properties to set + * @param {binlogdata.IStreamTablesRequest=} [properties] Properties to set */ - function RowChange(properties) { + function StreamTablesRequest(properties) { + this.tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -66499,103 +67153,106 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * RowChange before. - * @member {query.IRow|null|undefined} before - * @memberof binlogdata.RowChange + * StreamTablesRequest position. + * @member {string} position + * @memberof binlogdata.StreamTablesRequest * @instance */ - RowChange.prototype.before = null; + StreamTablesRequest.prototype.position = ""; /** - * RowChange after. - * @member {query.IRow|null|undefined} after - * @memberof binlogdata.RowChange + * StreamTablesRequest tables. + * @member {Array.} tables + * @memberof binlogdata.StreamTablesRequest * @instance */ - RowChange.prototype.after = null; + StreamTablesRequest.prototype.tables = $util.emptyArray; /** - * RowChange data_columns. - * @member {binlogdata.RowChange.IBitmap|null|undefined} data_columns - * @memberof binlogdata.RowChange + * StreamTablesRequest charset. + * @member {binlogdata.ICharset|null|undefined} charset + * @memberof binlogdata.StreamTablesRequest * @instance */ - RowChange.prototype.data_columns = null; + StreamTablesRequest.prototype.charset = null; /** - * Creates a new RowChange instance using the specified properties. + * Creates a new StreamTablesRequest instance using the specified properties. * @function create - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static - * @param {binlogdata.IRowChange=} [properties] Properties to set - * @returns {binlogdata.RowChange} RowChange instance + * @param {binlogdata.IStreamTablesRequest=} [properties] Properties to set + * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest instance */ - RowChange.create = function create(properties) { - return new RowChange(properties); + StreamTablesRequest.create = function create(properties) { + return new StreamTablesRequest(properties); }; /** - * Encodes the specified RowChange message. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. + * Encodes the specified StreamTablesRequest message. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. * @function encode - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static - * @param {binlogdata.IRowChange} message RowChange message or plain object to encode + * @param {binlogdata.IStreamTablesRequest} message StreamTablesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RowChange.encode = function encode(message, writer) { + StreamTablesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.before != null && Object.hasOwnProperty.call(message, "before")) - $root.query.Row.encode(message.before, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.after != null && Object.hasOwnProperty.call(message, "after")) - $root.query.Row.encode(message.after, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.data_columns != null && Object.hasOwnProperty.call(message, "data_columns")) - $root.binlogdata.RowChange.Bitmap.encode(message.data_columns, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.tables[i]); + if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) + $root.binlogdata.Charset.encode(message.charset, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified RowChange message, length delimited. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. + * Encodes the specified StreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.StreamTablesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static - * @param {binlogdata.IRowChange} message RowChange message or plain object to encode + * @param {binlogdata.IStreamTablesRequest} message StreamTablesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RowChange.encodeDelimited = function encodeDelimited(message, writer) { + StreamTablesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RowChange message from the specified reader or buffer. + * Decodes a StreamTablesRequest message from the specified reader or buffer. * @function decode - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.RowChange} RowChange + * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RowChange.decode = function decode(reader, length) { + StreamTablesRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowChange(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamTablesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.before = $root.query.Row.decode(reader, reader.uint32()); + message.position = reader.string(); break; } case 2: { - message.after = $root.query.Row.decode(reader, reader.uint32()); + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); break; } case 3: { - message.data_columns = $root.binlogdata.RowChange.Bitmap.decode(reader, reader.uint32()); + message.charset = $root.binlogdata.Charset.decode(reader, reader.uint32()); break; } default: @@ -66607,408 +67264,157 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a RowChange message from the specified reader or buffer, length delimited. + * Decodes a StreamTablesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.RowChange} RowChange + * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RowChange.decodeDelimited = function decodeDelimited(reader) { + StreamTablesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RowChange message. + * Verifies a StreamTablesRequest message. * @function verify - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RowChange.verify = function verify(message) { + StreamTablesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.before != null && message.hasOwnProperty("before")) { - let error = $root.query.Row.verify(message.before); - if (error) - return "before." + error; - } - if (message.after != null && message.hasOwnProperty("after")) { - let error = $root.query.Row.verify(message.after); - if (error) - return "after." + error; + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; } - if (message.data_columns != null && message.hasOwnProperty("data_columns")) { - let error = $root.binlogdata.RowChange.Bitmap.verify(message.data_columns); + if (message.charset != null && message.hasOwnProperty("charset")) { + let error = $root.binlogdata.Charset.verify(message.charset); if (error) - return "data_columns." + error; + return "charset." + error; } return null; }; /** - * Creates a RowChange message from a plain object. Also converts values to their respective internal types. + * Creates a StreamTablesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static * @param {Object.} object Plain object - * @returns {binlogdata.RowChange} RowChange + * @returns {binlogdata.StreamTablesRequest} StreamTablesRequest */ - RowChange.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.RowChange) + StreamTablesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.StreamTablesRequest) return object; - let message = new $root.binlogdata.RowChange(); - if (object.before != null) { - if (typeof object.before !== "object") - throw TypeError(".binlogdata.RowChange.before: object expected"); - message.before = $root.query.Row.fromObject(object.before); - } - if (object.after != null) { - if (typeof object.after !== "object") - throw TypeError(".binlogdata.RowChange.after: object expected"); - message.after = $root.query.Row.fromObject(object.after); + let message = new $root.binlogdata.StreamTablesRequest(); + if (object.position != null) + message.position = String(object.position); + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".binlogdata.StreamTablesRequest.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); } - if (object.data_columns != null) { - if (typeof object.data_columns !== "object") - throw TypeError(".binlogdata.RowChange.data_columns: object expected"); - message.data_columns = $root.binlogdata.RowChange.Bitmap.fromObject(object.data_columns); + if (object.charset != null) { + if (typeof object.charset !== "object") + throw TypeError(".binlogdata.StreamTablesRequest.charset: object expected"); + message.charset = $root.binlogdata.Charset.fromObject(object.charset); } return message; }; /** - * Creates a plain object from a RowChange message. Also converts values to other types if specified. + * Creates a plain object from a StreamTablesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static - * @param {binlogdata.RowChange} message RowChange + * @param {binlogdata.StreamTablesRequest} message StreamTablesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RowChange.toObject = function toObject(message, options) { + StreamTablesRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.tables = []; if (options.defaults) { - object.before = null; - object.after = null; - object.data_columns = null; + object.position = ""; + object.charset = null; } - if (message.before != null && message.hasOwnProperty("before")) - object.before = $root.query.Row.toObject(message.before, options); - if (message.after != null && message.hasOwnProperty("after")) - object.after = $root.query.Row.toObject(message.after, options); - if (message.data_columns != null && message.hasOwnProperty("data_columns")) - object.data_columns = $root.binlogdata.RowChange.Bitmap.toObject(message.data_columns, options); + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } + if (message.charset != null && message.hasOwnProperty("charset")) + object.charset = $root.binlogdata.Charset.toObject(message.charset, options); return object; }; /** - * Converts this RowChange to JSON. + * Converts this StreamTablesRequest to JSON. * @function toJSON - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @instance * @returns {Object.} JSON object */ - RowChange.prototype.toJSON = function toJSON() { + StreamTablesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RowChange + * Gets the default type url for StreamTablesRequest * @function getTypeUrl - * @memberof binlogdata.RowChange + * @memberof binlogdata.StreamTablesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RowChange.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamTablesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.RowChange"; + return typeUrlPrefix + "/binlogdata.StreamTablesRequest"; }; - RowChange.Bitmap = (function() { - - /** - * Properties of a Bitmap. - * @memberof binlogdata.RowChange - * @interface IBitmap - * @property {number|Long|null} [count] Bitmap count - * @property {Uint8Array|null} [cols] Bitmap cols - */ - - /** - * Constructs a new Bitmap. - * @memberof binlogdata.RowChange - * @classdesc Represents a Bitmap. - * @implements IBitmap - * @constructor - * @param {binlogdata.RowChange.IBitmap=} [properties] Properties to set - */ - function Bitmap(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]]; - } + return StreamTablesRequest; + })(); - /** - * Bitmap count. - * @member {number|Long} count - * @memberof binlogdata.RowChange.Bitmap - * @instance - */ - Bitmap.prototype.count = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + binlogdata.StreamTablesResponse = (function() { - /** - * Bitmap cols. - * @member {Uint8Array} cols - * @memberof binlogdata.RowChange.Bitmap - * @instance - */ - Bitmap.prototype.cols = $util.newBuffer([]); - - /** - * Creates a new Bitmap instance using the specified properties. - * @function create - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {binlogdata.RowChange.IBitmap=} [properties] Properties to set - * @returns {binlogdata.RowChange.Bitmap} Bitmap instance - */ - Bitmap.create = function create(properties) { - return new Bitmap(properties); - }; - - /** - * Encodes the specified Bitmap message. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. - * @function encode - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {binlogdata.RowChange.IBitmap} message Bitmap message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Bitmap.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.count != null && Object.hasOwnProperty.call(message, "count")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.count); - if (message.cols != null && Object.hasOwnProperty.call(message, "cols")) - writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.cols); - return writer; - }; - - /** - * Encodes the specified Bitmap message, length delimited. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. - * @function encodeDelimited - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {binlogdata.RowChange.IBitmap} message Bitmap message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Bitmap.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a Bitmap message from the specified reader or buffer. - * @function decode - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.RowChange.Bitmap} Bitmap - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Bitmap.decode = function decode(reader, length) { - if (!(reader instanceof $Reader)) - reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowChange.Bitmap(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.count = reader.int64(); - break; - } - case 2: { - message.cols = reader.bytes(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a Bitmap message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.RowChange.Bitmap} Bitmap - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Bitmap.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a Bitmap message. - * @function verify - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Bitmap.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.count != null && message.hasOwnProperty("count")) - if (!$util.isInteger(message.count) && !(message.count && $util.isInteger(message.count.low) && $util.isInteger(message.count.high))) - return "count: integer|Long expected"; - if (message.cols != null && message.hasOwnProperty("cols")) - if (!(message.cols && typeof message.cols.length === "number" || $util.isString(message.cols))) - return "cols: buffer expected"; - return null; - }; - - /** - * Creates a Bitmap message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {Object.} object Plain object - * @returns {binlogdata.RowChange.Bitmap} Bitmap - */ - Bitmap.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.RowChange.Bitmap) - return object; - let message = new $root.binlogdata.RowChange.Bitmap(); - if (object.count != null) - if ($util.Long) - (message.count = $util.Long.fromValue(object.count)).unsigned = false; - else if (typeof object.count === "string") - message.count = parseInt(object.count, 10); - else if (typeof object.count === "number") - message.count = object.count; - else if (typeof object.count === "object") - message.count = new $util.LongBits(object.count.low >>> 0, object.count.high >>> 0).toNumber(); - if (object.cols != null) - if (typeof object.cols === "string") - $util.base64.decode(object.cols, message.cols = $util.newBuffer($util.base64.length(object.cols)), 0); - else if (object.cols.length >= 0) - message.cols = object.cols; - return message; - }; - - /** - * Creates a plain object from a Bitmap message. Also converts values to other types if specified. - * @function toObject - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {binlogdata.RowChange.Bitmap} message Bitmap - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Bitmap.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.count = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.count = options.longs === String ? "0" : 0; - if (options.bytes === String) - object.cols = ""; - else { - object.cols = []; - if (options.bytes !== Array) - object.cols = $util.newBuffer(object.cols); - } - } - if (message.count != null && message.hasOwnProperty("count")) - if (typeof message.count === "number") - object.count = options.longs === String ? String(message.count) : message.count; - else - object.count = options.longs === String ? $util.Long.prototype.toString.call(message.count) : options.longs === Number ? new $util.LongBits(message.count.low >>> 0, message.count.high >>> 0).toNumber() : message.count; - if (message.cols != null && message.hasOwnProperty("cols")) - object.cols = options.bytes === String ? $util.base64.encode(message.cols, 0, message.cols.length) : options.bytes === Array ? Array.prototype.slice.call(message.cols) : message.cols; - return object; - }; - - /** - * Converts this Bitmap to JSON. - * @function toJSON - * @memberof binlogdata.RowChange.Bitmap - * @instance - * @returns {Object.} JSON object - */ - Bitmap.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for Bitmap - * @function getTypeUrl - * @memberof binlogdata.RowChange.Bitmap - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Bitmap.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/binlogdata.RowChange.Bitmap"; - }; - - return Bitmap; - })(); - - return RowChange; - })(); - - binlogdata.RowEvent = (function() { - - /** - * Properties of a RowEvent. - * @memberof binlogdata - * @interface IRowEvent - * @property {string|null} [table_name] RowEvent table_name - * @property {Array.|null} [row_changes] RowEvent row_changes - * @property {string|null} [keyspace] RowEvent keyspace - * @property {string|null} [shard] RowEvent shard - */ + /** + * Properties of a StreamTablesResponse. + * @memberof binlogdata + * @interface IStreamTablesResponse + * @property {binlogdata.IBinlogTransaction|null} [binlog_transaction] StreamTablesResponse binlog_transaction + */ /** - * Constructs a new RowEvent. + * Constructs a new StreamTablesResponse. * @memberof binlogdata - * @classdesc Represents a RowEvent. - * @implements IRowEvent + * @classdesc Represents a StreamTablesResponse. + * @implements IStreamTablesResponse * @constructor - * @param {binlogdata.IRowEvent=} [properties] Properties to set + * @param {binlogdata.IStreamTablesResponse=} [properties] Properties to set */ - function RowEvent(properties) { - this.row_changes = []; + function StreamTablesResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -67016,120 +67422,75 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * RowEvent table_name. - * @member {string} table_name - * @memberof binlogdata.RowEvent - * @instance - */ - RowEvent.prototype.table_name = ""; - - /** - * RowEvent row_changes. - * @member {Array.} row_changes - * @memberof binlogdata.RowEvent - * @instance - */ - RowEvent.prototype.row_changes = $util.emptyArray; - - /** - * RowEvent keyspace. - * @member {string} keyspace - * @memberof binlogdata.RowEvent - * @instance - */ - RowEvent.prototype.keyspace = ""; - - /** - * RowEvent shard. - * @member {string} shard - * @memberof binlogdata.RowEvent + * StreamTablesResponse binlog_transaction. + * @member {binlogdata.IBinlogTransaction|null|undefined} binlog_transaction + * @memberof binlogdata.StreamTablesResponse * @instance */ - RowEvent.prototype.shard = ""; + StreamTablesResponse.prototype.binlog_transaction = null; /** - * Creates a new RowEvent instance using the specified properties. + * Creates a new StreamTablesResponse instance using the specified properties. * @function create - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static - * @param {binlogdata.IRowEvent=} [properties] Properties to set - * @returns {binlogdata.RowEvent} RowEvent instance + * @param {binlogdata.IStreamTablesResponse=} [properties] Properties to set + * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse instance */ - RowEvent.create = function create(properties) { - return new RowEvent(properties); + StreamTablesResponse.create = function create(properties) { + return new StreamTablesResponse(properties); }; /** - * Encodes the specified RowEvent message. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. + * Encodes the specified StreamTablesResponse message. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. * @function encode - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static - * @param {binlogdata.IRowEvent} message RowEvent message or plain object to encode + * @param {binlogdata.IStreamTablesResponse} message StreamTablesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RowEvent.encode = function encode(message, writer) { + StreamTablesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); - if (message.row_changes != null && message.row_changes.length) - for (let i = 0; i < message.row_changes.length; ++i) - $root.binlogdata.RowChange.encode(message.row_changes[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard); + if (message.binlog_transaction != null && Object.hasOwnProperty.call(message, "binlog_transaction")) + $root.binlogdata.BinlogTransaction.encode(message.binlog_transaction, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RowEvent message, length delimited. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. + * Encodes the specified StreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.StreamTablesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static - * @param {binlogdata.IRowEvent} message RowEvent message or plain object to encode + * @param {binlogdata.IStreamTablesResponse} message StreamTablesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RowEvent.encodeDelimited = function encodeDelimited(message, writer) { + StreamTablesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RowEvent message from the specified reader or buffer. + * Decodes a StreamTablesResponse message from the specified reader or buffer. * @function decode - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.RowEvent} RowEvent + * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RowEvent.decode = function decode(reader, length) { + StreamTablesResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowEvent(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.StreamTablesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.table_name = reader.string(); - break; - } - case 2: { - if (!(message.row_changes && message.row_changes.length)) - message.row_changes = []; - message.row_changes.push($root.binlogdata.RowChange.decode(reader, reader.uint32())); - break; - } - case 3: { - message.keyspace = reader.string(); - break; - } - case 4: { - message.shard = reader.string(); + message.binlog_transaction = $root.binlogdata.BinlogTransaction.decode(reader, reader.uint32()); break; } default: @@ -67141,169 +67502,128 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a RowEvent message from the specified reader or buffer, length delimited. + * Decodes a StreamTablesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.RowEvent} RowEvent + * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RowEvent.decodeDelimited = function decodeDelimited(reader) { + StreamTablesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RowEvent message. + * Verifies a StreamTablesResponse message. * @function verify - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RowEvent.verify = function verify(message) { + StreamTablesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.table_name != null && message.hasOwnProperty("table_name")) - if (!$util.isString(message.table_name)) - return "table_name: string expected"; - if (message.row_changes != null && message.hasOwnProperty("row_changes")) { - if (!Array.isArray(message.row_changes)) - return "row_changes: array expected"; - for (let i = 0; i < message.row_changes.length; ++i) { - let error = $root.binlogdata.RowChange.verify(message.row_changes[i]); - if (error) - return "row_changes." + error; - } + if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) { + let error = $root.binlogdata.BinlogTransaction.verify(message.binlog_transaction); + if (error) + return "binlog_transaction." + error; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; return null; }; /** - * Creates a RowEvent message from a plain object. Also converts values to their respective internal types. + * Creates a StreamTablesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static * @param {Object.} object Plain object - * @returns {binlogdata.RowEvent} RowEvent + * @returns {binlogdata.StreamTablesResponse} StreamTablesResponse */ - RowEvent.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.RowEvent) + StreamTablesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.StreamTablesResponse) return object; - let message = new $root.binlogdata.RowEvent(); - if (object.table_name != null) - message.table_name = String(object.table_name); - if (object.row_changes) { - if (!Array.isArray(object.row_changes)) - throw TypeError(".binlogdata.RowEvent.row_changes: array expected"); - message.row_changes = []; - for (let i = 0; i < object.row_changes.length; ++i) { - if (typeof object.row_changes[i] !== "object") - throw TypeError(".binlogdata.RowEvent.row_changes: object expected"); - message.row_changes[i] = $root.binlogdata.RowChange.fromObject(object.row_changes[i]); - } + let message = new $root.binlogdata.StreamTablesResponse(); + if (object.binlog_transaction != null) { + if (typeof object.binlog_transaction !== "object") + throw TypeError(".binlogdata.StreamTablesResponse.binlog_transaction: object expected"); + message.binlog_transaction = $root.binlogdata.BinlogTransaction.fromObject(object.binlog_transaction); } - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); return message; }; /** - * Creates a plain object from a RowEvent message. Also converts values to other types if specified. + * Creates a plain object from a StreamTablesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static - * @param {binlogdata.RowEvent} message RowEvent + * @param {binlogdata.StreamTablesResponse} message StreamTablesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RowEvent.toObject = function toObject(message, options) { + StreamTablesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.row_changes = []; - if (options.defaults) { - object.table_name = ""; - object.keyspace = ""; - object.shard = ""; - } - if (message.table_name != null && message.hasOwnProperty("table_name")) - object.table_name = message.table_name; - if (message.row_changes && message.row_changes.length) { - object.row_changes = []; - for (let j = 0; j < message.row_changes.length; ++j) - object.row_changes[j] = $root.binlogdata.RowChange.toObject(message.row_changes[j], options); - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; + if (options.defaults) + object.binlog_transaction = null; + if (message.binlog_transaction != null && message.hasOwnProperty("binlog_transaction")) + object.binlog_transaction = $root.binlogdata.BinlogTransaction.toObject(message.binlog_transaction, options); return object; }; /** - * Converts this RowEvent to JSON. + * Converts this StreamTablesResponse to JSON. * @function toJSON - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @instance * @returns {Object.} JSON object */ - RowEvent.prototype.toJSON = function toJSON() { + StreamTablesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RowEvent + * Gets the default type url for StreamTablesResponse * @function getTypeUrl - * @memberof binlogdata.RowEvent + * @memberof binlogdata.StreamTablesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RowEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamTablesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.RowEvent"; + return typeUrlPrefix + "/binlogdata.StreamTablesResponse"; }; - return RowEvent; + return StreamTablesResponse; })(); - binlogdata.FieldEvent = (function() { + binlogdata.CharsetConversion = (function() { /** - * Properties of a FieldEvent. + * Properties of a CharsetConversion. * @memberof binlogdata - * @interface IFieldEvent - * @property {string|null} [table_name] FieldEvent table_name - * @property {Array.|null} [fields] FieldEvent fields - * @property {string|null} [keyspace] FieldEvent keyspace - * @property {string|null} [shard] FieldEvent shard + * @interface ICharsetConversion + * @property {string|null} [from_charset] CharsetConversion from_charset + * @property {string|null} [to_charset] CharsetConversion to_charset */ /** - * Constructs a new FieldEvent. + * Constructs a new CharsetConversion. * @memberof binlogdata - * @classdesc Represents a FieldEvent. - * @implements IFieldEvent + * @classdesc Represents a CharsetConversion. + * @implements ICharsetConversion * @constructor - * @param {binlogdata.IFieldEvent=} [properties] Properties to set + * @param {binlogdata.ICharsetConversion=} [properties] Properties to set */ - function FieldEvent(properties) { - this.fields = []; + function CharsetConversion(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -67311,120 +67631,89 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * FieldEvent table_name. - * @member {string} table_name - * @memberof binlogdata.FieldEvent - * @instance - */ - FieldEvent.prototype.table_name = ""; - - /** - * FieldEvent fields. - * @member {Array.} fields - * @memberof binlogdata.FieldEvent - * @instance - */ - FieldEvent.prototype.fields = $util.emptyArray; - - /** - * FieldEvent keyspace. - * @member {string} keyspace - * @memberof binlogdata.FieldEvent + * CharsetConversion from_charset. + * @member {string} from_charset + * @memberof binlogdata.CharsetConversion * @instance */ - FieldEvent.prototype.keyspace = ""; + CharsetConversion.prototype.from_charset = ""; /** - * FieldEvent shard. - * @member {string} shard - * @memberof binlogdata.FieldEvent + * CharsetConversion to_charset. + * @member {string} to_charset + * @memberof binlogdata.CharsetConversion * @instance */ - FieldEvent.prototype.shard = ""; + CharsetConversion.prototype.to_charset = ""; /** - * Creates a new FieldEvent instance using the specified properties. + * Creates a new CharsetConversion instance using the specified properties. * @function create - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static - * @param {binlogdata.IFieldEvent=} [properties] Properties to set - * @returns {binlogdata.FieldEvent} FieldEvent instance + * @param {binlogdata.ICharsetConversion=} [properties] Properties to set + * @returns {binlogdata.CharsetConversion} CharsetConversion instance */ - FieldEvent.create = function create(properties) { - return new FieldEvent(properties); + CharsetConversion.create = function create(properties) { + return new CharsetConversion(properties); }; /** - * Encodes the specified FieldEvent message. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. + * Encodes the specified CharsetConversion message. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. * @function encode - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static - * @param {binlogdata.IFieldEvent} message FieldEvent message or plain object to encode + * @param {binlogdata.ICharsetConversion} message CharsetConversion message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FieldEvent.encode = function encode(message, writer) { + CharsetConversion.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); - if (message.fields != null && message.fields.length) - for (let i = 0; i < message.fields.length; ++i) - $root.query.Field.encode(message.fields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard); + if (message.from_charset != null && Object.hasOwnProperty.call(message, "from_charset")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.from_charset); + if (message.to_charset != null && Object.hasOwnProperty.call(message, "to_charset")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.to_charset); return writer; }; /** - * Encodes the specified FieldEvent message, length delimited. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. + * Encodes the specified CharsetConversion message, length delimited. Does not implicitly {@link binlogdata.CharsetConversion.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static - * @param {binlogdata.IFieldEvent} message FieldEvent message or plain object to encode + * @param {binlogdata.ICharsetConversion} message CharsetConversion message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FieldEvent.encodeDelimited = function encodeDelimited(message, writer) { + CharsetConversion.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a FieldEvent message from the specified reader or buffer. + * Decodes a CharsetConversion message from the specified reader or buffer. * @function decode - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.FieldEvent} FieldEvent + * @returns {binlogdata.CharsetConversion} CharsetConversion * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FieldEvent.decode = function decode(reader, length) { + CharsetConversion.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.FieldEvent(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.CharsetConversion(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.table_name = reader.string(); + message.from_charset = reader.string(); break; } case 2: { - if (!(message.fields && message.fields.length)) - message.fields = []; - message.fields.push($root.query.Field.decode(reader, reader.uint32())); - break; - } - case 3: { - message.keyspace = reader.string(); - break; - } - case 4: { - message.shard = reader.string(); + message.to_charset = reader.string(); break; } default: @@ -67436,169 +67725,141 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a FieldEvent message from the specified reader or buffer, length delimited. + * Decodes a CharsetConversion message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.FieldEvent} FieldEvent + * @returns {binlogdata.CharsetConversion} CharsetConversion * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FieldEvent.decodeDelimited = function decodeDelimited(reader) { + CharsetConversion.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a FieldEvent message. + * Verifies a CharsetConversion message. * @function verify - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - FieldEvent.verify = function verify(message) { + CharsetConversion.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.table_name != null && message.hasOwnProperty("table_name")) - if (!$util.isString(message.table_name)) - return "table_name: string expected"; - if (message.fields != null && message.hasOwnProperty("fields")) { - if (!Array.isArray(message.fields)) - return "fields: array expected"; - for (let i = 0; i < message.fields.length; ++i) { - let error = $root.query.Field.verify(message.fields[i]); - if (error) - return "fields." + error; - } - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; + if (message.from_charset != null && message.hasOwnProperty("from_charset")) + if (!$util.isString(message.from_charset)) + return "from_charset: string expected"; + if (message.to_charset != null && message.hasOwnProperty("to_charset")) + if (!$util.isString(message.to_charset)) + return "to_charset: string expected"; return null; }; /** - * Creates a FieldEvent message from a plain object. Also converts values to their respective internal types. + * Creates a CharsetConversion message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static * @param {Object.} object Plain object - * @returns {binlogdata.FieldEvent} FieldEvent + * @returns {binlogdata.CharsetConversion} CharsetConversion */ - FieldEvent.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.FieldEvent) + CharsetConversion.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.CharsetConversion) return object; - let message = new $root.binlogdata.FieldEvent(); - if (object.table_name != null) - message.table_name = String(object.table_name); - if (object.fields) { - if (!Array.isArray(object.fields)) - throw TypeError(".binlogdata.FieldEvent.fields: array expected"); - message.fields = []; - for (let i = 0; i < object.fields.length; ++i) { - if (typeof object.fields[i] !== "object") - throw TypeError(".binlogdata.FieldEvent.fields: object expected"); - message.fields[i] = $root.query.Field.fromObject(object.fields[i]); - } - } - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); + let message = new $root.binlogdata.CharsetConversion(); + if (object.from_charset != null) + message.from_charset = String(object.from_charset); + if (object.to_charset != null) + message.to_charset = String(object.to_charset); return message; }; /** - * Creates a plain object from a FieldEvent message. Also converts values to other types if specified. + * Creates a plain object from a CharsetConversion message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static - * @param {binlogdata.FieldEvent} message FieldEvent + * @param {binlogdata.CharsetConversion} message CharsetConversion * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - FieldEvent.toObject = function toObject(message, options) { + CharsetConversion.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.fields = []; if (options.defaults) { - object.table_name = ""; - object.keyspace = ""; - object.shard = ""; - } - if (message.table_name != null && message.hasOwnProperty("table_name")) - object.table_name = message.table_name; - if (message.fields && message.fields.length) { - object.fields = []; - for (let j = 0; j < message.fields.length; ++j) - object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + object.from_charset = ""; + object.to_charset = ""; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; + if (message.from_charset != null && message.hasOwnProperty("from_charset")) + object.from_charset = message.from_charset; + if (message.to_charset != null && message.hasOwnProperty("to_charset")) + object.to_charset = message.to_charset; return object; }; /** - * Converts this FieldEvent to JSON. + * Converts this CharsetConversion to JSON. * @function toJSON - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @instance * @returns {Object.} JSON object */ - FieldEvent.prototype.toJSON = function toJSON() { + CharsetConversion.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for FieldEvent + * Gets the default type url for CharsetConversion * @function getTypeUrl - * @memberof binlogdata.FieldEvent + * @memberof binlogdata.CharsetConversion * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - FieldEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CharsetConversion.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.FieldEvent"; + return typeUrlPrefix + "/binlogdata.CharsetConversion"; }; - return FieldEvent; + return CharsetConversion; })(); - binlogdata.ShardGtid = (function() { + binlogdata.Rule = (function() { /** - * Properties of a ShardGtid. + * Properties of a Rule. * @memberof binlogdata - * @interface IShardGtid - * @property {string|null} [keyspace] ShardGtid keyspace - * @property {string|null} [shard] ShardGtid shard - * @property {string|null} [gtid] ShardGtid gtid - * @property {Array.|null} [table_p_ks] ShardGtid table_p_ks + * @interface IRule + * @property {string|null} [match] Rule match + * @property {string|null} [filter] Rule filter + * @property {Object.|null} [convert_enum_to_text] Rule convert_enum_to_text + * @property {Object.|null} [convert_charset] Rule convert_charset + * @property {string|null} [source_unique_key_columns] Rule source_unique_key_columns + * @property {string|null} [target_unique_key_columns] Rule target_unique_key_columns + * @property {string|null} [source_unique_key_target_columns] Rule source_unique_key_target_columns + * @property {Object.|null} [convert_int_to_enum] Rule convert_int_to_enum */ /** - * Constructs a new ShardGtid. + * Constructs a new Rule. * @memberof binlogdata - * @classdesc Represents a ShardGtid. - * @implements IShardGtid + * @classdesc Represents a Rule. + * @implements IRule * @constructor - * @param {binlogdata.IShardGtid=} [properties] Properties to set + * @param {binlogdata.IRule=} [properties] Properties to set */ - function ShardGtid(properties) { - this.table_p_ks = []; + function Rule(properties) { + this.convert_enum_to_text = {}; + this.convert_charset = {}; + this.convert_int_to_enum = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -67606,120 +67867,235 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * ShardGtid keyspace. - * @member {string} keyspace - * @memberof binlogdata.ShardGtid + * Rule match. + * @member {string} match + * @memberof binlogdata.Rule * @instance */ - ShardGtid.prototype.keyspace = ""; + Rule.prototype.match = ""; /** - * ShardGtid shard. - * @member {string} shard - * @memberof binlogdata.ShardGtid + * Rule filter. + * @member {string} filter + * @memberof binlogdata.Rule * @instance */ - ShardGtid.prototype.shard = ""; + Rule.prototype.filter = ""; /** - * ShardGtid gtid. - * @member {string} gtid - * @memberof binlogdata.ShardGtid + * Rule convert_enum_to_text. + * @member {Object.} convert_enum_to_text + * @memberof binlogdata.Rule * @instance */ - ShardGtid.prototype.gtid = ""; + Rule.prototype.convert_enum_to_text = $util.emptyObject; /** - * ShardGtid table_p_ks. - * @member {Array.} table_p_ks - * @memberof binlogdata.ShardGtid + * Rule convert_charset. + * @member {Object.} convert_charset + * @memberof binlogdata.Rule * @instance */ - ShardGtid.prototype.table_p_ks = $util.emptyArray; + Rule.prototype.convert_charset = $util.emptyObject; /** - * Creates a new ShardGtid instance using the specified properties. + * Rule source_unique_key_columns. + * @member {string} source_unique_key_columns + * @memberof binlogdata.Rule + * @instance + */ + Rule.prototype.source_unique_key_columns = ""; + + /** + * Rule target_unique_key_columns. + * @member {string} target_unique_key_columns + * @memberof binlogdata.Rule + * @instance + */ + Rule.prototype.target_unique_key_columns = ""; + + /** + * Rule source_unique_key_target_columns. + * @member {string} source_unique_key_target_columns + * @memberof binlogdata.Rule + * @instance + */ + Rule.prototype.source_unique_key_target_columns = ""; + + /** + * Rule convert_int_to_enum. + * @member {Object.} convert_int_to_enum + * @memberof binlogdata.Rule + * @instance + */ + Rule.prototype.convert_int_to_enum = $util.emptyObject; + + /** + * Creates a new Rule instance using the specified properties. * @function create - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static - * @param {binlogdata.IShardGtid=} [properties] Properties to set - * @returns {binlogdata.ShardGtid} ShardGtid instance + * @param {binlogdata.IRule=} [properties] Properties to set + * @returns {binlogdata.Rule} Rule instance */ - ShardGtid.create = function create(properties) { - return new ShardGtid(properties); + Rule.create = function create(properties) { + return new Rule(properties); }; /** - * Encodes the specified ShardGtid message. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. + * Encodes the specified Rule message. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. * @function encode - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static - * @param {binlogdata.IShardGtid} message ShardGtid message or plain object to encode + * @param {binlogdata.IRule} message Rule message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardGtid.encode = function encode(message, writer) { + Rule.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); - if (message.table_p_ks != null && message.table_p_ks.length) - for (let i = 0; i < message.table_p_ks.length; ++i) - $root.binlogdata.TableLastPK.encode(message.table_p_ks[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.match != null && Object.hasOwnProperty.call(message, "match")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.match); + if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.filter); + if (message.convert_enum_to_text != null && Object.hasOwnProperty.call(message, "convert_enum_to_text")) + for (let keys = Object.keys(message.convert_enum_to_text), i = 0; i < keys.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.convert_enum_to_text[keys[i]]).ldelim(); + if (message.convert_charset != null && Object.hasOwnProperty.call(message, "convert_charset")) + for (let keys = Object.keys(message.convert_charset), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 4, wireType 2 =*/34).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.binlogdata.CharsetConversion.encode(message.convert_charset[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.source_unique_key_columns != null && Object.hasOwnProperty.call(message, "source_unique_key_columns")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_unique_key_columns); + if (message.target_unique_key_columns != null && Object.hasOwnProperty.call(message, "target_unique_key_columns")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.target_unique_key_columns); + if (message.source_unique_key_target_columns != null && Object.hasOwnProperty.call(message, "source_unique_key_target_columns")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.source_unique_key_target_columns); + if (message.convert_int_to_enum != null && Object.hasOwnProperty.call(message, "convert_int_to_enum")) + for (let keys = Object.keys(message.convert_int_to_enum), i = 0; i < keys.length; ++i) + writer.uint32(/* id 8, wireType 2 =*/66).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 0 =*/16).bool(message.convert_int_to_enum[keys[i]]).ldelim(); return writer; }; /** - * Encodes the specified ShardGtid message, length delimited. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. + * Encodes the specified Rule message, length delimited. Does not implicitly {@link binlogdata.Rule.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static - * @param {binlogdata.IShardGtid} message ShardGtid message or plain object to encode + * @param {binlogdata.IRule} message Rule message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardGtid.encodeDelimited = function encodeDelimited(message, writer) { + Rule.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardGtid message from the specified reader or buffer. + * Decodes a Rule message from the specified reader or buffer. * @function decode - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.ShardGtid} ShardGtid + * @returns {binlogdata.Rule} Rule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardGtid.decode = function decode(reader, length) { + Rule.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.ShardGtid(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Rule(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.match = reader.string(); break; } case 2: { - message.shard = reader.string(); + message.filter = reader.string(); break; } case 3: { - message.gtid = reader.string(); + if (message.convert_enum_to_text === $util.emptyObject) + message.convert_enum_to_text = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = ""; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = reader.string(); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.convert_enum_to_text[key] = value; break; } case 4: { - if (!(message.table_p_ks && message.table_p_ks.length)) - message.table_p_ks = []; - message.table_p_ks.push($root.binlogdata.TableLastPK.decode(reader, reader.uint32())); + if (message.convert_charset === $util.emptyObject) + message.convert_charset = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.binlogdata.CharsetConversion.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.convert_charset[key] = value; + break; + } + case 5: { + message.source_unique_key_columns = reader.string(); + break; + } + case 6: { + message.target_unique_key_columns = reader.string(); + break; + } + case 7: { + message.source_unique_key_target_columns = reader.string(); + break; + } + case 8: { + if (message.convert_int_to_enum === $util.emptyObject) + message.convert_int_to_enum = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = false; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = reader.bool(); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.convert_int_to_enum[key] = value; break; } default: @@ -67731,166 +68107,230 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a ShardGtid message from the specified reader or buffer, length delimited. + * Decodes a Rule message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.ShardGtid} ShardGtid + * @returns {binlogdata.Rule} Rule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardGtid.decodeDelimited = function decodeDelimited(reader) { + Rule.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardGtid message. + * Verifies a Rule message. * @function verify - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardGtid.verify = function verify(message) { + Rule.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.gtid != null && message.hasOwnProperty("gtid")) - if (!$util.isString(message.gtid)) - return "gtid: string expected"; - if (message.table_p_ks != null && message.hasOwnProperty("table_p_ks")) { - if (!Array.isArray(message.table_p_ks)) - return "table_p_ks: array expected"; - for (let i = 0; i < message.table_p_ks.length; ++i) { - let error = $root.binlogdata.TableLastPK.verify(message.table_p_ks[i]); + if (message.match != null && message.hasOwnProperty("match")) + if (!$util.isString(message.match)) + return "match: string expected"; + if (message.filter != null && message.hasOwnProperty("filter")) + if (!$util.isString(message.filter)) + return "filter: string expected"; + if (message.convert_enum_to_text != null && message.hasOwnProperty("convert_enum_to_text")) { + if (!$util.isObject(message.convert_enum_to_text)) + return "convert_enum_to_text: object expected"; + let key = Object.keys(message.convert_enum_to_text); + for (let i = 0; i < key.length; ++i) + if (!$util.isString(message.convert_enum_to_text[key[i]])) + return "convert_enum_to_text: string{k:string} expected"; + } + if (message.convert_charset != null && message.hasOwnProperty("convert_charset")) { + if (!$util.isObject(message.convert_charset)) + return "convert_charset: object expected"; + let key = Object.keys(message.convert_charset); + for (let i = 0; i < key.length; ++i) { + let error = $root.binlogdata.CharsetConversion.verify(message.convert_charset[key[i]]); if (error) - return "table_p_ks." + error; + return "convert_charset." + error; } } + if (message.source_unique_key_columns != null && message.hasOwnProperty("source_unique_key_columns")) + if (!$util.isString(message.source_unique_key_columns)) + return "source_unique_key_columns: string expected"; + if (message.target_unique_key_columns != null && message.hasOwnProperty("target_unique_key_columns")) + if (!$util.isString(message.target_unique_key_columns)) + return "target_unique_key_columns: string expected"; + if (message.source_unique_key_target_columns != null && message.hasOwnProperty("source_unique_key_target_columns")) + if (!$util.isString(message.source_unique_key_target_columns)) + return "source_unique_key_target_columns: string expected"; + if (message.convert_int_to_enum != null && message.hasOwnProperty("convert_int_to_enum")) { + if (!$util.isObject(message.convert_int_to_enum)) + return "convert_int_to_enum: object expected"; + let key = Object.keys(message.convert_int_to_enum); + for (let i = 0; i < key.length; ++i) + if (typeof message.convert_int_to_enum[key[i]] !== "boolean") + return "convert_int_to_enum: boolean{k:string} expected"; + } return null; }; /** - * Creates a ShardGtid message from a plain object. Also converts values to their respective internal types. + * Creates a Rule message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static * @param {Object.} object Plain object - * @returns {binlogdata.ShardGtid} ShardGtid + * @returns {binlogdata.Rule} Rule */ - ShardGtid.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.ShardGtid) + Rule.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.Rule) return object; - let message = new $root.binlogdata.ShardGtid(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.gtid != null) - message.gtid = String(object.gtid); - if (object.table_p_ks) { - if (!Array.isArray(object.table_p_ks)) - throw TypeError(".binlogdata.ShardGtid.table_p_ks: array expected"); - message.table_p_ks = []; - for (let i = 0; i < object.table_p_ks.length; ++i) { - if (typeof object.table_p_ks[i] !== "object") - throw TypeError(".binlogdata.ShardGtid.table_p_ks: object expected"); - message.table_p_ks[i] = $root.binlogdata.TableLastPK.fromObject(object.table_p_ks[i]); - } - } - return message; - }; - - /** - * Creates a plain object from a ShardGtid message. Also converts values to other types if specified. - * @function toObject - * @memberof binlogdata.ShardGtid + let message = new $root.binlogdata.Rule(); + if (object.match != null) + message.match = String(object.match); + if (object.filter != null) + message.filter = String(object.filter); + if (object.convert_enum_to_text) { + if (typeof object.convert_enum_to_text !== "object") + throw TypeError(".binlogdata.Rule.convert_enum_to_text: object expected"); + message.convert_enum_to_text = {}; + for (let keys = Object.keys(object.convert_enum_to_text), i = 0; i < keys.length; ++i) + message.convert_enum_to_text[keys[i]] = String(object.convert_enum_to_text[keys[i]]); + } + if (object.convert_charset) { + if (typeof object.convert_charset !== "object") + throw TypeError(".binlogdata.Rule.convert_charset: object expected"); + message.convert_charset = {}; + for (let keys = Object.keys(object.convert_charset), i = 0; i < keys.length; ++i) { + if (typeof object.convert_charset[keys[i]] !== "object") + throw TypeError(".binlogdata.Rule.convert_charset: object expected"); + message.convert_charset[keys[i]] = $root.binlogdata.CharsetConversion.fromObject(object.convert_charset[keys[i]]); + } + } + if (object.source_unique_key_columns != null) + message.source_unique_key_columns = String(object.source_unique_key_columns); + if (object.target_unique_key_columns != null) + message.target_unique_key_columns = String(object.target_unique_key_columns); + if (object.source_unique_key_target_columns != null) + message.source_unique_key_target_columns = String(object.source_unique_key_target_columns); + if (object.convert_int_to_enum) { + if (typeof object.convert_int_to_enum !== "object") + throw TypeError(".binlogdata.Rule.convert_int_to_enum: object expected"); + message.convert_int_to_enum = {}; + for (let keys = Object.keys(object.convert_int_to_enum), i = 0; i < keys.length; ++i) + message.convert_int_to_enum[keys[i]] = Boolean(object.convert_int_to_enum[keys[i]]); + } + return message; + }; + + /** + * Creates a plain object from a Rule message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.Rule * @static - * @param {binlogdata.ShardGtid} message ShardGtid + * @param {binlogdata.Rule} message Rule * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardGtid.toObject = function toObject(message, options) { + Rule.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.table_p_ks = []; + if (options.objects || options.defaults) { + object.convert_enum_to_text = {}; + object.convert_charset = {}; + object.convert_int_to_enum = {}; + } if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.gtid = ""; + object.match = ""; + object.filter = ""; + object.source_unique_key_columns = ""; + object.target_unique_key_columns = ""; + object.source_unique_key_target_columns = ""; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.gtid != null && message.hasOwnProperty("gtid")) - object.gtid = message.gtid; - if (message.table_p_ks && message.table_p_ks.length) { - object.table_p_ks = []; - for (let j = 0; j < message.table_p_ks.length; ++j) - object.table_p_ks[j] = $root.binlogdata.TableLastPK.toObject(message.table_p_ks[j], options); + if (message.match != null && message.hasOwnProperty("match")) + object.match = message.match; + if (message.filter != null && message.hasOwnProperty("filter")) + object.filter = message.filter; + let keys2; + if (message.convert_enum_to_text && (keys2 = Object.keys(message.convert_enum_to_text)).length) { + object.convert_enum_to_text = {}; + for (let j = 0; j < keys2.length; ++j) + object.convert_enum_to_text[keys2[j]] = message.convert_enum_to_text[keys2[j]]; + } + if (message.convert_charset && (keys2 = Object.keys(message.convert_charset)).length) { + object.convert_charset = {}; + for (let j = 0; j < keys2.length; ++j) + object.convert_charset[keys2[j]] = $root.binlogdata.CharsetConversion.toObject(message.convert_charset[keys2[j]], options); + } + if (message.source_unique_key_columns != null && message.hasOwnProperty("source_unique_key_columns")) + object.source_unique_key_columns = message.source_unique_key_columns; + if (message.target_unique_key_columns != null && message.hasOwnProperty("target_unique_key_columns")) + object.target_unique_key_columns = message.target_unique_key_columns; + if (message.source_unique_key_target_columns != null && message.hasOwnProperty("source_unique_key_target_columns")) + object.source_unique_key_target_columns = message.source_unique_key_target_columns; + if (message.convert_int_to_enum && (keys2 = Object.keys(message.convert_int_to_enum)).length) { + object.convert_int_to_enum = {}; + for (let j = 0; j < keys2.length; ++j) + object.convert_int_to_enum[keys2[j]] = message.convert_int_to_enum[keys2[j]]; } return object; }; /** - * Converts this ShardGtid to JSON. + * Converts this Rule to JSON. * @function toJSON - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @instance * @returns {Object.} JSON object */ - ShardGtid.prototype.toJSON = function toJSON() { + Rule.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardGtid + * Gets the default type url for Rule * @function getTypeUrl - * @memberof binlogdata.ShardGtid + * @memberof binlogdata.Rule * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardGtid.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Rule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.ShardGtid"; + return typeUrlPrefix + "/binlogdata.Rule"; }; - return ShardGtid; + return Rule; })(); - binlogdata.VGtid = (function() { + binlogdata.Filter = (function() { /** - * Properties of a VGtid. + * Properties of a Filter. * @memberof binlogdata - * @interface IVGtid - * @property {Array.|null} [shard_gtids] VGtid shard_gtids + * @interface IFilter + * @property {Array.|null} [rules] Filter rules + * @property {binlogdata.Filter.FieldEventMode|null} [field_event_mode] Filter field_event_mode + * @property {number|Long|null} [workflow_type] Filter workflow_type + * @property {string|null} [workflow_name] Filter workflow_name */ /** - * Constructs a new VGtid. + * Constructs a new Filter. * @memberof binlogdata - * @classdesc Represents a VGtid. - * @implements IVGtid + * @classdesc Represents a Filter. + * @implements IFilter * @constructor - * @param {binlogdata.IVGtid=} [properties] Properties to set + * @param {binlogdata.IFilter=} [properties] Properties to set */ - function VGtid(properties) { - this.shard_gtids = []; + function Filter(properties) { + this.rules = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -67898,78 +68338,120 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VGtid shard_gtids. - * @member {Array.} shard_gtids - * @memberof binlogdata.VGtid + * Filter rules. + * @member {Array.} rules + * @memberof binlogdata.Filter * @instance */ - VGtid.prototype.shard_gtids = $util.emptyArray; + Filter.prototype.rules = $util.emptyArray; /** - * Creates a new VGtid instance using the specified properties. + * Filter field_event_mode. + * @member {binlogdata.Filter.FieldEventMode} field_event_mode + * @memberof binlogdata.Filter + * @instance + */ + Filter.prototype.field_event_mode = 0; + + /** + * Filter workflow_type. + * @member {number|Long} workflow_type + * @memberof binlogdata.Filter + * @instance + */ + Filter.prototype.workflow_type = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Filter workflow_name. + * @member {string} workflow_name + * @memberof binlogdata.Filter + * @instance + */ + Filter.prototype.workflow_name = ""; + + /** + * Creates a new Filter instance using the specified properties. * @function create - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static - * @param {binlogdata.IVGtid=} [properties] Properties to set - * @returns {binlogdata.VGtid} VGtid instance + * @param {binlogdata.IFilter=} [properties] Properties to set + * @returns {binlogdata.Filter} Filter instance */ - VGtid.create = function create(properties) { - return new VGtid(properties); + Filter.create = function create(properties) { + return new Filter(properties); }; /** - * Encodes the specified VGtid message. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. + * Encodes the specified Filter message. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. * @function encode - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static - * @param {binlogdata.IVGtid} message VGtid message or plain object to encode + * @param {binlogdata.IFilter} message Filter message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VGtid.encode = function encode(message, writer) { + Filter.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard_gtids != null && message.shard_gtids.length) - for (let i = 0; i < message.shard_gtids.length; ++i) - $root.binlogdata.ShardGtid.encode(message.shard_gtids[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.rules != null && message.rules.length) + for (let i = 0; i < message.rules.length; ++i) + $root.binlogdata.Rule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.field_event_mode != null && Object.hasOwnProperty.call(message, "field_event_mode")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.field_event_mode); + if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.workflow_type); + if (message.workflow_name != null && Object.hasOwnProperty.call(message, "workflow_name")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.workflow_name); return writer; }; /** - * Encodes the specified VGtid message, length delimited. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. + * Encodes the specified Filter message, length delimited. Does not implicitly {@link binlogdata.Filter.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static - * @param {binlogdata.IVGtid} message VGtid message or plain object to encode + * @param {binlogdata.IFilter} message Filter message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VGtid.encodeDelimited = function encodeDelimited(message, writer) { + Filter.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VGtid message from the specified reader or buffer. + * Decodes a Filter message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VGtid} VGtid + * @returns {binlogdata.Filter} Filter * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VGtid.decode = function decode(reader, length) { + Filter.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VGtid(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Filter(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.shard_gtids && message.shard_gtids.length)) - message.shard_gtids = []; - message.shard_gtids.push($root.binlogdata.ShardGtid.decode(reader, reader.uint32())); + if (!(message.rules && message.rules.length)) + message.rules = []; + message.rules.push($root.binlogdata.Rule.decode(reader, reader.uint32())); + break; + } + case 2: { + message.field_event_mode = reader.int32(); + break; + } + case 3: { + message.workflow_type = reader.int64(); + break; + } + case 4: { + message.workflow_name = reader.string(); break; } default: @@ -67981,140 +68463,302 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VGtid message from the specified reader or buffer, length delimited. + * Decodes a Filter message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VGtid} VGtid + * @returns {binlogdata.Filter} Filter * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VGtid.decodeDelimited = function decodeDelimited(reader) { + Filter.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VGtid message. + * Verifies a Filter message. * @function verify - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VGtid.verify = function verify(message) { + Filter.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard_gtids != null && message.hasOwnProperty("shard_gtids")) { - if (!Array.isArray(message.shard_gtids)) - return "shard_gtids: array expected"; - for (let i = 0; i < message.shard_gtids.length; ++i) { - let error = $root.binlogdata.ShardGtid.verify(message.shard_gtids[i]); + 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.binlogdata.Rule.verify(message.rules[i]); if (error) - return "shard_gtids." + error; + return "rules." + error; } } + if (message.field_event_mode != null && message.hasOwnProperty("field_event_mode")) + switch (message.field_event_mode) { + default: + return "field_event_mode: enum value expected"; + case 0: + case 1: + break; + } + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + if (!$util.isInteger(message.workflow_type) && !(message.workflow_type && $util.isInteger(message.workflow_type.low) && $util.isInteger(message.workflow_type.high))) + return "workflow_type: integer|Long expected"; + if (message.workflow_name != null && message.hasOwnProperty("workflow_name")) + if (!$util.isString(message.workflow_name)) + return "workflow_name: string expected"; return null; }; /** - * Creates a VGtid message from a plain object. Also converts values to their respective internal types. + * Creates a Filter message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static * @param {Object.} object Plain object - * @returns {binlogdata.VGtid} VGtid + * @returns {binlogdata.Filter} Filter */ - VGtid.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VGtid) + Filter.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.Filter) return object; - let message = new $root.binlogdata.VGtid(); - if (object.shard_gtids) { - if (!Array.isArray(object.shard_gtids)) - throw TypeError(".binlogdata.VGtid.shard_gtids: array expected"); - message.shard_gtids = []; - for (let i = 0; i < object.shard_gtids.length; ++i) { - if (typeof object.shard_gtids[i] !== "object") - throw TypeError(".binlogdata.VGtid.shard_gtids: object expected"); - message.shard_gtids[i] = $root.binlogdata.ShardGtid.fromObject(object.shard_gtids[i]); + let message = new $root.binlogdata.Filter(); + if (object.rules) { + if (!Array.isArray(object.rules)) + throw TypeError(".binlogdata.Filter.rules: array expected"); + message.rules = []; + for (let i = 0; i < object.rules.length; ++i) { + if (typeof object.rules[i] !== "object") + throw TypeError(".binlogdata.Filter.rules: object expected"); + message.rules[i] = $root.binlogdata.Rule.fromObject(object.rules[i]); + } + } + switch (object.field_event_mode) { + default: + if (typeof object.field_event_mode === "number") { + message.field_event_mode = object.field_event_mode; + break; } + break; + case "ERR_ON_MISMATCH": + case 0: + message.field_event_mode = 0; + break; + case "BEST_EFFORT": + case 1: + message.field_event_mode = 1; + break; } + if (object.workflow_type != null) + if ($util.Long) + (message.workflow_type = $util.Long.fromValue(object.workflow_type)).unsigned = false; + else if (typeof object.workflow_type === "string") + message.workflow_type = parseInt(object.workflow_type, 10); + else if (typeof object.workflow_type === "number") + message.workflow_type = object.workflow_type; + else if (typeof object.workflow_type === "object") + message.workflow_type = new $util.LongBits(object.workflow_type.low >>> 0, object.workflow_type.high >>> 0).toNumber(); + if (object.workflow_name != null) + message.workflow_name = String(object.workflow_name); return message; }; /** - * Creates a plain object from a VGtid message. Also converts values to other types if specified. + * Creates a plain object from a Filter message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static - * @param {binlogdata.VGtid} message VGtid + * @param {binlogdata.Filter} message Filter * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VGtid.toObject = function toObject(message, options) { + Filter.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.shard_gtids = []; - if (message.shard_gtids && message.shard_gtids.length) { - object.shard_gtids = []; - for (let j = 0; j < message.shard_gtids.length; ++j) - object.shard_gtids[j] = $root.binlogdata.ShardGtid.toObject(message.shard_gtids[j], options); + object.rules = []; + if (options.defaults) { + object.field_event_mode = options.enums === String ? "ERR_ON_MISMATCH" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.workflow_type = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.workflow_type = options.longs === String ? "0" : 0; + object.workflow_name = ""; } + if (message.rules && message.rules.length) { + object.rules = []; + for (let j = 0; j < message.rules.length; ++j) + object.rules[j] = $root.binlogdata.Rule.toObject(message.rules[j], options); + } + if (message.field_event_mode != null && message.hasOwnProperty("field_event_mode")) + object.field_event_mode = options.enums === String ? $root.binlogdata.Filter.FieldEventMode[message.field_event_mode] === undefined ? message.field_event_mode : $root.binlogdata.Filter.FieldEventMode[message.field_event_mode] : message.field_event_mode; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + if (typeof message.workflow_type === "number") + object.workflow_type = options.longs === String ? String(message.workflow_type) : message.workflow_type; + else + object.workflow_type = options.longs === String ? $util.Long.prototype.toString.call(message.workflow_type) : options.longs === Number ? new $util.LongBits(message.workflow_type.low >>> 0, message.workflow_type.high >>> 0).toNumber() : message.workflow_type; + if (message.workflow_name != null && message.hasOwnProperty("workflow_name")) + object.workflow_name = message.workflow_name; return object; }; /** - * Converts this VGtid to JSON. + * Converts this Filter to JSON. * @function toJSON - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @instance * @returns {Object.} JSON object */ - VGtid.prototype.toJSON = function toJSON() { + Filter.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VGtid + * Gets the default type url for Filter * @function getTypeUrl - * @memberof binlogdata.VGtid + * @memberof binlogdata.Filter * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VGtid.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Filter.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VGtid"; + return typeUrlPrefix + "/binlogdata.Filter"; }; - return VGtid; + /** + * FieldEventMode enum. + * @name binlogdata.Filter.FieldEventMode + * @enum {number} + * @property {number} ERR_ON_MISMATCH=0 ERR_ON_MISMATCH value + * @property {number} BEST_EFFORT=1 BEST_EFFORT value + */ + Filter.FieldEventMode = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "ERR_ON_MISMATCH"] = 0; + values[valuesById[1] = "BEST_EFFORT"] = 1; + return values; + })(); + + return Filter; })(); - binlogdata.KeyspaceShard = (function() { + /** + * OnDDLAction enum. + * @name binlogdata.OnDDLAction + * @enum {number} + * @property {number} IGNORE=0 IGNORE value + * @property {number} STOP=1 STOP value + * @property {number} EXEC=2 EXEC value + * @property {number} EXEC_IGNORE=3 EXEC_IGNORE value + */ + binlogdata.OnDDLAction = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "IGNORE"] = 0; + values[valuesById[1] = "STOP"] = 1; + values[valuesById[2] = "EXEC"] = 2; + values[valuesById[3] = "EXEC_IGNORE"] = 3; + return values; + })(); + + /** + * VReplicationWorkflowType enum. + * @name binlogdata.VReplicationWorkflowType + * @enum {number} + * @property {number} Materialize=0 Materialize value + * @property {number} MoveTables=1 MoveTables value + * @property {number} CreateLookupIndex=2 CreateLookupIndex value + * @property {number} Migrate=3 Migrate value + * @property {number} Reshard=4 Reshard value + * @property {number} OnlineDDL=5 OnlineDDL value + */ + binlogdata.VReplicationWorkflowType = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "Materialize"] = 0; + values[valuesById[1] = "MoveTables"] = 1; + values[valuesById[2] = "CreateLookupIndex"] = 2; + values[valuesById[3] = "Migrate"] = 3; + values[valuesById[4] = "Reshard"] = 4; + values[valuesById[5] = "OnlineDDL"] = 5; + return values; + })(); + + /** + * VReplicationWorkflowSubType enum. + * @name binlogdata.VReplicationWorkflowSubType + * @enum {number} + * @property {number} None=0 None value + * @property {number} Partial=1 Partial value + */ + binlogdata.VReplicationWorkflowSubType = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "None"] = 0; + values[valuesById[1] = "Partial"] = 1; + return values; + })(); + + /** + * VReplicationWorkflowState enum. + * @name binlogdata.VReplicationWorkflowState + * @enum {number} + * @property {number} Unknown=0 Unknown value + * @property {number} Init=1 Init value + * @property {number} Stopped=2 Stopped value + * @property {number} Copying=3 Copying value + * @property {number} Running=4 Running value + * @property {number} Error=5 Error value + * @property {number} Lagging=6 Lagging value + */ + binlogdata.VReplicationWorkflowState = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "Unknown"] = 0; + values[valuesById[1] = "Init"] = 1; + values[valuesById[2] = "Stopped"] = 2; + values[valuesById[3] = "Copying"] = 3; + values[valuesById[4] = "Running"] = 4; + values[valuesById[5] = "Error"] = 5; + values[valuesById[6] = "Lagging"] = 6; + return values; + })(); + + binlogdata.BinlogSource = (function() { /** - * Properties of a KeyspaceShard. + * Properties of a BinlogSource. * @memberof binlogdata - * @interface IKeyspaceShard - * @property {string|null} [keyspace] KeyspaceShard keyspace - * @property {string|null} [shard] KeyspaceShard shard + * @interface IBinlogSource + * @property {string|null} [keyspace] BinlogSource keyspace + * @property {string|null} [shard] BinlogSource shard + * @property {topodata.TabletType|null} [tablet_type] BinlogSource tablet_type + * @property {topodata.IKeyRange|null} [key_range] BinlogSource key_range + * @property {Array.|null} [tables] BinlogSource tables + * @property {binlogdata.IFilter|null} [filter] BinlogSource filter + * @property {binlogdata.OnDDLAction|null} [on_ddl] BinlogSource on_ddl + * @property {string|null} [external_mysql] BinlogSource external_mysql + * @property {boolean|null} [stop_after_copy] BinlogSource stop_after_copy + * @property {string|null} [external_cluster] BinlogSource external_cluster + * @property {string|null} [source_time_zone] BinlogSource source_time_zone + * @property {string|null} [target_time_zone] BinlogSource target_time_zone */ /** - * Constructs a new KeyspaceShard. + * Constructs a new BinlogSource. * @memberof binlogdata - * @classdesc Represents a KeyspaceShard. - * @implements IKeyspaceShard + * @classdesc Represents a BinlogSource. + * @implements IBinlogSource * @constructor - * @param {binlogdata.IKeyspaceShard=} [properties] Properties to set + * @param {binlogdata.IBinlogSource=} [properties] Properties to set */ - function KeyspaceShard(properties) { + function BinlogSource(properties) { + this.tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -68122,80 +68766,181 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * KeyspaceShard keyspace. + * BinlogSource keyspace. * @member {string} keyspace - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @instance */ - KeyspaceShard.prototype.keyspace = ""; + BinlogSource.prototype.keyspace = ""; /** - * KeyspaceShard shard. + * BinlogSource shard. * @member {string} shard - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @instance */ - KeyspaceShard.prototype.shard = ""; + BinlogSource.prototype.shard = ""; /** - * Creates a new KeyspaceShard instance using the specified properties. - * @function create - * @memberof binlogdata.KeyspaceShard - * @static - * @param {binlogdata.IKeyspaceShard=} [properties] Properties to set - * @returns {binlogdata.KeyspaceShard} KeyspaceShard instance + * BinlogSource tablet_type. + * @member {topodata.TabletType} tablet_type + * @memberof binlogdata.BinlogSource + * @instance */ - KeyspaceShard.create = function create(properties) { - return new KeyspaceShard(properties); + BinlogSource.prototype.tablet_type = 0; + + /** + * BinlogSource key_range. + * @member {topodata.IKeyRange|null|undefined} key_range + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.key_range = null; + + /** + * BinlogSource tables. + * @member {Array.} tables + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.tables = $util.emptyArray; + + /** + * BinlogSource filter. + * @member {binlogdata.IFilter|null|undefined} filter + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.filter = null; + + /** + * BinlogSource on_ddl. + * @member {binlogdata.OnDDLAction} on_ddl + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.on_ddl = 0; + + /** + * BinlogSource external_mysql. + * @member {string} external_mysql + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.external_mysql = ""; + + /** + * BinlogSource stop_after_copy. + * @member {boolean} stop_after_copy + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.stop_after_copy = false; + + /** + * BinlogSource external_cluster. + * @member {string} external_cluster + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.external_cluster = ""; + + /** + * BinlogSource source_time_zone. + * @member {string} source_time_zone + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.source_time_zone = ""; + + /** + * BinlogSource target_time_zone. + * @member {string} target_time_zone + * @memberof binlogdata.BinlogSource + * @instance + */ + BinlogSource.prototype.target_time_zone = ""; + + /** + * Creates a new BinlogSource instance using the specified properties. + * @function create + * @memberof binlogdata.BinlogSource + * @static + * @param {binlogdata.IBinlogSource=} [properties] Properties to set + * @returns {binlogdata.BinlogSource} BinlogSource instance + */ + BinlogSource.create = function create(properties) { + return new BinlogSource(properties); }; /** - * Encodes the specified KeyspaceShard message. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. + * Encodes the specified BinlogSource message. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. * @function encode - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static - * @param {binlogdata.IKeyspaceShard} message KeyspaceShard message or plain object to encode + * @param {binlogdata.IBinlogSource} message BinlogSource message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - KeyspaceShard.encode = function encode(message, writer) { + BinlogSource.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); + if (message.key_range != null && Object.hasOwnProperty.call(message, "key_range")) + $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.tables[i]); + if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) + $root.binlogdata.Filter.encode(message.filter, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) + writer.uint32(/* id 7, wireType 0 =*/56).int32(message.on_ddl); + if (message.external_mysql != null && Object.hasOwnProperty.call(message, "external_mysql")) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.external_mysql); + if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) + writer.uint32(/* id 9, wireType 0 =*/72).bool(message.stop_after_copy); + if (message.external_cluster != null && Object.hasOwnProperty.call(message, "external_cluster")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.external_cluster); + if (message.source_time_zone != null && Object.hasOwnProperty.call(message, "source_time_zone")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.source_time_zone); + if (message.target_time_zone != null && Object.hasOwnProperty.call(message, "target_time_zone")) + writer.uint32(/* id 12, wireType 2 =*/98).string(message.target_time_zone); return writer; }; /** - * Encodes the specified KeyspaceShard message, length delimited. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. + * Encodes the specified BinlogSource message, length delimited. Does not implicitly {@link binlogdata.BinlogSource.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static - * @param {binlogdata.IKeyspaceShard} message KeyspaceShard message or plain object to encode + * @param {binlogdata.IBinlogSource} message BinlogSource message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - KeyspaceShard.encodeDelimited = function encodeDelimited(message, writer) { + BinlogSource.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a KeyspaceShard message from the specified reader or buffer. + * Decodes a BinlogSource message from the specified reader or buffer. * @function decode - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.KeyspaceShard} KeyspaceShard + * @returns {binlogdata.BinlogSource} BinlogSource * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - KeyspaceShard.decode = function decode(reader, length) { + BinlogSource.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.KeyspaceShard(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.BinlogSource(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -68207,6 +68952,48 @@ export const binlogdata = $root.binlogdata = (() => { message.shard = reader.string(); break; } + case 3: { + message.tablet_type = reader.int32(); + break; + } + case 4: { + message.key_range = $root.topodata.KeyRange.decode(reader, reader.uint32()); + break; + } + case 5: { + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); + break; + } + case 6: { + message.filter = $root.binlogdata.Filter.decode(reader, reader.uint32()); + break; + } + case 7: { + message.on_ddl = reader.int32(); + break; + } + case 8: { + message.external_mysql = reader.string(); + break; + } + case 9: { + message.stop_after_copy = reader.bool(); + break; + } + case 10: { + message.external_cluster = reader.string(); + break; + } + case 11: { + message.source_time_zone = reader.string(); + break; + } + case 12: { + message.target_time_zone = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -68216,30 +69003,30 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a KeyspaceShard message from the specified reader or buffer, length delimited. + * Decodes a BinlogSource message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.KeyspaceShard} KeyspaceShard + * @returns {binlogdata.BinlogSource} BinlogSource * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - KeyspaceShard.decodeDelimited = function decodeDelimited(reader) { + BinlogSource.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a KeyspaceShard message. + * Verifies a BinlogSource message. * @function verify - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - KeyspaceShard.verify = function verify(message) { + BinlogSource.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -68248,123 +69035,349 @@ export const binlogdata = $root.binlogdata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + switch (message.tablet_type) { + default: + return "tablet_type: enum value expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + if (message.key_range != null && message.hasOwnProperty("key_range")) { + let error = $root.topodata.KeyRange.verify(message.key_range); + if (error) + return "key_range." + error; + } + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; + } + if (message.filter != null && message.hasOwnProperty("filter")) { + let error = $root.binlogdata.Filter.verify(message.filter); + if (error) + return "filter." + error; + } + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + switch (message.on_ddl) { + default: + return "on_ddl: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.external_mysql != null && message.hasOwnProperty("external_mysql")) + if (!$util.isString(message.external_mysql)) + return "external_mysql: string expected"; + 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.external_cluster != null && message.hasOwnProperty("external_cluster")) + if (!$util.isString(message.external_cluster)) + return "external_cluster: string expected"; + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + if (!$util.isString(message.source_time_zone)) + return "source_time_zone: string expected"; + if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) + if (!$util.isString(message.target_time_zone)) + return "target_time_zone: string expected"; return null; }; /** - * Creates a KeyspaceShard message from a plain object. Also converts values to their respective internal types. + * Creates a BinlogSource message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static * @param {Object.} object Plain object - * @returns {binlogdata.KeyspaceShard} KeyspaceShard + * @returns {binlogdata.BinlogSource} BinlogSource */ - KeyspaceShard.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.KeyspaceShard) + BinlogSource.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.BinlogSource) return object; - let message = new $root.binlogdata.KeyspaceShard(); + let message = new $root.binlogdata.BinlogSource(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); + switch (object.tablet_type) { + default: + if (typeof object.tablet_type === "number") { + message.tablet_type = object.tablet_type; + break; + } + break; + case "UNKNOWN": + case 0: + message.tablet_type = 0; + break; + case "PRIMARY": + case 1: + message.tablet_type = 1; + break; + case "MASTER": + case 1: + message.tablet_type = 1; + break; + case "REPLICA": + case 2: + message.tablet_type = 2; + break; + case "RDONLY": + case 3: + message.tablet_type = 3; + break; + case "BATCH": + case 3: + message.tablet_type = 3; + break; + case "SPARE": + case 4: + message.tablet_type = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_type = 5; + break; + case "BACKUP": + case 6: + message.tablet_type = 6; + break; + case "RESTORE": + case 7: + message.tablet_type = 7; + break; + case "DRAINED": + case 8: + message.tablet_type = 8; + break; + } + if (object.key_range != null) { + if (typeof object.key_range !== "object") + throw TypeError(".binlogdata.BinlogSource.key_range: object expected"); + message.key_range = $root.topodata.KeyRange.fromObject(object.key_range); + } + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".binlogdata.BinlogSource.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); + } + if (object.filter != null) { + if (typeof object.filter !== "object") + throw TypeError(".binlogdata.BinlogSource.filter: object expected"); + message.filter = $root.binlogdata.Filter.fromObject(object.filter); + } + switch (object.on_ddl) { + default: + if (typeof object.on_ddl === "number") { + message.on_ddl = object.on_ddl; + break; + } + break; + case "IGNORE": + case 0: + message.on_ddl = 0; + break; + case "STOP": + case 1: + message.on_ddl = 1; + break; + case "EXEC": + case 2: + message.on_ddl = 2; + break; + case "EXEC_IGNORE": + case 3: + message.on_ddl = 3; + break; + } + if (object.external_mysql != null) + message.external_mysql = String(object.external_mysql); + if (object.stop_after_copy != null) + message.stop_after_copy = Boolean(object.stop_after_copy); + if (object.external_cluster != null) + message.external_cluster = String(object.external_cluster); + if (object.source_time_zone != null) + message.source_time_zone = String(object.source_time_zone); + if (object.target_time_zone != null) + message.target_time_zone = String(object.target_time_zone); return message; }; /** - * Creates a plain object from a KeyspaceShard message. Also converts values to other types if specified. + * Creates a plain object from a BinlogSource message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static - * @param {binlogdata.KeyspaceShard} message KeyspaceShard + * @param {binlogdata.BinlogSource} message BinlogSource * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - KeyspaceShard.toObject = function toObject(message, options) { + BinlogSource.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.tables = []; if (options.defaults) { object.keyspace = ""; object.shard = ""; + object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + object.key_range = null; + object.filter = null; + object.on_ddl = options.enums === String ? "IGNORE" : 0; + object.external_mysql = ""; + object.stop_after_copy = false; + object.external_cluster = ""; + object.source_time_zone = ""; + object.target_time_zone = ""; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; + if (message.key_range != null && message.hasOwnProperty("key_range")) + object.key_range = $root.topodata.KeyRange.toObject(message.key_range, options); + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } + if (message.filter != null && message.hasOwnProperty("filter")) + object.filter = $root.binlogdata.Filter.toObject(message.filter, options); + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; + if (message.external_mysql != null && message.hasOwnProperty("external_mysql")) + object.external_mysql = message.external_mysql; + if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) + object.stop_after_copy = message.stop_after_copy; + if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) + object.external_cluster = message.external_cluster; + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + object.source_time_zone = message.source_time_zone; + if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) + object.target_time_zone = message.target_time_zone; return object; }; /** - * Converts this KeyspaceShard to JSON. + * Converts this BinlogSource to JSON. * @function toJSON - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @instance * @returns {Object.} JSON object */ - KeyspaceShard.prototype.toJSON = function toJSON() { + BinlogSource.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for KeyspaceShard + * Gets the default type url for BinlogSource * @function getTypeUrl - * @memberof binlogdata.KeyspaceShard + * @memberof binlogdata.BinlogSource * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - KeyspaceShard.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BinlogSource.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.KeyspaceShard"; + return typeUrlPrefix + "/binlogdata.BinlogSource"; }; - return KeyspaceShard; + return BinlogSource; })(); /** - * MigrationType enum. - * @name binlogdata.MigrationType + * VEventType enum. + * @name binlogdata.VEventType * @enum {number} - * @property {number} TABLES=0 TABLES value - * @property {number} SHARDS=1 SHARDS value + * @property {number} UNKNOWN=0 UNKNOWN value + * @property {number} GTID=1 GTID value + * @property {number} BEGIN=2 BEGIN value + * @property {number} COMMIT=3 COMMIT value + * @property {number} ROLLBACK=4 ROLLBACK value + * @property {number} DDL=5 DDL value + * @property {number} INSERT=6 INSERT value + * @property {number} REPLACE=7 REPLACE value + * @property {number} UPDATE=8 UPDATE value + * @property {number} DELETE=9 DELETE value + * @property {number} SET=10 SET value + * @property {number} OTHER=11 OTHER value + * @property {number} ROW=12 ROW value + * @property {number} FIELD=13 FIELD value + * @property {number} HEARTBEAT=14 HEARTBEAT value + * @property {number} VGTID=15 VGTID value + * @property {number} JOURNAL=16 JOURNAL value + * @property {number} VERSION=17 VERSION value + * @property {number} LASTPK=18 LASTPK value + * @property {number} SAVEPOINT=19 SAVEPOINT value + * @property {number} COPY_COMPLETED=20 COPY_COMPLETED value */ - binlogdata.MigrationType = (function() { + binlogdata.VEventType = (function() { const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "TABLES"] = 0; - values[valuesById[1] = "SHARDS"] = 1; + values[valuesById[0] = "UNKNOWN"] = 0; + values[valuesById[1] = "GTID"] = 1; + values[valuesById[2] = "BEGIN"] = 2; + values[valuesById[3] = "COMMIT"] = 3; + values[valuesById[4] = "ROLLBACK"] = 4; + values[valuesById[5] = "DDL"] = 5; + values[valuesById[6] = "INSERT"] = 6; + values[valuesById[7] = "REPLACE"] = 7; + values[valuesById[8] = "UPDATE"] = 8; + values[valuesById[9] = "DELETE"] = 9; + values[valuesById[10] = "SET"] = 10; + values[valuesById[11] = "OTHER"] = 11; + values[valuesById[12] = "ROW"] = 12; + values[valuesById[13] = "FIELD"] = 13; + values[valuesById[14] = "HEARTBEAT"] = 14; + values[valuesById[15] = "VGTID"] = 15; + values[valuesById[16] = "JOURNAL"] = 16; + values[valuesById[17] = "VERSION"] = 17; + values[valuesById[18] = "LASTPK"] = 18; + values[valuesById[19] = "SAVEPOINT"] = 19; + values[valuesById[20] = "COPY_COMPLETED"] = 20; return values; })(); - binlogdata.Journal = (function() { + binlogdata.RowChange = (function() { /** - * Properties of a Journal. + * Properties of a RowChange. * @memberof binlogdata - * @interface IJournal - * @property {number|Long|null} [id] Journal id - * @property {binlogdata.MigrationType|null} [migration_type] Journal migration_type - * @property {Array.|null} [tables] Journal tables - * @property {string|null} [local_position] Journal local_position - * @property {Array.|null} [shard_gtids] Journal shard_gtids - * @property {Array.|null} [participants] Journal participants - * @property {Array.|null} [source_workflows] Journal source_workflows + * @interface IRowChange + * @property {query.IRow|null} [before] RowChange before + * @property {query.IRow|null} [after] RowChange after + * @property {binlogdata.RowChange.IBitmap|null} [data_columns] RowChange data_columns */ /** - * Constructs a new Journal. + * Constructs a new RowChange. * @memberof binlogdata - * @classdesc Represents a Journal. - * @implements IJournal + * @classdesc Represents a RowChange. + * @implements IRowChange * @constructor - * @param {binlogdata.IJournal=} [properties] Properties to set + * @param {binlogdata.IRowChange=} [properties] Properties to set */ - function Journal(properties) { - this.tables = []; - this.shard_gtids = []; - this.participants = []; - this.source_workflows = []; + function RowChange(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -68372,171 +69385,103 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * Journal id. - * @member {number|Long} id - * @memberof binlogdata.Journal + * RowChange before. + * @member {query.IRow|null|undefined} before + * @memberof binlogdata.RowChange * @instance */ - Journal.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + RowChange.prototype.before = null; /** - * Journal migration_type. - * @member {binlogdata.MigrationType} migration_type - * @memberof binlogdata.Journal - * @instance - */ - Journal.prototype.migration_type = 0; - - /** - * Journal tables. - * @member {Array.} tables - * @memberof binlogdata.Journal - * @instance - */ - Journal.prototype.tables = $util.emptyArray; - - /** - * Journal local_position. - * @member {string} local_position - * @memberof binlogdata.Journal - * @instance - */ - Journal.prototype.local_position = ""; - - /** - * Journal shard_gtids. - * @member {Array.} shard_gtids - * @memberof binlogdata.Journal - * @instance - */ - Journal.prototype.shard_gtids = $util.emptyArray; - - /** - * Journal participants. - * @member {Array.} participants - * @memberof binlogdata.Journal + * RowChange after. + * @member {query.IRow|null|undefined} after + * @memberof binlogdata.RowChange * @instance */ - Journal.prototype.participants = $util.emptyArray; + RowChange.prototype.after = null; /** - * Journal source_workflows. - * @member {Array.} source_workflows - * @memberof binlogdata.Journal + * RowChange data_columns. + * @member {binlogdata.RowChange.IBitmap|null|undefined} data_columns + * @memberof binlogdata.RowChange * @instance */ - Journal.prototype.source_workflows = $util.emptyArray; + RowChange.prototype.data_columns = null; /** - * Creates a new Journal instance using the specified properties. + * Creates a new RowChange instance using the specified properties. * @function create - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static - * @param {binlogdata.IJournal=} [properties] Properties to set - * @returns {binlogdata.Journal} Journal instance + * @param {binlogdata.IRowChange=} [properties] Properties to set + * @returns {binlogdata.RowChange} RowChange instance */ - Journal.create = function create(properties) { - return new Journal(properties); + RowChange.create = function create(properties) { + return new RowChange(properties); }; /** - * Encodes the specified Journal message. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. + * Encodes the specified RowChange message. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. * @function encode - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static - * @param {binlogdata.IJournal} message Journal message or plain object to encode + * @param {binlogdata.IRowChange} message RowChange message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Journal.encode = function encode(message, writer) { + RowChange.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); - if (message.migration_type != null && Object.hasOwnProperty.call(message, "migration_type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.migration_type); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.tables[i]); - if (message.local_position != null && Object.hasOwnProperty.call(message, "local_position")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.local_position); - if (message.shard_gtids != null && message.shard_gtids.length) - for (let i = 0; i < message.shard_gtids.length; ++i) - $root.binlogdata.ShardGtid.encode(message.shard_gtids[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.participants != null && message.participants.length) - for (let i = 0; i < message.participants.length; ++i) - $root.binlogdata.KeyspaceShard.encode(message.participants[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.source_workflows != null && message.source_workflows.length) - for (let i = 0; i < message.source_workflows.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.source_workflows[i]); + if (message.before != null && Object.hasOwnProperty.call(message, "before")) + $root.query.Row.encode(message.before, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.after != null && Object.hasOwnProperty.call(message, "after")) + $root.query.Row.encode(message.after, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.data_columns != null && Object.hasOwnProperty.call(message, "data_columns")) + $root.binlogdata.RowChange.Bitmap.encode(message.data_columns, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified Journal message, length delimited. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. + * Encodes the specified RowChange message, length delimited. Does not implicitly {@link binlogdata.RowChange.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static - * @param {binlogdata.IJournal} message Journal message or plain object to encode + * @param {binlogdata.IRowChange} message RowChange message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Journal.encodeDelimited = function encodeDelimited(message, writer) { + RowChange.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Journal message from the specified reader or buffer. + * Decodes a RowChange message from the specified reader or buffer. * @function decode - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.Journal} Journal + * @returns {binlogdata.RowChange} RowChange * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Journal.decode = function decode(reader, length) { + RowChange.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Journal(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowChange(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.id = reader.int64(); + message.before = $root.query.Row.decode(reader, reader.uint32()); break; } case 2: { - message.migration_type = reader.int32(); + message.after = $root.query.Row.decode(reader, reader.uint32()); break; } case 3: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); - break; - } - case 4: { - message.local_position = reader.string(); - break; - } - case 5: { - if (!(message.shard_gtids && message.shard_gtids.length)) - message.shard_gtids = []; - message.shard_gtids.push($root.binlogdata.ShardGtid.decode(reader, reader.uint32())); - break; - } - case 6: { - if (!(message.participants && message.participants.length)) - message.participants = []; - message.participants.push($root.binlogdata.KeyspaceShard.decode(reader, reader.uint32())); - break; - } - case 7: { - if (!(message.source_workflows && message.source_workflows.length)) - message.source_workflows = []; - message.source_workflows.push(reader.string()); + message.data_columns = $root.binlogdata.RowChange.Bitmap.decode(reader, reader.uint32()); break; } default: @@ -68548,537 +69493,531 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a Journal message from the specified reader or buffer, length delimited. + * Decodes a RowChange message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.Journal} Journal + * @returns {binlogdata.RowChange} RowChange * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Journal.decodeDelimited = function decodeDelimited(reader) { + RowChange.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Journal message. + * Verifies a RowChange message. * @function verify - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Journal.verify = function verify(message) { + RowChange.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) - return "id: integer|Long expected"; - if (message.migration_type != null && message.hasOwnProperty("migration_type")) - switch (message.migration_type) { - default: - return "migration_type: enum value expected"; - case 0: - case 1: - break; - } - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; - } - if (message.local_position != null && message.hasOwnProperty("local_position")) - if (!$util.isString(message.local_position)) - return "local_position: string expected"; - if (message.shard_gtids != null && message.hasOwnProperty("shard_gtids")) { - if (!Array.isArray(message.shard_gtids)) - return "shard_gtids: array expected"; - for (let i = 0; i < message.shard_gtids.length; ++i) { - let error = $root.binlogdata.ShardGtid.verify(message.shard_gtids[i]); - if (error) - return "shard_gtids." + error; - } + if (message.before != null && message.hasOwnProperty("before")) { + let error = $root.query.Row.verify(message.before); + if (error) + return "before." + error; } - if (message.participants != null && message.hasOwnProperty("participants")) { - if (!Array.isArray(message.participants)) - return "participants: array expected"; - for (let i = 0; i < message.participants.length; ++i) { - let error = $root.binlogdata.KeyspaceShard.verify(message.participants[i]); - if (error) - return "participants." + error; - } + if (message.after != null && message.hasOwnProperty("after")) { + let error = $root.query.Row.verify(message.after); + if (error) + return "after." + error; } - if (message.source_workflows != null && message.hasOwnProperty("source_workflows")) { - if (!Array.isArray(message.source_workflows)) - return "source_workflows: array expected"; - for (let i = 0; i < message.source_workflows.length; ++i) - if (!$util.isString(message.source_workflows[i])) - return "source_workflows: string[] expected"; + if (message.data_columns != null && message.hasOwnProperty("data_columns")) { + let error = $root.binlogdata.RowChange.Bitmap.verify(message.data_columns); + if (error) + return "data_columns." + error; } return null; }; /** - * Creates a Journal message from a plain object. Also converts values to their respective internal types. + * Creates a RowChange message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static * @param {Object.} object Plain object - * @returns {binlogdata.Journal} Journal + * @returns {binlogdata.RowChange} RowChange */ - Journal.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.Journal) + RowChange.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.RowChange) return object; - let message = new $root.binlogdata.Journal(); - if (object.id != null) - if ($util.Long) - (message.id = $util.Long.fromValue(object.id)).unsigned = false; - else if (typeof object.id === "string") - message.id = parseInt(object.id, 10); - else if (typeof object.id === "number") - message.id = object.id; - else if (typeof object.id === "object") - message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); - switch (object.migration_type) { - default: - if (typeof object.migration_type === "number") { - message.migration_type = object.migration_type; - break; - } - break; - case "TABLES": - case 0: - message.migration_type = 0; - break; - case "SHARDS": - case 1: - message.migration_type = 1; - break; - } - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".binlogdata.Journal.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); - } - if (object.local_position != null) - message.local_position = String(object.local_position); - if (object.shard_gtids) { - if (!Array.isArray(object.shard_gtids)) - throw TypeError(".binlogdata.Journal.shard_gtids: array expected"); - message.shard_gtids = []; - for (let i = 0; i < object.shard_gtids.length; ++i) { - if (typeof object.shard_gtids[i] !== "object") - throw TypeError(".binlogdata.Journal.shard_gtids: object expected"); - message.shard_gtids[i] = $root.binlogdata.ShardGtid.fromObject(object.shard_gtids[i]); - } + let message = new $root.binlogdata.RowChange(); + if (object.before != null) { + if (typeof object.before !== "object") + throw TypeError(".binlogdata.RowChange.before: object expected"); + message.before = $root.query.Row.fromObject(object.before); } - if (object.participants) { - if (!Array.isArray(object.participants)) - throw TypeError(".binlogdata.Journal.participants: array expected"); - message.participants = []; - for (let i = 0; i < object.participants.length; ++i) { - if (typeof object.participants[i] !== "object") - throw TypeError(".binlogdata.Journal.participants: object expected"); - message.participants[i] = $root.binlogdata.KeyspaceShard.fromObject(object.participants[i]); - } + if (object.after != null) { + if (typeof object.after !== "object") + throw TypeError(".binlogdata.RowChange.after: object expected"); + message.after = $root.query.Row.fromObject(object.after); } - if (object.source_workflows) { - if (!Array.isArray(object.source_workflows)) - throw TypeError(".binlogdata.Journal.source_workflows: array expected"); - message.source_workflows = []; - for (let i = 0; i < object.source_workflows.length; ++i) - message.source_workflows[i] = String(object.source_workflows[i]); + if (object.data_columns != null) { + if (typeof object.data_columns !== "object") + throw TypeError(".binlogdata.RowChange.data_columns: object expected"); + message.data_columns = $root.binlogdata.RowChange.Bitmap.fromObject(object.data_columns); } return message; }; /** - * Creates a plain object from a Journal message. Also converts values to other types if specified. + * Creates a plain object from a RowChange message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static - * @param {binlogdata.Journal} message Journal + * @param {binlogdata.RowChange} message RowChange * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Journal.toObject = function toObject(message, options) { + RowChange.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.tables = []; - object.shard_gtids = []; - object.participants = []; - object.source_workflows = []; - } if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.id = options.longs === String ? "0" : 0; - object.migration_type = options.enums === String ? "TABLES" : 0; - object.local_position = ""; - } - if (message.id != null && message.hasOwnProperty("id")) - if (typeof message.id === "number") - object.id = options.longs === String ? String(message.id) : message.id; - else - object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; - if (message.migration_type != null && message.hasOwnProperty("migration_type")) - object.migration_type = options.enums === String ? $root.binlogdata.MigrationType[message.migration_type] === undefined ? message.migration_type : $root.binlogdata.MigrationType[message.migration_type] : message.migration_type; - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; - } - if (message.local_position != null && message.hasOwnProperty("local_position")) - object.local_position = message.local_position; - if (message.shard_gtids && message.shard_gtids.length) { - object.shard_gtids = []; - for (let j = 0; j < message.shard_gtids.length; ++j) - object.shard_gtids[j] = $root.binlogdata.ShardGtid.toObject(message.shard_gtids[j], options); - } - if (message.participants && message.participants.length) { - object.participants = []; - for (let j = 0; j < message.participants.length; ++j) - object.participants[j] = $root.binlogdata.KeyspaceShard.toObject(message.participants[j], options); - } - if (message.source_workflows && message.source_workflows.length) { - object.source_workflows = []; - for (let j = 0; j < message.source_workflows.length; ++j) - object.source_workflows[j] = message.source_workflows[j]; + object.before = null; + object.after = null; + object.data_columns = null; } + if (message.before != null && message.hasOwnProperty("before")) + object.before = $root.query.Row.toObject(message.before, options); + if (message.after != null && message.hasOwnProperty("after")) + object.after = $root.query.Row.toObject(message.after, options); + if (message.data_columns != null && message.hasOwnProperty("data_columns")) + object.data_columns = $root.binlogdata.RowChange.Bitmap.toObject(message.data_columns, options); return object; }; /** - * Converts this Journal to JSON. + * Converts this RowChange to JSON. * @function toJSON - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @instance * @returns {Object.} JSON object */ - Journal.prototype.toJSON = function toJSON() { + RowChange.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Journal + * Gets the default type url for RowChange * @function getTypeUrl - * @memberof binlogdata.Journal + * @memberof binlogdata.RowChange * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Journal.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RowChange.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.Journal"; + return typeUrlPrefix + "/binlogdata.RowChange"; }; - return Journal; - })(); - - binlogdata.VEvent = (function() { + RowChange.Bitmap = (function() { - /** - * Properties of a VEvent. - * @memberof binlogdata - * @interface IVEvent - * @property {binlogdata.VEventType|null} [type] VEvent type - * @property {number|Long|null} [timestamp] VEvent timestamp - * @property {string|null} [gtid] VEvent gtid - * @property {string|null} [statement] VEvent statement - * @property {binlogdata.IRowEvent|null} [row_event] VEvent row_event - * @property {binlogdata.IFieldEvent|null} [field_event] VEvent field_event - * @property {binlogdata.IVGtid|null} [vgtid] VEvent vgtid - * @property {binlogdata.IJournal|null} [journal] VEvent journal - * @property {string|null} [dml] VEvent dml - * @property {number|Long|null} [current_time] VEvent current_time - * @property {binlogdata.ILastPKEvent|null} [last_p_k_event] VEvent last_p_k_event - * @property {string|null} [keyspace] VEvent keyspace - * @property {string|null} [shard] VEvent shard - * @property {boolean|null} [throttled] VEvent throttled - */ + /** + * Properties of a Bitmap. + * @memberof binlogdata.RowChange + * @interface IBitmap + * @property {number|Long|null} [count] Bitmap count + * @property {Uint8Array|null} [cols] Bitmap cols + */ - /** - * Constructs a new VEvent. - * @memberof binlogdata - * @classdesc Represents a VEvent. - * @implements IVEvent - * @constructor - * @param {binlogdata.IVEvent=} [properties] Properties to set - */ - function VEvent(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]]; - } + /** + * Constructs a new Bitmap. + * @memberof binlogdata.RowChange + * @classdesc Represents a Bitmap. + * @implements IBitmap + * @constructor + * @param {binlogdata.RowChange.IBitmap=} [properties] Properties to set + */ + function Bitmap(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]]; + } - /** - * VEvent type. - * @member {binlogdata.VEventType} type - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.type = 0; + /** + * Bitmap count. + * @member {number|Long} count + * @memberof binlogdata.RowChange.Bitmap + * @instance + */ + Bitmap.prototype.count = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * VEvent timestamp. - * @member {number|Long} timestamp - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * Bitmap cols. + * @member {Uint8Array} cols + * @memberof binlogdata.RowChange.Bitmap + * @instance + */ + Bitmap.prototype.cols = $util.newBuffer([]); - /** - * VEvent gtid. - * @member {string} gtid - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.gtid = ""; + /** + * Creates a new Bitmap instance using the specified properties. + * @function create + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {binlogdata.RowChange.IBitmap=} [properties] Properties to set + * @returns {binlogdata.RowChange.Bitmap} Bitmap instance + */ + Bitmap.create = function create(properties) { + return new Bitmap(properties); + }; - /** - * VEvent statement. - * @member {string} statement - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.statement = ""; + /** + * Encodes the specified Bitmap message. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. + * @function encode + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {binlogdata.RowChange.IBitmap} message Bitmap message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Bitmap.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.count != null && Object.hasOwnProperty.call(message, "count")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.count); + if (message.cols != null && Object.hasOwnProperty.call(message, "cols")) + writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.cols); + return writer; + }; - /** - * VEvent row_event. - * @member {binlogdata.IRowEvent|null|undefined} row_event - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.row_event = null; + /** + * Encodes the specified Bitmap message, length delimited. Does not implicitly {@link binlogdata.RowChange.Bitmap.verify|verify} messages. + * @function encodeDelimited + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {binlogdata.RowChange.IBitmap} message Bitmap message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Bitmap.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * VEvent field_event. - * @member {binlogdata.IFieldEvent|null|undefined} field_event - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.field_event = null; + /** + * Decodes a Bitmap message from the specified reader or buffer. + * @function decode + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {binlogdata.RowChange.Bitmap} Bitmap + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Bitmap.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowChange.Bitmap(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.count = reader.int64(); + break; + } + case 2: { + message.cols = reader.bytes(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; - /** - * VEvent vgtid. - * @member {binlogdata.IVGtid|null|undefined} vgtid - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.vgtid = null; + /** + * Decodes a Bitmap message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {binlogdata.RowChange.Bitmap} Bitmap + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Bitmap.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Bitmap message. + * @function verify + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Bitmap.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.count != null && message.hasOwnProperty("count")) + if (!$util.isInteger(message.count) && !(message.count && $util.isInteger(message.count.low) && $util.isInteger(message.count.high))) + return "count: integer|Long expected"; + if (message.cols != null && message.hasOwnProperty("cols")) + if (!(message.cols && typeof message.cols.length === "number" || $util.isString(message.cols))) + return "cols: buffer expected"; + return null; + }; + + /** + * Creates a Bitmap message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {Object.} object Plain object + * @returns {binlogdata.RowChange.Bitmap} Bitmap + */ + Bitmap.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.RowChange.Bitmap) + return object; + let message = new $root.binlogdata.RowChange.Bitmap(); + if (object.count != null) + if ($util.Long) + (message.count = $util.Long.fromValue(object.count)).unsigned = false; + else if (typeof object.count === "string") + message.count = parseInt(object.count, 10); + else if (typeof object.count === "number") + message.count = object.count; + else if (typeof object.count === "object") + message.count = new $util.LongBits(object.count.low >>> 0, object.count.high >>> 0).toNumber(); + if (object.cols != null) + if (typeof object.cols === "string") + $util.base64.decode(object.cols, message.cols = $util.newBuffer($util.base64.length(object.cols)), 0); + else if (object.cols.length >= 0) + message.cols = object.cols; + return message; + }; + + /** + * Creates a plain object from a Bitmap message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {binlogdata.RowChange.Bitmap} message Bitmap + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Bitmap.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.count = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.count = options.longs === String ? "0" : 0; + if (options.bytes === String) + object.cols = ""; + else { + object.cols = []; + if (options.bytes !== Array) + object.cols = $util.newBuffer(object.cols); + } + } + if (message.count != null && message.hasOwnProperty("count")) + if (typeof message.count === "number") + object.count = options.longs === String ? String(message.count) : message.count; + else + object.count = options.longs === String ? $util.Long.prototype.toString.call(message.count) : options.longs === Number ? new $util.LongBits(message.count.low >>> 0, message.count.high >>> 0).toNumber() : message.count; + if (message.cols != null && message.hasOwnProperty("cols")) + object.cols = options.bytes === String ? $util.base64.encode(message.cols, 0, message.cols.length) : options.bytes === Array ? Array.prototype.slice.call(message.cols) : message.cols; + return object; + }; + + /** + * Converts this Bitmap to JSON. + * @function toJSON + * @memberof binlogdata.RowChange.Bitmap + * @instance + * @returns {Object.} JSON object + */ + Bitmap.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Bitmap + * @function getTypeUrl + * @memberof binlogdata.RowChange.Bitmap + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Bitmap.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/binlogdata.RowChange.Bitmap"; + }; + + return Bitmap; + })(); + + return RowChange; + })(); + + binlogdata.RowEvent = (function() { /** - * VEvent journal. - * @member {binlogdata.IJournal|null|undefined} journal - * @memberof binlogdata.VEvent - * @instance + * Properties of a RowEvent. + * @memberof binlogdata + * @interface IRowEvent + * @property {string|null} [table_name] RowEvent table_name + * @property {Array.|null} [row_changes] RowEvent row_changes + * @property {string|null} [keyspace] RowEvent keyspace + * @property {string|null} [shard] RowEvent shard */ - VEvent.prototype.journal = null; /** - * VEvent dml. - * @member {string} dml - * @memberof binlogdata.VEvent - * @instance + * Constructs a new RowEvent. + * @memberof binlogdata + * @classdesc Represents a RowEvent. + * @implements IRowEvent + * @constructor + * @param {binlogdata.IRowEvent=} [properties] Properties to set */ - VEvent.prototype.dml = ""; + function RowEvent(properties) { + this.row_changes = []; + 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]]; + } /** - * VEvent current_time. - * @member {number|Long} current_time - * @memberof binlogdata.VEvent + * RowEvent table_name. + * @member {string} table_name + * @memberof binlogdata.RowEvent * @instance */ - VEvent.prototype.current_time = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + RowEvent.prototype.table_name = ""; /** - * VEvent last_p_k_event. - * @member {binlogdata.ILastPKEvent|null|undefined} last_p_k_event - * @memberof binlogdata.VEvent + * RowEvent row_changes. + * @member {Array.} row_changes + * @memberof binlogdata.RowEvent * @instance */ - VEvent.prototype.last_p_k_event = null; + RowEvent.prototype.row_changes = $util.emptyArray; /** - * VEvent keyspace. + * RowEvent keyspace. * @member {string} keyspace - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @instance */ - VEvent.prototype.keyspace = ""; + RowEvent.prototype.keyspace = ""; /** - * VEvent shard. + * RowEvent shard. * @member {string} shard - * @memberof binlogdata.VEvent - * @instance - */ - VEvent.prototype.shard = ""; - - /** - * VEvent throttled. - * @member {boolean} throttled - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @instance */ - VEvent.prototype.throttled = false; + RowEvent.prototype.shard = ""; /** - * Creates a new VEvent instance using the specified properties. + * Creates a new RowEvent instance using the specified properties. * @function create - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static - * @param {binlogdata.IVEvent=} [properties] Properties to set - * @returns {binlogdata.VEvent} VEvent instance + * @param {binlogdata.IRowEvent=} [properties] Properties to set + * @returns {binlogdata.RowEvent} RowEvent instance */ - VEvent.create = function create(properties) { - return new VEvent(properties); + RowEvent.create = function create(properties) { + return new RowEvent(properties); }; /** - * Encodes the specified VEvent message. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. + * Encodes the specified RowEvent message. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. * @function encode - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static - * @param {binlogdata.IVEvent} message VEvent message or plain object to encode + * @param {binlogdata.IRowEvent} message RowEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VEvent.encode = function encode(message, writer) { + RowEvent.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); - if (message.timestamp != null && Object.hasOwnProperty.call(message, "timestamp")) - writer.uint32(/* id 2, wireType 0 =*/16).int64(message.timestamp); - if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); - if (message.statement != null && Object.hasOwnProperty.call(message, "statement")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.statement); - if (message.row_event != null && Object.hasOwnProperty.call(message, "row_event")) - $root.binlogdata.RowEvent.encode(message.row_event, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.field_event != null && Object.hasOwnProperty.call(message, "field_event")) - $root.binlogdata.FieldEvent.encode(message.field_event, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.vgtid != null && Object.hasOwnProperty.call(message, "vgtid")) - $root.binlogdata.VGtid.encode(message.vgtid, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); - if (message.journal != null && Object.hasOwnProperty.call(message, "journal")) - $root.binlogdata.Journal.encode(message.journal, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); - if (message.dml != null && Object.hasOwnProperty.call(message, "dml")) - writer.uint32(/* id 9, wireType 2 =*/74).string(message.dml); - if (message.current_time != null && Object.hasOwnProperty.call(message, "current_time")) - writer.uint32(/* id 20, wireType 0 =*/160).int64(message.current_time); - if (message.last_p_k_event != null && Object.hasOwnProperty.call(message, "last_p_k_event")) - $root.binlogdata.LastPKEvent.encode(message.last_p_k_event, writer.uint32(/* id 21, wireType 2 =*/170).fork()).ldelim(); + if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); + if (message.row_changes != null && message.row_changes.length) + for (let i = 0; i < message.row_changes.length; ++i) + $root.binlogdata.RowChange.encode(message.row_changes[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 22, wireType 2 =*/178).string(message.keyspace); + writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 23, wireType 2 =*/186).string(message.shard); - if (message.throttled != null && Object.hasOwnProperty.call(message, "throttled")) - writer.uint32(/* id 24, wireType 0 =*/192).bool(message.throttled); + writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard); return writer; }; /** - * Encodes the specified VEvent message, length delimited. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. + * Encodes the specified RowEvent message, length delimited. Does not implicitly {@link binlogdata.RowEvent.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static - * @param {binlogdata.IVEvent} message VEvent message or plain object to encode + * @param {binlogdata.IRowEvent} message RowEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VEvent.encodeDelimited = function encodeDelimited(message, writer) { + RowEvent.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VEvent message from the specified reader or buffer. + * Decodes a RowEvent message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VEvent} VEvent + * @returns {binlogdata.RowEvent} RowEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VEvent.decode = function decode(reader, length) { + RowEvent.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VEvent(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.RowEvent(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.type = reader.int32(); + message.table_name = reader.string(); break; } case 2: { - message.timestamp = reader.int64(); + if (!(message.row_changes && message.row_changes.length)) + message.row_changes = []; + message.row_changes.push($root.binlogdata.RowChange.decode(reader, reader.uint32())); break; } case 3: { - message.gtid = reader.string(); - break; - } - case 4: { - message.statement = reader.string(); - break; - } - case 5: { - message.row_event = $root.binlogdata.RowEvent.decode(reader, reader.uint32()); - break; - } - case 6: { - message.field_event = $root.binlogdata.FieldEvent.decode(reader, reader.uint32()); - break; - } - case 7: { - message.vgtid = $root.binlogdata.VGtid.decode(reader, reader.uint32()); - break; - } - case 8: { - message.journal = $root.binlogdata.Journal.decode(reader, reader.uint32()); - break; - } - case 9: { - message.dml = reader.string(); - break; - } - case 20: { - message.current_time = reader.int64(); - break; - } - case 21: { - message.last_p_k_event = $root.binlogdata.LastPKEvent.decode(reader, reader.uint32()); - break; - } - case 22: { message.keyspace = reader.string(); break; } - case 23: { + case 4: { message.shard = reader.string(); break; } - case 24: { - message.throttled = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -69088,98 +70027,43 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VEvent message from the specified reader or buffer, length delimited. + * Decodes a RowEvent message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VEvent} VEvent + * @returns {binlogdata.RowEvent} RowEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VEvent.decodeDelimited = function decodeDelimited(reader) { + RowEvent.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VEvent message. + * Verifies a RowEvent message. * @function verify - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VEvent.verify = function verify(message) { + RowEvent.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "type: enum value expected"; - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - case 9: - case 10: - case 11: - case 12: - case 13: - case 14: - case 15: - case 16: - case 17: - case 18: - case 19: - case 20: - break; + if (message.table_name != null && message.hasOwnProperty("table_name")) + if (!$util.isString(message.table_name)) + return "table_name: string expected"; + if (message.row_changes != null && message.hasOwnProperty("row_changes")) { + if (!Array.isArray(message.row_changes)) + return "row_changes: array expected"; + for (let i = 0; i < message.row_changes.length; ++i) { + let error = $root.binlogdata.RowChange.verify(message.row_changes[i]); + if (error) + return "row_changes." + error; } - if (message.timestamp != null && message.hasOwnProperty("timestamp")) - if (!$util.isInteger(message.timestamp) && !(message.timestamp && $util.isInteger(message.timestamp.low) && $util.isInteger(message.timestamp.high))) - return "timestamp: integer|Long expected"; - if (message.gtid != null && message.hasOwnProperty("gtid")) - if (!$util.isString(message.gtid)) - return "gtid: string expected"; - if (message.statement != null && message.hasOwnProperty("statement")) - if (!$util.isString(message.statement)) - return "statement: string expected"; - if (message.row_event != null && message.hasOwnProperty("row_event")) { - let error = $root.binlogdata.RowEvent.verify(message.row_event); - if (error) - return "row_event." + error; - } - if (message.field_event != null && message.hasOwnProperty("field_event")) { - let error = $root.binlogdata.FieldEvent.verify(message.field_event); - if (error) - return "field_event." + error; - } - if (message.vgtid != null && message.hasOwnProperty("vgtid")) { - let error = $root.binlogdata.VGtid.verify(message.vgtid); - if (error) - return "vgtid." + error; - } - if (message.journal != null && message.hasOwnProperty("journal")) { - let error = $root.binlogdata.Journal.verify(message.journal); - if (error) - return "journal." + error; - } - if (message.dml != null && message.hasOwnProperty("dml")) - if (!$util.isString(message.dml)) - return "dml: string expected"; - if (message.current_time != null && message.hasOwnProperty("current_time")) - if (!$util.isInteger(message.current_time) && !(message.current_time && $util.isInteger(message.current_time.low) && $util.isInteger(message.current_time.high))) - return "current_time: integer|Long expected"; - if (message.last_p_k_event != null && message.hasOwnProperty("last_p_k_event")) { - let error = $root.binlogdata.LastPKEvent.verify(message.last_p_k_event); - if (error) - return "last_p_k_event." + error; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) @@ -69187,299 +70071,125 @@ export const binlogdata = $root.binlogdata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.throttled != null && message.hasOwnProperty("throttled")) - if (typeof message.throttled !== "boolean") - return "throttled: boolean expected"; return null; }; /** - * Creates a VEvent message from a plain object. Also converts values to their respective internal types. + * Creates a RowEvent message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static * @param {Object.} object Plain object - * @returns {binlogdata.VEvent} VEvent + * @returns {binlogdata.RowEvent} RowEvent */ - VEvent.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VEvent) + RowEvent.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.RowEvent) return object; - let message = new $root.binlogdata.VEvent(); - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; + let message = new $root.binlogdata.RowEvent(); + if (object.table_name != null) + message.table_name = String(object.table_name); + if (object.row_changes) { + if (!Array.isArray(object.row_changes)) + throw TypeError(".binlogdata.RowEvent.row_changes: array expected"); + message.row_changes = []; + for (let i = 0; i < object.row_changes.length; ++i) { + if (typeof object.row_changes[i] !== "object") + throw TypeError(".binlogdata.RowEvent.row_changes: object expected"); + message.row_changes[i] = $root.binlogdata.RowChange.fromObject(object.row_changes[i]); } - break; - case "UNKNOWN": - case 0: - message.type = 0; - break; - case "GTID": - case 1: - message.type = 1; - break; - case "BEGIN": - case 2: - message.type = 2; - break; - case "COMMIT": - case 3: - message.type = 3; - break; - case "ROLLBACK": - case 4: - message.type = 4; - break; - case "DDL": - case 5: - message.type = 5; - break; - case "INSERT": - case 6: - message.type = 6; - break; - case "REPLACE": - case 7: - message.type = 7; - break; - case "UPDATE": - case 8: - message.type = 8; - break; - case "DELETE": - case 9: - message.type = 9; - break; - case "SET": - case 10: - message.type = 10; - break; - case "OTHER": - case 11: - message.type = 11; - break; - case "ROW": - case 12: - message.type = 12; - break; - case "FIELD": - case 13: - message.type = 13; - break; - case "HEARTBEAT": - case 14: - message.type = 14; - break; - case "VGTID": - case 15: - message.type = 15; - break; - case "JOURNAL": - case 16: - message.type = 16; - break; - case "VERSION": - case 17: - message.type = 17; - break; - case "LASTPK": - case 18: - message.type = 18; - break; - case "SAVEPOINT": - case 19: - message.type = 19; - break; - case "COPY_COMPLETED": - case 20: - message.type = 20; - break; - } - if (object.timestamp != null) - if ($util.Long) - (message.timestamp = $util.Long.fromValue(object.timestamp)).unsigned = false; - else if (typeof object.timestamp === "string") - message.timestamp = parseInt(object.timestamp, 10); - else if (typeof object.timestamp === "number") - message.timestamp = object.timestamp; - else if (typeof object.timestamp === "object") - message.timestamp = new $util.LongBits(object.timestamp.low >>> 0, object.timestamp.high >>> 0).toNumber(); - if (object.gtid != null) - message.gtid = String(object.gtid); - if (object.statement != null) - message.statement = String(object.statement); - if (object.row_event != null) { - if (typeof object.row_event !== "object") - throw TypeError(".binlogdata.VEvent.row_event: object expected"); - message.row_event = $root.binlogdata.RowEvent.fromObject(object.row_event); - } - if (object.field_event != null) { - if (typeof object.field_event !== "object") - throw TypeError(".binlogdata.VEvent.field_event: object expected"); - message.field_event = $root.binlogdata.FieldEvent.fromObject(object.field_event); - } - if (object.vgtid != null) { - if (typeof object.vgtid !== "object") - throw TypeError(".binlogdata.VEvent.vgtid: object expected"); - message.vgtid = $root.binlogdata.VGtid.fromObject(object.vgtid); - } - if (object.journal != null) { - if (typeof object.journal !== "object") - throw TypeError(".binlogdata.VEvent.journal: object expected"); - message.journal = $root.binlogdata.Journal.fromObject(object.journal); - } - if (object.dml != null) - message.dml = String(object.dml); - if (object.current_time != null) - if ($util.Long) - (message.current_time = $util.Long.fromValue(object.current_time)).unsigned = false; - else if (typeof object.current_time === "string") - message.current_time = parseInt(object.current_time, 10); - else if (typeof object.current_time === "number") - message.current_time = object.current_time; - else if (typeof object.current_time === "object") - message.current_time = new $util.LongBits(object.current_time.low >>> 0, object.current_time.high >>> 0).toNumber(); - if (object.last_p_k_event != null) { - if (typeof object.last_p_k_event !== "object") - throw TypeError(".binlogdata.VEvent.last_p_k_event: object expected"); - message.last_p_k_event = $root.binlogdata.LastPKEvent.fromObject(object.last_p_k_event); } if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - if (object.throttled != null) - message.throttled = Boolean(object.throttled); return message; }; /** - * Creates a plain object from a VEvent message. Also converts values to other types if specified. + * Creates a plain object from a RowEvent message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static - * @param {binlogdata.VEvent} message VEvent + * @param {binlogdata.RowEvent} message RowEvent * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VEvent.toObject = function toObject(message, options) { + RowEvent.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.row_changes = []; if (options.defaults) { - object.type = options.enums === String ? "UNKNOWN" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.timestamp = options.longs === String ? "0" : 0; - object.gtid = ""; - object.statement = ""; - object.row_event = null; - object.field_event = null; - object.vgtid = null; - object.journal = null; - object.dml = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.current_time = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.current_time = options.longs === String ? "0" : 0; - object.last_p_k_event = null; + object.table_name = ""; object.keyspace = ""; object.shard = ""; - object.throttled = false; } - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.binlogdata.VEventType[message.type] === undefined ? message.type : $root.binlogdata.VEventType[message.type] : message.type; - if (message.timestamp != null && message.hasOwnProperty("timestamp")) - if (typeof message.timestamp === "number") - object.timestamp = options.longs === String ? String(message.timestamp) : message.timestamp; - else - object.timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.timestamp) : options.longs === Number ? new $util.LongBits(message.timestamp.low >>> 0, message.timestamp.high >>> 0).toNumber() : message.timestamp; - if (message.gtid != null && message.hasOwnProperty("gtid")) - object.gtid = message.gtid; - if (message.statement != null && message.hasOwnProperty("statement")) - object.statement = message.statement; - if (message.row_event != null && message.hasOwnProperty("row_event")) - object.row_event = $root.binlogdata.RowEvent.toObject(message.row_event, options); - if (message.field_event != null && message.hasOwnProperty("field_event")) - object.field_event = $root.binlogdata.FieldEvent.toObject(message.field_event, options); - if (message.vgtid != null && message.hasOwnProperty("vgtid")) - object.vgtid = $root.binlogdata.VGtid.toObject(message.vgtid, options); - if (message.journal != null && message.hasOwnProperty("journal")) - object.journal = $root.binlogdata.Journal.toObject(message.journal, options); - if (message.dml != null && message.hasOwnProperty("dml")) - object.dml = message.dml; - if (message.current_time != null && message.hasOwnProperty("current_time")) - if (typeof message.current_time === "number") - object.current_time = options.longs === String ? String(message.current_time) : message.current_time; - else - object.current_time = options.longs === String ? $util.Long.prototype.toString.call(message.current_time) : options.longs === Number ? new $util.LongBits(message.current_time.low >>> 0, message.current_time.high >>> 0).toNumber() : message.current_time; - if (message.last_p_k_event != null && message.hasOwnProperty("last_p_k_event")) - object.last_p_k_event = $root.binlogdata.LastPKEvent.toObject(message.last_p_k_event, options); + if (message.table_name != null && message.hasOwnProperty("table_name")) + object.table_name = message.table_name; + if (message.row_changes && message.row_changes.length) { + object.row_changes = []; + for (let j = 0; j < message.row_changes.length; ++j) + object.row_changes[j] = $root.binlogdata.RowChange.toObject(message.row_changes[j], options); + } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.throttled != null && message.hasOwnProperty("throttled")) - object.throttled = message.throttled; return object; }; /** - * Converts this VEvent to JSON. + * Converts this RowEvent to JSON. * @function toJSON - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @instance * @returns {Object.} JSON object */ - VEvent.prototype.toJSON = function toJSON() { + RowEvent.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VEvent + * Gets the default type url for RowEvent * @function getTypeUrl - * @memberof binlogdata.VEvent + * @memberof binlogdata.RowEvent * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RowEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VEvent"; + return typeUrlPrefix + "/binlogdata.RowEvent"; }; - return VEvent; + return RowEvent; })(); - binlogdata.MinimalTable = (function() { + binlogdata.FieldEvent = (function() { /** - * Properties of a MinimalTable. + * Properties of a FieldEvent. * @memberof binlogdata - * @interface IMinimalTable - * @property {string|null} [name] MinimalTable name - * @property {Array.|null} [fields] MinimalTable fields - * @property {Array.|null} [p_k_columns] MinimalTable p_k_columns + * @interface IFieldEvent + * @property {string|null} [table_name] FieldEvent table_name + * @property {Array.|null} [fields] FieldEvent fields + * @property {string|null} [keyspace] FieldEvent keyspace + * @property {string|null} [shard] FieldEvent shard */ /** - * Constructs a new MinimalTable. + * Constructs a new FieldEvent. * @memberof binlogdata - * @classdesc Represents a MinimalTable. - * @implements IMinimalTable + * @classdesc Represents a FieldEvent. + * @implements IFieldEvent * @constructor - * @param {binlogdata.IMinimalTable=} [properties] Properties to set + * @param {binlogdata.IFieldEvent=} [properties] Properties to set */ - function MinimalTable(properties) { + function FieldEvent(properties) { this.fields = []; - this.p_k_columns = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -69487,100 +70197,106 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * MinimalTable name. - * @member {string} name - * @memberof binlogdata.MinimalTable + * FieldEvent table_name. + * @member {string} table_name + * @memberof binlogdata.FieldEvent * @instance */ - MinimalTable.prototype.name = ""; + FieldEvent.prototype.table_name = ""; /** - * MinimalTable fields. + * FieldEvent fields. * @member {Array.} fields - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @instance */ - MinimalTable.prototype.fields = $util.emptyArray; + FieldEvent.prototype.fields = $util.emptyArray; /** - * MinimalTable p_k_columns. - * @member {Array.} p_k_columns - * @memberof binlogdata.MinimalTable + * FieldEvent keyspace. + * @member {string} keyspace + * @memberof binlogdata.FieldEvent * @instance */ - MinimalTable.prototype.p_k_columns = $util.emptyArray; + FieldEvent.prototype.keyspace = ""; /** - * Creates a new MinimalTable instance using the specified properties. + * FieldEvent shard. + * @member {string} shard + * @memberof binlogdata.FieldEvent + * @instance + */ + FieldEvent.prototype.shard = ""; + + /** + * Creates a new FieldEvent instance using the specified properties. * @function create - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static - * @param {binlogdata.IMinimalTable=} [properties] Properties to set - * @returns {binlogdata.MinimalTable} MinimalTable instance + * @param {binlogdata.IFieldEvent=} [properties] Properties to set + * @returns {binlogdata.FieldEvent} FieldEvent instance */ - MinimalTable.create = function create(properties) { - return new MinimalTable(properties); + FieldEvent.create = function create(properties) { + return new FieldEvent(properties); }; /** - * Encodes the specified MinimalTable message. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. + * Encodes the specified FieldEvent message. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. * @function encode - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static - * @param {binlogdata.IMinimalTable} message MinimalTable message or plain object to encode + * @param {binlogdata.IFieldEvent} message FieldEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MinimalTable.encode = function encode(message, writer) { + FieldEvent.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); if (message.fields != null && message.fields.length) for (let i = 0; i < message.fields.length; ++i) $root.query.Field.encode(message.fields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.p_k_columns != null && message.p_k_columns.length) { - writer.uint32(/* id 3, wireType 2 =*/26).fork(); - for (let i = 0; i < message.p_k_columns.length; ++i) - writer.int64(message.p_k_columns[i]); - writer.ldelim(); - } + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard); return writer; }; /** - * Encodes the specified MinimalTable message, length delimited. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. + * Encodes the specified FieldEvent message, length delimited. Does not implicitly {@link binlogdata.FieldEvent.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static - * @param {binlogdata.IMinimalTable} message MinimalTable message or plain object to encode + * @param {binlogdata.IFieldEvent} message FieldEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MinimalTable.encodeDelimited = function encodeDelimited(message, writer) { + FieldEvent.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MinimalTable message from the specified reader or buffer. + * Decodes a FieldEvent message from the specified reader or buffer. * @function decode - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.MinimalTable} MinimalTable + * @returns {binlogdata.FieldEvent} FieldEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MinimalTable.decode = function decode(reader, length) { + FieldEvent.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.MinimalTable(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.FieldEvent(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.table_name = reader.string(); break; } case 2: { @@ -69590,14 +70306,11 @@ export const binlogdata = $root.binlogdata = (() => { break; } case 3: { - if (!(message.p_k_columns && message.p_k_columns.length)) - message.p_k_columns = []; - if ((tag & 7) === 2) { - let end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) - message.p_k_columns.push(reader.int64()); - } else - message.p_k_columns.push(reader.int64()); + message.keyspace = reader.string(); + break; + } + case 4: { + message.shard = reader.string(); break; } default: @@ -69609,35 +70322,35 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a MinimalTable message from the specified reader or buffer, length delimited. + * Decodes a FieldEvent message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.MinimalTable} MinimalTable + * @returns {binlogdata.FieldEvent} FieldEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MinimalTable.decodeDelimited = function decodeDelimited(reader) { + FieldEvent.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MinimalTable message. + * Verifies a FieldEvent message. * @function verify - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MinimalTable.verify = function verify(message) { + FieldEvent.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; + if (message.table_name != null && message.hasOwnProperty("table_name")) + if (!$util.isString(message.table_name)) + return "table_name: string expected"; if (message.fields != null && message.hasOwnProperty("fields")) { if (!Array.isArray(message.fields)) return "fields: array expected"; @@ -69647,142 +70360,131 @@ export const binlogdata = $root.binlogdata = (() => { return "fields." + error; } } - if (message.p_k_columns != null && message.hasOwnProperty("p_k_columns")) { - if (!Array.isArray(message.p_k_columns)) - return "p_k_columns: array expected"; - for (let i = 0; i < message.p_k_columns.length; ++i) - if (!$util.isInteger(message.p_k_columns[i]) && !(message.p_k_columns[i] && $util.isInteger(message.p_k_columns[i].low) && $util.isInteger(message.p_k_columns[i].high))) - return "p_k_columns: integer|Long[] expected"; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; return null; }; /** - * Creates a MinimalTable message from a plain object. Also converts values to their respective internal types. + * Creates a FieldEvent message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static * @param {Object.} object Plain object - * @returns {binlogdata.MinimalTable} MinimalTable + * @returns {binlogdata.FieldEvent} FieldEvent */ - MinimalTable.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.MinimalTable) + FieldEvent.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.FieldEvent) return object; - let message = new $root.binlogdata.MinimalTable(); - if (object.name != null) - message.name = String(object.name); + let message = new $root.binlogdata.FieldEvent(); + if (object.table_name != null) + message.table_name = String(object.table_name); if (object.fields) { if (!Array.isArray(object.fields)) - throw TypeError(".binlogdata.MinimalTable.fields: array expected"); + throw TypeError(".binlogdata.FieldEvent.fields: array expected"); message.fields = []; for (let i = 0; i < object.fields.length; ++i) { if (typeof object.fields[i] !== "object") - throw TypeError(".binlogdata.MinimalTable.fields: object expected"); + throw TypeError(".binlogdata.FieldEvent.fields: object expected"); message.fields[i] = $root.query.Field.fromObject(object.fields[i]); } } - if (object.p_k_columns) { - if (!Array.isArray(object.p_k_columns)) - throw TypeError(".binlogdata.MinimalTable.p_k_columns: array expected"); - message.p_k_columns = []; - for (let i = 0; i < object.p_k_columns.length; ++i) - if ($util.Long) - (message.p_k_columns[i] = $util.Long.fromValue(object.p_k_columns[i])).unsigned = false; - else if (typeof object.p_k_columns[i] === "string") - message.p_k_columns[i] = parseInt(object.p_k_columns[i], 10); - else if (typeof object.p_k_columns[i] === "number") - message.p_k_columns[i] = object.p_k_columns[i]; - else if (typeof object.p_k_columns[i] === "object") - message.p_k_columns[i] = new $util.LongBits(object.p_k_columns[i].low >>> 0, object.p_k_columns[i].high >>> 0).toNumber(); - } + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); return message; }; /** - * Creates a plain object from a MinimalTable message. Also converts values to other types if specified. + * Creates a plain object from a FieldEvent message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static - * @param {binlogdata.MinimalTable} message MinimalTable + * @param {binlogdata.FieldEvent} message FieldEvent * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MinimalTable.toObject = function toObject(message, options) { + FieldEvent.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { + if (options.arrays || options.defaults) object.fields = []; - object.p_k_columns = []; + if (options.defaults) { + object.table_name = ""; + object.keyspace = ""; + object.shard = ""; } - if (options.defaults) - object.name = ""; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; + if (message.table_name != null && message.hasOwnProperty("table_name")) + object.table_name = message.table_name; if (message.fields && message.fields.length) { object.fields = []; for (let j = 0; j < message.fields.length; ++j) object.fields[j] = $root.query.Field.toObject(message.fields[j], options); } - if (message.p_k_columns && message.p_k_columns.length) { - object.p_k_columns = []; - for (let j = 0; j < message.p_k_columns.length; ++j) - if (typeof message.p_k_columns[j] === "number") - object.p_k_columns[j] = options.longs === String ? String(message.p_k_columns[j]) : message.p_k_columns[j]; - else - object.p_k_columns[j] = options.longs === String ? $util.Long.prototype.toString.call(message.p_k_columns[j]) : options.longs === Number ? new $util.LongBits(message.p_k_columns[j].low >>> 0, message.p_k_columns[j].high >>> 0).toNumber() : message.p_k_columns[j]; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; return object; }; /** - * Converts this MinimalTable to JSON. + * Converts this FieldEvent to JSON. * @function toJSON - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @instance * @returns {Object.} JSON object */ - MinimalTable.prototype.toJSON = function toJSON() { + FieldEvent.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MinimalTable + * Gets the default type url for FieldEvent * @function getTypeUrl - * @memberof binlogdata.MinimalTable + * @memberof binlogdata.FieldEvent * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MinimalTable.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + FieldEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.MinimalTable"; + return typeUrlPrefix + "/binlogdata.FieldEvent"; }; - return MinimalTable; + return FieldEvent; })(); - binlogdata.MinimalSchema = (function() { + binlogdata.ShardGtid = (function() { /** - * Properties of a MinimalSchema. + * Properties of a ShardGtid. * @memberof binlogdata - * @interface IMinimalSchema - * @property {Array.|null} [tables] MinimalSchema tables + * @interface IShardGtid + * @property {string|null} [keyspace] ShardGtid keyspace + * @property {string|null} [shard] ShardGtid shard + * @property {string|null} [gtid] ShardGtid gtid + * @property {Array.|null} [table_p_ks] ShardGtid table_p_ks */ /** - * Constructs a new MinimalSchema. + * Constructs a new ShardGtid. * @memberof binlogdata - * @classdesc Represents a MinimalSchema. - * @implements IMinimalSchema + * @classdesc Represents a ShardGtid. + * @implements IShardGtid * @constructor - * @param {binlogdata.IMinimalSchema=} [properties] Properties to set + * @param {binlogdata.IShardGtid=} [properties] Properties to set */ - function MinimalSchema(properties) { - this.tables = []; + function ShardGtid(properties) { + this.table_p_ks = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -69790,78 +70492,120 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * MinimalSchema tables. - * @member {Array.} tables - * @memberof binlogdata.MinimalSchema + * ShardGtid keyspace. + * @member {string} keyspace + * @memberof binlogdata.ShardGtid * @instance */ - MinimalSchema.prototype.tables = $util.emptyArray; + ShardGtid.prototype.keyspace = ""; /** - * Creates a new MinimalSchema instance using the specified properties. + * ShardGtid shard. + * @member {string} shard + * @memberof binlogdata.ShardGtid + * @instance + */ + ShardGtid.prototype.shard = ""; + + /** + * ShardGtid gtid. + * @member {string} gtid + * @memberof binlogdata.ShardGtid + * @instance + */ + ShardGtid.prototype.gtid = ""; + + /** + * ShardGtid table_p_ks. + * @member {Array.} table_p_ks + * @memberof binlogdata.ShardGtid + * @instance + */ + ShardGtid.prototype.table_p_ks = $util.emptyArray; + + /** + * Creates a new ShardGtid instance using the specified properties. * @function create - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static - * @param {binlogdata.IMinimalSchema=} [properties] Properties to set - * @returns {binlogdata.MinimalSchema} MinimalSchema instance + * @param {binlogdata.IShardGtid=} [properties] Properties to set + * @returns {binlogdata.ShardGtid} ShardGtid instance */ - MinimalSchema.create = function create(properties) { - return new MinimalSchema(properties); + ShardGtid.create = function create(properties) { + return new ShardGtid(properties); }; /** - * Encodes the specified MinimalSchema message. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. + * Encodes the specified ShardGtid message. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. * @function encode - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static - * @param {binlogdata.IMinimalSchema} message MinimalSchema message or plain object to encode + * @param {binlogdata.IShardGtid} message ShardGtid message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MinimalSchema.encode = function encode(message, writer) { + ShardGtid.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - $root.binlogdata.MinimalTable.encode(message.tables[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - return writer; + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); + if (message.table_p_ks != null && message.table_p_ks.length) + for (let i = 0; i < message.table_p_ks.length; ++i) + $root.binlogdata.TableLastPK.encode(message.table_p_ks[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + return writer; }; /** - * Encodes the specified MinimalSchema message, length delimited. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. + * Encodes the specified ShardGtid message, length delimited. Does not implicitly {@link binlogdata.ShardGtid.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static - * @param {binlogdata.IMinimalSchema} message MinimalSchema message or plain object to encode + * @param {binlogdata.IShardGtid} message ShardGtid message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MinimalSchema.encodeDelimited = function encodeDelimited(message, writer) { + ShardGtid.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MinimalSchema message from the specified reader or buffer. + * Decodes a ShardGtid message from the specified reader or buffer. * @function decode - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.MinimalSchema} MinimalSchema + * @returns {binlogdata.ShardGtid} ShardGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MinimalSchema.decode = function decode(reader, length) { + ShardGtid.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.MinimalSchema(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.ShardGtid(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push($root.binlogdata.MinimalTable.decode(reader, reader.uint32())); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.gtid = reader.string(); + break; + } + case 4: { + if (!(message.table_p_ks && message.table_p_ks.length)) + message.table_p_ks = []; + message.table_p_ks.push($root.binlogdata.TableLastPK.decode(reader, reader.uint32())); break; } default: @@ -69873,145 +70617,166 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a MinimalSchema message from the specified reader or buffer, length delimited. + * Decodes a ShardGtid message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.MinimalSchema} MinimalSchema + * @returns {binlogdata.ShardGtid} ShardGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MinimalSchema.decodeDelimited = function decodeDelimited(reader) { + ShardGtid.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MinimalSchema message. + * Verifies a ShardGtid message. * @function verify - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MinimalSchema.verify = function verify(message) { + ShardGtid.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) { - let error = $root.binlogdata.MinimalTable.verify(message.tables[i]); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.gtid != null && message.hasOwnProperty("gtid")) + if (!$util.isString(message.gtid)) + return "gtid: string expected"; + if (message.table_p_ks != null && message.hasOwnProperty("table_p_ks")) { + if (!Array.isArray(message.table_p_ks)) + return "table_p_ks: array expected"; + for (let i = 0; i < message.table_p_ks.length; ++i) { + let error = $root.binlogdata.TableLastPK.verify(message.table_p_ks[i]); if (error) - return "tables." + error; + return "table_p_ks." + error; } } return null; }; /** - * Creates a MinimalSchema message from a plain object. Also converts values to their respective internal types. + * Creates a ShardGtid message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static * @param {Object.} object Plain object - * @returns {binlogdata.MinimalSchema} MinimalSchema + * @returns {binlogdata.ShardGtid} ShardGtid */ - MinimalSchema.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.MinimalSchema) + ShardGtid.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.ShardGtid) return object; - let message = new $root.binlogdata.MinimalSchema(); - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".binlogdata.MinimalSchema.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) { - if (typeof object.tables[i] !== "object") - throw TypeError(".binlogdata.MinimalSchema.tables: object expected"); - message.tables[i] = $root.binlogdata.MinimalTable.fromObject(object.tables[i]); + let message = new $root.binlogdata.ShardGtid(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.gtid != null) + message.gtid = String(object.gtid); + if (object.table_p_ks) { + if (!Array.isArray(object.table_p_ks)) + throw TypeError(".binlogdata.ShardGtid.table_p_ks: array expected"); + message.table_p_ks = []; + for (let i = 0; i < object.table_p_ks.length; ++i) { + if (typeof object.table_p_ks[i] !== "object") + throw TypeError(".binlogdata.ShardGtid.table_p_ks: object expected"); + message.table_p_ks[i] = $root.binlogdata.TableLastPK.fromObject(object.table_p_ks[i]); } } return message; }; /** - * Creates a plain object from a MinimalSchema message. Also converts values to other types if specified. + * Creates a plain object from a ShardGtid message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static - * @param {binlogdata.MinimalSchema} message MinimalSchema + * @param {binlogdata.ShardGtid} message ShardGtid * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MinimalSchema.toObject = function toObject(message, options) { + ShardGtid.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.tables = []; - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = $root.binlogdata.MinimalTable.toObject(message.tables[j], options); + object.table_p_ks = []; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.gtid = ""; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.gtid != null && message.hasOwnProperty("gtid")) + object.gtid = message.gtid; + if (message.table_p_ks && message.table_p_ks.length) { + object.table_p_ks = []; + for (let j = 0; j < message.table_p_ks.length; ++j) + object.table_p_ks[j] = $root.binlogdata.TableLastPK.toObject(message.table_p_ks[j], options); } return object; }; /** - * Converts this MinimalSchema to JSON. + * Converts this ShardGtid to JSON. * @function toJSON - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @instance * @returns {Object.} JSON object */ - MinimalSchema.prototype.toJSON = function toJSON() { + ShardGtid.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MinimalSchema + * Gets the default type url for ShardGtid * @function getTypeUrl - * @memberof binlogdata.MinimalSchema + * @memberof binlogdata.ShardGtid * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MinimalSchema.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardGtid.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.MinimalSchema"; + return typeUrlPrefix + "/binlogdata.ShardGtid"; }; - return MinimalSchema; + return ShardGtid; })(); - binlogdata.VStreamRequest = (function() { + binlogdata.VGtid = (function() { /** - * Properties of a VStreamRequest. + * Properties of a VGtid. * @memberof binlogdata - * @interface IVStreamRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamRequest immediate_caller_id - * @property {query.ITarget|null} [target] VStreamRequest target - * @property {string|null} [position] VStreamRequest position - * @property {binlogdata.IFilter|null} [filter] VStreamRequest filter - * @property {Array.|null} [table_last_p_ks] VStreamRequest table_last_p_ks + * @interface IVGtid + * @property {Array.|null} [shard_gtids] VGtid shard_gtids */ /** - * Constructs a new VStreamRequest. + * Constructs a new VGtid. * @memberof binlogdata - * @classdesc Represents a VStreamRequest. - * @implements IVStreamRequest + * @classdesc Represents a VGtid. + * @implements IVGtid * @constructor - * @param {binlogdata.IVStreamRequest=} [properties] Properties to set + * @param {binlogdata.IVGtid=} [properties] Properties to set */ - function VStreamRequest(properties) { - this.table_last_p_ks = []; + function VGtid(properties) { + this.shard_gtids = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -70019,148 +70784,78 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof binlogdata.VStreamRequest - * @instance - */ - VStreamRequest.prototype.effective_caller_id = null; - - /** - * VStreamRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof binlogdata.VStreamRequest - * @instance - */ - VStreamRequest.prototype.immediate_caller_id = null; - - /** - * VStreamRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof binlogdata.VStreamRequest - * @instance - */ - VStreamRequest.prototype.target = null; - - /** - * VStreamRequest position. - * @member {string} position - * @memberof binlogdata.VStreamRequest - * @instance - */ - VStreamRequest.prototype.position = ""; - - /** - * VStreamRequest filter. - * @member {binlogdata.IFilter|null|undefined} filter - * @memberof binlogdata.VStreamRequest - * @instance - */ - VStreamRequest.prototype.filter = null; - - /** - * VStreamRequest table_last_p_ks. - * @member {Array.} table_last_p_ks - * @memberof binlogdata.VStreamRequest + * VGtid shard_gtids. + * @member {Array.} shard_gtids + * @memberof binlogdata.VGtid * @instance */ - VStreamRequest.prototype.table_last_p_ks = $util.emptyArray; + VGtid.prototype.shard_gtids = $util.emptyArray; /** - * Creates a new VStreamRequest instance using the specified properties. + * Creates a new VGtid instance using the specified properties. * @function create - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static - * @param {binlogdata.IVStreamRequest=} [properties] Properties to set - * @returns {binlogdata.VStreamRequest} VStreamRequest instance + * @param {binlogdata.IVGtid=} [properties] Properties to set + * @returns {binlogdata.VGtid} VGtid instance */ - VStreamRequest.create = function create(properties) { - return new VStreamRequest(properties); + VGtid.create = function create(properties) { + return new VGtid(properties); }; /** - * Encodes the specified VStreamRequest message. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. + * Encodes the specified VGtid message. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static - * @param {binlogdata.IVStreamRequest} message VStreamRequest message or plain object to encode + * @param {binlogdata.IVGtid} message VGtid message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRequest.encode = function encode(message, writer) { + VGtid.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.position); - if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) - $root.binlogdata.Filter.encode(message.filter, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.table_last_p_ks != null && message.table_last_p_ks.length) - for (let i = 0; i < message.table_last_p_ks.length; ++i) - $root.binlogdata.TableLastPK.encode(message.table_last_p_ks[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.shard_gtids != null && message.shard_gtids.length) + for (let i = 0; i < message.shard_gtids.length; ++i) + $root.binlogdata.ShardGtid.encode(message.shard_gtids[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified VStreamRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. + * Encodes the specified VGtid message, length delimited. Does not implicitly {@link binlogdata.VGtid.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static - * @param {binlogdata.IVStreamRequest} message VStreamRequest message or plain object to encode + * @param {binlogdata.IVGtid} message VGtid message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRequest.encodeDelimited = function encodeDelimited(message, writer) { + VGtid.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamRequest message from the specified reader or buffer. + * Decodes a VGtid message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamRequest} VStreamRequest + * @returns {binlogdata.VGtid} VGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRequest.decode = function decode(reader, length) { + VGtid.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VGtid(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); - break; - } - case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.position = reader.string(); - break; - } - case 5: { - message.filter = $root.binlogdata.Filter.decode(reader, reader.uint32()); - break; - } - case 6: { - if (!(message.table_last_p_ks && message.table_last_p_ks.length)) - message.table_last_p_ks = []; - message.table_last_p_ks.push($root.binlogdata.TableLastPK.decode(reader, reader.uint32())); + if (!(message.shard_gtids && message.shard_gtids.length)) + message.shard_gtids = []; + message.shard_gtids.push($root.binlogdata.ShardGtid.decode(reader, reader.uint32())); break; } default: @@ -70172,202 +70867,140 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamRequest message from the specified reader or buffer, length delimited. + * Decodes a VGtid message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamRequest} VStreamRequest + * @returns {binlogdata.VGtid} VGtid * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRequest.decodeDelimited = function decodeDelimited(reader) { + VGtid.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamRequest message. + * Verifies a VGtid message. * @function verify - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamRequest.verify = function verify(message) { + VGtid.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; - } - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; - if (message.filter != null && message.hasOwnProperty("filter")) { - let error = $root.binlogdata.Filter.verify(message.filter); - if (error) - return "filter." + error; - } - if (message.table_last_p_ks != null && message.hasOwnProperty("table_last_p_ks")) { - if (!Array.isArray(message.table_last_p_ks)) - return "table_last_p_ks: array expected"; - for (let i = 0; i < message.table_last_p_ks.length; ++i) { - let error = $root.binlogdata.TableLastPK.verify(message.table_last_p_ks[i]); + if (message.shard_gtids != null && message.hasOwnProperty("shard_gtids")) { + if (!Array.isArray(message.shard_gtids)) + return "shard_gtids: array expected"; + for (let i = 0; i < message.shard_gtids.length; ++i) { + let error = $root.binlogdata.ShardGtid.verify(message.shard_gtids[i]); if (error) - return "table_last_p_ks." + error; + return "shard_gtids." + error; } } return null; }; /** - * Creates a VStreamRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VGtid message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamRequest} VStreamRequest + * @returns {binlogdata.VGtid} VGtid */ - VStreamRequest.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamRequest) + VGtid.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VGtid) return object; - let message = new $root.binlogdata.VStreamRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".binlogdata.VStreamRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); - } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".binlogdata.VStreamRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".binlogdata.VStreamRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); - } - if (object.position != null) - message.position = String(object.position); - if (object.filter != null) { - if (typeof object.filter !== "object") - throw TypeError(".binlogdata.VStreamRequest.filter: object expected"); - message.filter = $root.binlogdata.Filter.fromObject(object.filter); - } - if (object.table_last_p_ks) { - if (!Array.isArray(object.table_last_p_ks)) - throw TypeError(".binlogdata.VStreamRequest.table_last_p_ks: array expected"); - message.table_last_p_ks = []; - for (let i = 0; i < object.table_last_p_ks.length; ++i) { - if (typeof object.table_last_p_ks[i] !== "object") - throw TypeError(".binlogdata.VStreamRequest.table_last_p_ks: object expected"); - message.table_last_p_ks[i] = $root.binlogdata.TableLastPK.fromObject(object.table_last_p_ks[i]); + let message = new $root.binlogdata.VGtid(); + if (object.shard_gtids) { + if (!Array.isArray(object.shard_gtids)) + throw TypeError(".binlogdata.VGtid.shard_gtids: array expected"); + message.shard_gtids = []; + for (let i = 0; i < object.shard_gtids.length; ++i) { + if (typeof object.shard_gtids[i] !== "object") + throw TypeError(".binlogdata.VGtid.shard_gtids: object expected"); + message.shard_gtids[i] = $root.binlogdata.ShardGtid.fromObject(object.shard_gtids[i]); } } return message; }; /** - * Creates a plain object from a VStreamRequest message. Also converts values to other types if specified. + * Creates a plain object from a VGtid message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static - * @param {binlogdata.VStreamRequest} message VStreamRequest + * @param {binlogdata.VGtid} message VGtid * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VStreamRequest.toObject = function toObject(message, options) { + VGtid.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.table_last_p_ks = []; - if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.position = ""; - object.filter = null; - } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; - if (message.filter != null && message.hasOwnProperty("filter")) - object.filter = $root.binlogdata.Filter.toObject(message.filter, options); - if (message.table_last_p_ks && message.table_last_p_ks.length) { - object.table_last_p_ks = []; - for (let j = 0; j < message.table_last_p_ks.length; ++j) - object.table_last_p_ks[j] = $root.binlogdata.TableLastPK.toObject(message.table_last_p_ks[j], options); + object.shard_gtids = []; + if (message.shard_gtids && message.shard_gtids.length) { + object.shard_gtids = []; + for (let j = 0; j < message.shard_gtids.length; ++j) + object.shard_gtids[j] = $root.binlogdata.ShardGtid.toObject(message.shard_gtids[j], options); } return object; }; /** - * Converts this VStreamRequest to JSON. + * Converts this VGtid to JSON. * @function toJSON - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @instance * @returns {Object.} JSON object */ - VStreamRequest.prototype.toJSON = function toJSON() { + VGtid.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamRequest + * Gets the default type url for VGtid * @function getTypeUrl - * @memberof binlogdata.VStreamRequest + * @memberof binlogdata.VGtid * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VGtid.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamRequest"; + return typeUrlPrefix + "/binlogdata.VGtid"; }; - return VStreamRequest; + return VGtid; })(); - binlogdata.VStreamResponse = (function() { + binlogdata.KeyspaceShard = (function() { /** - * Properties of a VStreamResponse. + * Properties of a KeyspaceShard. * @memberof binlogdata - * @interface IVStreamResponse - * @property {Array.|null} [events] VStreamResponse events + * @interface IKeyspaceShard + * @property {string|null} [keyspace] KeyspaceShard keyspace + * @property {string|null} [shard] KeyspaceShard shard */ /** - * Constructs a new VStreamResponse. + * Constructs a new KeyspaceShard. * @memberof binlogdata - * @classdesc Represents a VStreamResponse. - * @implements IVStreamResponse + * @classdesc Represents a KeyspaceShard. + * @implements IKeyspaceShard * @constructor - * @param {binlogdata.IVStreamResponse=} [properties] Properties to set + * @param {binlogdata.IKeyspaceShard=} [properties] Properties to set */ - function VStreamResponse(properties) { - this.events = []; + function KeyspaceShard(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -70375,78 +71008,89 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamResponse events. - * @member {Array.} events - * @memberof binlogdata.VStreamResponse + * KeyspaceShard keyspace. + * @member {string} keyspace + * @memberof binlogdata.KeyspaceShard * @instance */ - VStreamResponse.prototype.events = $util.emptyArray; + KeyspaceShard.prototype.keyspace = ""; /** - * Creates a new VStreamResponse instance using the specified properties. + * KeyspaceShard shard. + * @member {string} shard + * @memberof binlogdata.KeyspaceShard + * @instance + */ + KeyspaceShard.prototype.shard = ""; + + /** + * Creates a new KeyspaceShard instance using the specified properties. * @function create - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static - * @param {binlogdata.IVStreamResponse=} [properties] Properties to set - * @returns {binlogdata.VStreamResponse} VStreamResponse instance + * @param {binlogdata.IKeyspaceShard=} [properties] Properties to set + * @returns {binlogdata.KeyspaceShard} KeyspaceShard instance */ - VStreamResponse.create = function create(properties) { - return new VStreamResponse(properties); + KeyspaceShard.create = function create(properties) { + return new KeyspaceShard(properties); }; /** - * Encodes the specified VStreamResponse message. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. + * Encodes the specified KeyspaceShard message. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static - * @param {binlogdata.IVStreamResponse} message VStreamResponse message or plain object to encode + * @param {binlogdata.IKeyspaceShard} message KeyspaceShard message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResponse.encode = function encode(message, writer) { + KeyspaceShard.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.binlogdata.VEvent.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); return writer; }; /** - * Encodes the specified VStreamResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. + * Encodes the specified KeyspaceShard message, length delimited. Does not implicitly {@link binlogdata.KeyspaceShard.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static - * @param {binlogdata.IVStreamResponse} message VStreamResponse message or plain object to encode + * @param {binlogdata.IKeyspaceShard} message KeyspaceShard message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResponse.encodeDelimited = function encodeDelimited(message, writer) { + KeyspaceShard.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamResponse message from the specified reader or buffer. + * Decodes a KeyspaceShard message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamResponse} VStreamResponse + * @returns {binlogdata.KeyspaceShard} KeyspaceShard * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResponse.decode = function decode(reader, length) { + KeyspaceShard.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.KeyspaceShard(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.binlogdata.VEvent.decode(reader, reader.uint32())); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); break; } default: @@ -70458,143 +71102,155 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamResponse message from the specified reader or buffer, length delimited. + * Decodes a KeyspaceShard message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamResponse} VStreamResponse + * @returns {binlogdata.KeyspaceShard} KeyspaceShard * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResponse.decodeDelimited = function decodeDelimited(reader) { + KeyspaceShard.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamResponse message. + * Verifies a KeyspaceShard message. * @function verify - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamResponse.verify = function verify(message) { + KeyspaceShard.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.binlogdata.VEvent.verify(message.events[i]); - if (error) - return "events." + error; - } - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; return null; }; /** - * Creates a VStreamResponse message from a plain object. Also converts values to their respective internal types. + * Creates a KeyspaceShard message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamResponse} VStreamResponse + * @returns {binlogdata.KeyspaceShard} KeyspaceShard */ - VStreamResponse.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamResponse) + KeyspaceShard.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.KeyspaceShard) return object; - let message = new $root.binlogdata.VStreamResponse(); - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".binlogdata.VStreamResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".binlogdata.VStreamResponse.events: object expected"); - message.events[i] = $root.binlogdata.VEvent.fromObject(object.events[i]); - } - } + let message = new $root.binlogdata.KeyspaceShard(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); return message; }; /** - * Creates a plain object from a VStreamResponse message. Also converts values to other types if specified. + * Creates a plain object from a KeyspaceShard message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static - * @param {binlogdata.VStreamResponse} message VStreamResponse + * @param {binlogdata.KeyspaceShard} message KeyspaceShard * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VStreamResponse.toObject = function toObject(message, options) { + KeyspaceShard.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.events = []; - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.binlogdata.VEvent.toObject(message.events[j], options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; return object; }; /** - * Converts this VStreamResponse to JSON. + * Converts this KeyspaceShard to JSON. * @function toJSON - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @instance * @returns {Object.} JSON object */ - VStreamResponse.prototype.toJSON = function toJSON() { + KeyspaceShard.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamResponse + * Gets the default type url for KeyspaceShard * @function getTypeUrl - * @memberof binlogdata.VStreamResponse + * @memberof binlogdata.KeyspaceShard * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + KeyspaceShard.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamResponse"; + return typeUrlPrefix + "/binlogdata.KeyspaceShard"; }; - return VStreamResponse; + return KeyspaceShard; })(); - binlogdata.VStreamRowsRequest = (function() { + /** + * MigrationType enum. + * @name binlogdata.MigrationType + * @enum {number} + * @property {number} TABLES=0 TABLES value + * @property {number} SHARDS=1 SHARDS value + */ + binlogdata.MigrationType = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "TABLES"] = 0; + values[valuesById[1] = "SHARDS"] = 1; + return values; + })(); + + binlogdata.Journal = (function() { /** - * Properties of a VStreamRowsRequest. + * Properties of a Journal. * @memberof binlogdata - * @interface IVStreamRowsRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamRowsRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamRowsRequest immediate_caller_id - * @property {query.ITarget|null} [target] VStreamRowsRequest target - * @property {string|null} [query] VStreamRowsRequest query - * @property {query.IQueryResult|null} [lastpk] VStreamRowsRequest lastpk + * @interface IJournal + * @property {number|Long|null} [id] Journal id + * @property {binlogdata.MigrationType|null} [migration_type] Journal migration_type + * @property {Array.|null} [tables] Journal tables + * @property {string|null} [local_position] Journal local_position + * @property {Array.|null} [shard_gtids] Journal shard_gtids + * @property {Array.|null} [participants] Journal participants + * @property {Array.|null} [source_workflows] Journal source_workflows */ /** - * Constructs a new VStreamRowsRequest. + * Constructs a new Journal. * @memberof binlogdata - * @classdesc Represents a VStreamRowsRequest. - * @implements IVStreamRowsRequest + * @classdesc Represents a Journal. + * @implements IJournal * @constructor - * @param {binlogdata.IVStreamRowsRequest=} [properties] Properties to set + * @param {binlogdata.IJournal=} [properties] Properties to set */ - function VStreamRowsRequest(properties) { + function Journal(properties) { + this.tables = []; + this.shard_gtids = []; + this.participants = []; + this.source_workflows = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -70602,131 +71258,171 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamRowsRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof binlogdata.VStreamRowsRequest + * Journal id. + * @member {number|Long} id + * @memberof binlogdata.Journal * @instance */ - VStreamRowsRequest.prototype.effective_caller_id = null; + Journal.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * VStreamRowsRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof binlogdata.VStreamRowsRequest + * Journal migration_type. + * @member {binlogdata.MigrationType} migration_type + * @memberof binlogdata.Journal * @instance */ - VStreamRowsRequest.prototype.immediate_caller_id = null; + Journal.prototype.migration_type = 0; /** - * VStreamRowsRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof binlogdata.VStreamRowsRequest + * Journal tables. + * @member {Array.} tables + * @memberof binlogdata.Journal * @instance */ - VStreamRowsRequest.prototype.target = null; + Journal.prototype.tables = $util.emptyArray; /** - * VStreamRowsRequest query. - * @member {string} query - * @memberof binlogdata.VStreamRowsRequest + * Journal local_position. + * @member {string} local_position + * @memberof binlogdata.Journal * @instance */ - VStreamRowsRequest.prototype.query = ""; + Journal.prototype.local_position = ""; /** - * VStreamRowsRequest lastpk. - * @member {query.IQueryResult|null|undefined} lastpk - * @memberof binlogdata.VStreamRowsRequest + * Journal shard_gtids. + * @member {Array.} shard_gtids + * @memberof binlogdata.Journal * @instance */ - VStreamRowsRequest.prototype.lastpk = null; + Journal.prototype.shard_gtids = $util.emptyArray; /** - * Creates a new VStreamRowsRequest instance using the specified properties. + * Journal participants. + * @member {Array.} participants + * @memberof binlogdata.Journal + * @instance + */ + Journal.prototype.participants = $util.emptyArray; + + /** + * Journal source_workflows. + * @member {Array.} source_workflows + * @memberof binlogdata.Journal + * @instance + */ + Journal.prototype.source_workflows = $util.emptyArray; + + /** + * Creates a new Journal instance using the specified properties. * @function create - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static - * @param {binlogdata.IVStreamRowsRequest=} [properties] Properties to set - * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest instance + * @param {binlogdata.IJournal=} [properties] Properties to set + * @returns {binlogdata.Journal} Journal instance */ - VStreamRowsRequest.create = function create(properties) { - return new VStreamRowsRequest(properties); + Journal.create = function create(properties) { + return new Journal(properties); }; /** - * Encodes the specified VStreamRowsRequest message. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. + * Encodes the specified Journal message. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static - * @param {binlogdata.IVStreamRowsRequest} message VStreamRowsRequest message or plain object to encode + * @param {binlogdata.IJournal} message Journal message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRowsRequest.encode = function encode(message, writer) { + Journal.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.query); - if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) - $root.query.QueryResult.encode(message.lastpk, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); + if (message.migration_type != null && Object.hasOwnProperty.call(message, "migration_type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.migration_type); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.tables[i]); + if (message.local_position != null && Object.hasOwnProperty.call(message, "local_position")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.local_position); + if (message.shard_gtids != null && message.shard_gtids.length) + for (let i = 0; i < message.shard_gtids.length; ++i) + $root.binlogdata.ShardGtid.encode(message.shard_gtids[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.participants != null && message.participants.length) + for (let i = 0; i < message.participants.length; ++i) + $root.binlogdata.KeyspaceShard.encode(message.participants[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.source_workflows != null && message.source_workflows.length) + for (let i = 0; i < message.source_workflows.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.source_workflows[i]); return writer; }; /** - * Encodes the specified VStreamRowsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. + * Encodes the specified Journal message, length delimited. Does not implicitly {@link binlogdata.Journal.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static - * @param {binlogdata.IVStreamRowsRequest} message VStreamRowsRequest message or plain object to encode + * @param {binlogdata.IJournal} message Journal message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRowsRequest.encodeDelimited = function encodeDelimited(message, writer) { + Journal.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamRowsRequest message from the specified reader or buffer. + * Decodes a Journal message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest + * @returns {binlogdata.Journal} Journal * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRowsRequest.decode = function decode(reader, length) { + Journal.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRowsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.Journal(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + message.id = reader.int64(); break; } case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + message.migration_type = reader.int32(); break; } case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); break; } case 4: { - message.query = reader.string(); + message.local_position = reader.string(); break; } case 5: { - message.lastpk = $root.query.QueryResult.decode(reader, reader.uint32()); + if (!(message.shard_gtids && message.shard_gtids.length)) + message.shard_gtids = []; + message.shard_gtids.push($root.binlogdata.ShardGtid.decode(reader, reader.uint32())); + break; + } + case 6: { + if (!(message.participants && message.participants.length)) + message.participants = []; + message.participants.push($root.binlogdata.KeyspaceShard.decode(reader, reader.uint32())); + break; + } + case 7: { + if (!(message.source_workflows && message.source_workflows.length)) + message.source_workflows = []; + message.source_workflows.push(reader.string()); break; } default: @@ -70738,184 +71434,277 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamRowsRequest message from the specified reader or buffer, length delimited. + * Decodes a Journal message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest + * @returns {binlogdata.Journal} Journal * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRowsRequest.decodeDelimited = function decodeDelimited(reader) { + Journal.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamRowsRequest message. + * Verifies a Journal message. * @function verify - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamRowsRequest.verify = function verify(message) { + Journal.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) + return "id: integer|Long expected"; + if (message.migration_type != null && message.hasOwnProperty("migration_type")) + switch (message.migration_type) { + default: + return "migration_type: enum value expected"; + case 0: + case 1: + break; + } + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; + if (message.local_position != null && message.hasOwnProperty("local_position")) + if (!$util.isString(message.local_position)) + return "local_position: string expected"; + if (message.shard_gtids != null && message.hasOwnProperty("shard_gtids")) { + if (!Array.isArray(message.shard_gtids)) + return "shard_gtids: array expected"; + for (let i = 0; i < message.shard_gtids.length; ++i) { + let error = $root.binlogdata.ShardGtid.verify(message.shard_gtids[i]); + if (error) + return "shard_gtids." + error; + } } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; + if (message.participants != null && message.hasOwnProperty("participants")) { + if (!Array.isArray(message.participants)) + return "participants: array expected"; + for (let i = 0; i < message.participants.length; ++i) { + let error = $root.binlogdata.KeyspaceShard.verify(message.participants[i]); + if (error) + return "participants." + error; + } } - if (message.query != null && message.hasOwnProperty("query")) - if (!$util.isString(message.query)) - return "query: string expected"; - if (message.lastpk != null && message.hasOwnProperty("lastpk")) { - let error = $root.query.QueryResult.verify(message.lastpk); - if (error) - return "lastpk." + error; + if (message.source_workflows != null && message.hasOwnProperty("source_workflows")) { + if (!Array.isArray(message.source_workflows)) + return "source_workflows: array expected"; + for (let i = 0; i < message.source_workflows.length; ++i) + if (!$util.isString(message.source_workflows[i])) + return "source_workflows: string[] expected"; } return null; }; /** - * Creates a VStreamRowsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a Journal message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest + * @returns {binlogdata.Journal} Journal */ - VStreamRowsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamRowsRequest) + Journal.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.Journal) return object; - let message = new $root.binlogdata.VStreamRowsRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".binlogdata.VStreamRowsRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + let message = new $root.binlogdata.Journal(); + if (object.id != null) + if ($util.Long) + (message.id = $util.Long.fromValue(object.id)).unsigned = false; + else if (typeof object.id === "string") + message.id = parseInt(object.id, 10); + else if (typeof object.id === "number") + message.id = object.id; + else if (typeof object.id === "object") + message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); + switch (object.migration_type) { + default: + if (typeof object.migration_type === "number") { + message.migration_type = object.migration_type; + break; + } + break; + case "TABLES": + case 0: + message.migration_type = 0; + break; + case "SHARDS": + case 1: + message.migration_type = 1; + break; } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".binlogdata.VStreamRowsRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".binlogdata.Journal.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".binlogdata.VStreamRowsRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + if (object.local_position != null) + message.local_position = String(object.local_position); + if (object.shard_gtids) { + if (!Array.isArray(object.shard_gtids)) + throw TypeError(".binlogdata.Journal.shard_gtids: array expected"); + message.shard_gtids = []; + for (let i = 0; i < object.shard_gtids.length; ++i) { + if (typeof object.shard_gtids[i] !== "object") + throw TypeError(".binlogdata.Journal.shard_gtids: object expected"); + message.shard_gtids[i] = $root.binlogdata.ShardGtid.fromObject(object.shard_gtids[i]); + } } - if (object.query != null) - message.query = String(object.query); - if (object.lastpk != null) { - if (typeof object.lastpk !== "object") - throw TypeError(".binlogdata.VStreamRowsRequest.lastpk: object expected"); - message.lastpk = $root.query.QueryResult.fromObject(object.lastpk); + if (object.participants) { + if (!Array.isArray(object.participants)) + throw TypeError(".binlogdata.Journal.participants: array expected"); + message.participants = []; + for (let i = 0; i < object.participants.length; ++i) { + if (typeof object.participants[i] !== "object") + throw TypeError(".binlogdata.Journal.participants: object expected"); + message.participants[i] = $root.binlogdata.KeyspaceShard.fromObject(object.participants[i]); + } + } + if (object.source_workflows) { + if (!Array.isArray(object.source_workflows)) + throw TypeError(".binlogdata.Journal.source_workflows: array expected"); + message.source_workflows = []; + for (let i = 0; i < object.source_workflows.length; ++i) + message.source_workflows[i] = String(object.source_workflows[i]); } return message; }; /** - * Creates a plain object from a VStreamRowsRequest message. Also converts values to other types if specified. + * Creates a plain object from a Journal message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static - * @param {binlogdata.VStreamRowsRequest} message VStreamRowsRequest + * @param {binlogdata.Journal} message Journal * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VStreamRowsRequest.toObject = function toObject(message, options) { + Journal.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.query = ""; - object.lastpk = null; + if (options.arrays || options.defaults) { + object.tables = []; + object.shard_gtids = []; + object.participants = []; + object.source_workflows = []; + } + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.id = options.longs === String ? "0" : 0; + object.migration_type = options.enums === String ? "TABLES" : 0; + object.local_position = ""; + } + if (message.id != null && message.hasOwnProperty("id")) + if (typeof message.id === "number") + object.id = options.longs === String ? String(message.id) : message.id; + else + object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; + if (message.migration_type != null && message.hasOwnProperty("migration_type")) + object.migration_type = options.enums === String ? $root.binlogdata.MigrationType[message.migration_type] === undefined ? message.migration_type : $root.binlogdata.MigrationType[message.migration_type] : message.migration_type; + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } + if (message.local_position != null && message.hasOwnProperty("local_position")) + object.local_position = message.local_position; + if (message.shard_gtids && message.shard_gtids.length) { + object.shard_gtids = []; + for (let j = 0; j < message.shard_gtids.length; ++j) + object.shard_gtids[j] = $root.binlogdata.ShardGtid.toObject(message.shard_gtids[j], options); + } + if (message.participants && message.participants.length) { + object.participants = []; + for (let j = 0; j < message.participants.length; ++j) + object.participants[j] = $root.binlogdata.KeyspaceShard.toObject(message.participants[j], options); + } + if (message.source_workflows && message.source_workflows.length) { + object.source_workflows = []; + for (let j = 0; j < message.source_workflows.length; ++j) + object.source_workflows[j] = message.source_workflows[j]; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = message.query; - if (message.lastpk != null && message.hasOwnProperty("lastpk")) - object.lastpk = $root.query.QueryResult.toObject(message.lastpk, options); return object; }; /** - * Converts this VStreamRowsRequest to JSON. + * Converts this Journal to JSON. * @function toJSON - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @instance * @returns {Object.} JSON object */ - VStreamRowsRequest.prototype.toJSON = function toJSON() { + Journal.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamRowsRequest + * Gets the default type url for Journal * @function getTypeUrl - * @memberof binlogdata.VStreamRowsRequest + * @memberof binlogdata.Journal * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamRowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Journal.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamRowsRequest"; + return typeUrlPrefix + "/binlogdata.Journal"; }; - return VStreamRowsRequest; + return Journal; })(); - binlogdata.VStreamRowsResponse = (function() { + binlogdata.VEvent = (function() { /** - * Properties of a VStreamRowsResponse. + * Properties of a VEvent. * @memberof binlogdata - * @interface IVStreamRowsResponse - * @property {Array.|null} [fields] VStreamRowsResponse fields - * @property {Array.|null} [pkfields] VStreamRowsResponse pkfields - * @property {string|null} [gtid] VStreamRowsResponse gtid - * @property {Array.|null} [rows] VStreamRowsResponse rows - * @property {query.IRow|null} [lastpk] VStreamRowsResponse lastpk - * @property {boolean|null} [throttled] VStreamRowsResponse throttled - * @property {boolean|null} [heartbeat] VStreamRowsResponse heartbeat + * @interface IVEvent + * @property {binlogdata.VEventType|null} [type] VEvent type + * @property {number|Long|null} [timestamp] VEvent timestamp + * @property {string|null} [gtid] VEvent gtid + * @property {string|null} [statement] VEvent statement + * @property {binlogdata.IRowEvent|null} [row_event] VEvent row_event + * @property {binlogdata.IFieldEvent|null} [field_event] VEvent field_event + * @property {binlogdata.IVGtid|null} [vgtid] VEvent vgtid + * @property {binlogdata.IJournal|null} [journal] VEvent journal + * @property {string|null} [dml] VEvent dml + * @property {number|Long|null} [current_time] VEvent current_time + * @property {binlogdata.ILastPKEvent|null} [last_p_k_event] VEvent last_p_k_event + * @property {string|null} [keyspace] VEvent keyspace + * @property {string|null} [shard] VEvent shard + * @property {boolean|null} [throttled] VEvent throttled */ /** - * Constructs a new VStreamRowsResponse. + * Constructs a new VEvent. * @memberof binlogdata - * @classdesc Represents a VStreamRowsResponse. - * @implements IVStreamRowsResponse + * @classdesc Represents a VEvent. + * @implements IVEvent * @constructor - * @param {binlogdata.IVStreamRowsResponse=} [properties] Properties to set + * @param {binlogdata.IVEvent=} [properties] Properties to set */ - function VStreamRowsResponse(properties) { - this.fields = []; - this.pkfields = []; - this.rows = []; + function VEvent(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -70923,146 +71712,209 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamRowsResponse fields. - * @member {Array.} fields - * @memberof binlogdata.VStreamRowsResponse + * VEvent type. + * @member {binlogdata.VEventType} type + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.fields = $util.emptyArray; + VEvent.prototype.type = 0; /** - * VStreamRowsResponse pkfields. - * @member {Array.} pkfields - * @memberof binlogdata.VStreamRowsResponse + * VEvent timestamp. + * @member {number|Long} timestamp + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.pkfields = $util.emptyArray; + VEvent.prototype.timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * VStreamRowsResponse gtid. + * VEvent gtid. * @member {string} gtid - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.gtid = ""; + VEvent.prototype.gtid = ""; /** - * VStreamRowsResponse rows. - * @member {Array.} rows - * @memberof binlogdata.VStreamRowsResponse + * VEvent statement. + * @member {string} statement + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.rows = $util.emptyArray; + VEvent.prototype.statement = ""; /** - * VStreamRowsResponse lastpk. - * @member {query.IRow|null|undefined} lastpk - * @memberof binlogdata.VStreamRowsResponse + * VEvent row_event. + * @member {binlogdata.IRowEvent|null|undefined} row_event + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.lastpk = null; + VEvent.prototype.row_event = null; /** - * VStreamRowsResponse throttled. - * @member {boolean} throttled - * @memberof binlogdata.VStreamRowsResponse + * VEvent field_event. + * @member {binlogdata.IFieldEvent|null|undefined} field_event + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.throttled = false; + VEvent.prototype.field_event = null; /** - * VStreamRowsResponse heartbeat. - * @member {boolean} heartbeat - * @memberof binlogdata.VStreamRowsResponse + * VEvent vgtid. + * @member {binlogdata.IVGtid|null|undefined} vgtid + * @memberof binlogdata.VEvent * @instance */ - VStreamRowsResponse.prototype.heartbeat = false; + VEvent.prototype.vgtid = null; /** - * Creates a new VStreamRowsResponse instance using the specified properties. + * VEvent journal. + * @member {binlogdata.IJournal|null|undefined} journal + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.journal = null; + + /** + * VEvent dml. + * @member {string} dml + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.dml = ""; + + /** + * VEvent current_time. + * @member {number|Long} current_time + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.current_time = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VEvent last_p_k_event. + * @member {binlogdata.ILastPKEvent|null|undefined} last_p_k_event + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.last_p_k_event = null; + + /** + * VEvent keyspace. + * @member {string} keyspace + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.keyspace = ""; + + /** + * VEvent shard. + * @member {string} shard + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.shard = ""; + + /** + * VEvent throttled. + * @member {boolean} throttled + * @memberof binlogdata.VEvent + * @instance + */ + VEvent.prototype.throttled = false; + + /** + * Creates a new VEvent instance using the specified properties. * @function create - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static - * @param {binlogdata.IVStreamRowsResponse=} [properties] Properties to set - * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse instance + * @param {binlogdata.IVEvent=} [properties] Properties to set + * @returns {binlogdata.VEvent} VEvent instance */ - VStreamRowsResponse.create = function create(properties) { - return new VStreamRowsResponse(properties); + VEvent.create = function create(properties) { + return new VEvent(properties); }; /** - * Encodes the specified VStreamRowsResponse message. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. + * Encodes the specified VEvent message. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static - * @param {binlogdata.IVStreamRowsResponse} message VStreamRowsResponse message or plain object to encode + * @param {binlogdata.IVEvent} message VEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRowsResponse.encode = function encode(message, writer) { + VEvent.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.fields != null && message.fields.length) - for (let i = 0; i < message.fields.length; ++i) - $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.pkfields != null && message.pkfields.length) - for (let i = 0; i < message.pkfields.length; ++i) - $root.query.Field.encode(message.pkfields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); + if (message.timestamp != null && Object.hasOwnProperty.call(message, "timestamp")) + writer.uint32(/* id 2, wireType 0 =*/16).int64(message.timestamp); if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); - if (message.rows != null && message.rows.length) - for (let i = 0; i < message.rows.length; ++i) - $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) - $root.query.Row.encode(message.lastpk, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.statement != null && Object.hasOwnProperty.call(message, "statement")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.statement); + if (message.row_event != null && Object.hasOwnProperty.call(message, "row_event")) + $root.binlogdata.RowEvent.encode(message.row_event, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.field_event != null && Object.hasOwnProperty.call(message, "field_event")) + $root.binlogdata.FieldEvent.encode(message.field_event, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.vgtid != null && Object.hasOwnProperty.call(message, "vgtid")) + $root.binlogdata.VGtid.encode(message.vgtid, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); + if (message.journal != null && Object.hasOwnProperty.call(message, "journal")) + $root.binlogdata.Journal.encode(message.journal, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); + if (message.dml != null && Object.hasOwnProperty.call(message, "dml")) + writer.uint32(/* id 9, wireType 2 =*/74).string(message.dml); + if (message.current_time != null && Object.hasOwnProperty.call(message, "current_time")) + writer.uint32(/* id 20, wireType 0 =*/160).int64(message.current_time); + if (message.last_p_k_event != null && Object.hasOwnProperty.call(message, "last_p_k_event")) + $root.binlogdata.LastPKEvent.encode(message.last_p_k_event, writer.uint32(/* id 21, wireType 2 =*/170).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 22, wireType 2 =*/178).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 23, wireType 2 =*/186).string(message.shard); if (message.throttled != null && Object.hasOwnProperty.call(message, "throttled")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.throttled); - if (message.heartbeat != null && Object.hasOwnProperty.call(message, "heartbeat")) - writer.uint32(/* id 7, wireType 0 =*/56).bool(message.heartbeat); + writer.uint32(/* id 24, wireType 0 =*/192).bool(message.throttled); return writer; }; /** - * Encodes the specified VStreamRowsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. + * Encodes the specified VEvent message, length delimited. Does not implicitly {@link binlogdata.VEvent.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static - * @param {binlogdata.IVStreamRowsResponse} message VStreamRowsResponse message or plain object to encode + * @param {binlogdata.IVEvent} message VEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamRowsResponse.encodeDelimited = function encodeDelimited(message, writer) { + VEvent.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamRowsResponse message from the specified reader or buffer. + * Decodes a VEvent message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse + * @returns {binlogdata.VEvent} VEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRowsResponse.decode = function decode(reader, length) { + VEvent.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRowsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VEvent(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.fields && message.fields.length)) - message.fields = []; - message.fields.push($root.query.Field.decode(reader, reader.uint32())); + message.type = reader.int32(); break; } case 2: { - if (!(message.pkfields && message.pkfields.length)) - message.pkfields = []; - message.pkfields.push($root.query.Field.decode(reader, reader.uint32())); + message.timestamp = reader.int64(); break; } case 3: { @@ -71070,21 +71922,47 @@ export const binlogdata = $root.binlogdata = (() => { break; } case 4: { - if (!(message.rows && message.rows.length)) - message.rows = []; - message.rows.push($root.query.Row.decode(reader, reader.uint32())); + message.statement = reader.string(); break; } case 5: { - message.lastpk = $root.query.Row.decode(reader, reader.uint32()); + message.row_event = $root.binlogdata.RowEvent.decode(reader, reader.uint32()); break; } case 6: { - message.throttled = reader.bool(); + message.field_event = $root.binlogdata.FieldEvent.decode(reader, reader.uint32()); break; } case 7: { - message.heartbeat = reader.bool(); + message.vgtid = $root.binlogdata.VGtid.decode(reader, reader.uint32()); + break; + } + case 8: { + message.journal = $root.binlogdata.Journal.decode(reader, reader.uint32()); + break; + } + case 9: { + message.dml = reader.string(); + break; + } + case 20: { + message.current_time = reader.int64(); + break; + } + case 21: { + message.last_p_k_event = $root.binlogdata.LastPKEvent.decode(reader, reader.uint32()); + break; + } + case 22: { + message.keyspace = reader.string(); + break; + } + case 23: { + message.shard = reader.string(); + break; + } + case 24: { + message.throttled = reader.bool(); break; } default: @@ -71096,230 +71974,398 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamRowsResponse message from the specified reader or buffer, length delimited. + * Decodes a VEvent message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse + * @returns {binlogdata.VEvent} VEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamRowsResponse.decodeDelimited = function decodeDelimited(reader) { + VEvent.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamRowsResponse message. + * Verifies a VEvent message. * @function verify - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamRowsResponse.verify = function verify(message) { + VEvent.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.fields != null && message.hasOwnProperty("fields")) { - if (!Array.isArray(message.fields)) - return "fields: array expected"; - for (let i = 0; i < message.fields.length; ++i) { - let error = $root.query.Field.verify(message.fields[i]); - if (error) - return "fields." + error; - } - } - if (message.pkfields != null && message.hasOwnProperty("pkfields")) { - if (!Array.isArray(message.pkfields)) - return "pkfields: array expected"; - for (let i = 0; i < message.pkfields.length; ++i) { - let error = $root.query.Field.verify(message.pkfields[i]); - if (error) - return "pkfields." + error; + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "type: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + case 9: + case 10: + case 11: + case 12: + case 13: + case 14: + case 15: + case 16: + case 17: + case 18: + case 19: + case 20: + break; } - } + if (message.timestamp != null && message.hasOwnProperty("timestamp")) + if (!$util.isInteger(message.timestamp) && !(message.timestamp && $util.isInteger(message.timestamp.low) && $util.isInteger(message.timestamp.high))) + return "timestamp: integer|Long expected"; if (message.gtid != null && message.hasOwnProperty("gtid")) if (!$util.isString(message.gtid)) return "gtid: string expected"; - if (message.rows != null && message.hasOwnProperty("rows")) { - if (!Array.isArray(message.rows)) - return "rows: array expected"; - for (let i = 0; i < message.rows.length; ++i) { - let error = $root.query.Row.verify(message.rows[i]); - if (error) - return "rows." + error; - } + if (message.statement != null && message.hasOwnProperty("statement")) + if (!$util.isString(message.statement)) + return "statement: string expected"; + if (message.row_event != null && message.hasOwnProperty("row_event")) { + let error = $root.binlogdata.RowEvent.verify(message.row_event); + if (error) + return "row_event." + error; } - if (message.lastpk != null && message.hasOwnProperty("lastpk")) { - let error = $root.query.Row.verify(message.lastpk); + if (message.field_event != null && message.hasOwnProperty("field_event")) { + let error = $root.binlogdata.FieldEvent.verify(message.field_event); if (error) - return "lastpk." + error; + return "field_event." + error; + } + if (message.vgtid != null && message.hasOwnProperty("vgtid")) { + let error = $root.binlogdata.VGtid.verify(message.vgtid); + if (error) + return "vgtid." + error; + } + if (message.journal != null && message.hasOwnProperty("journal")) { + let error = $root.binlogdata.Journal.verify(message.journal); + if (error) + return "journal." + error; + } + if (message.dml != null && message.hasOwnProperty("dml")) + if (!$util.isString(message.dml)) + return "dml: string expected"; + if (message.current_time != null && message.hasOwnProperty("current_time")) + if (!$util.isInteger(message.current_time) && !(message.current_time && $util.isInteger(message.current_time.low) && $util.isInteger(message.current_time.high))) + return "current_time: integer|Long expected"; + if (message.last_p_k_event != null && message.hasOwnProperty("last_p_k_event")) { + let error = $root.binlogdata.LastPKEvent.verify(message.last_p_k_event); + if (error) + return "last_p_k_event." + error; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; if (message.throttled != null && message.hasOwnProperty("throttled")) if (typeof message.throttled !== "boolean") return "throttled: boolean expected"; - if (message.heartbeat != null && message.hasOwnProperty("heartbeat")) - if (typeof message.heartbeat !== "boolean") - return "heartbeat: boolean expected"; return null; }; /** - * Creates a VStreamRowsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VEvent message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse + * @returns {binlogdata.VEvent} VEvent */ - VStreamRowsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamRowsResponse) + VEvent.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VEvent) return object; - let message = new $root.binlogdata.VStreamRowsResponse(); - if (object.fields) { - if (!Array.isArray(object.fields)) - throw TypeError(".binlogdata.VStreamRowsResponse.fields: array expected"); - message.fields = []; - for (let i = 0; i < object.fields.length; ++i) { - if (typeof object.fields[i] !== "object") - throw TypeError(".binlogdata.VStreamRowsResponse.fields: object expected"); - message.fields[i] = $root.query.Field.fromObject(object.fields[i]); - } - } - if (object.pkfields) { - if (!Array.isArray(object.pkfields)) - throw TypeError(".binlogdata.VStreamRowsResponse.pkfields: array expected"); - message.pkfields = []; - for (let i = 0; i < object.pkfields.length; ++i) { - if (typeof object.pkfields[i] !== "object") - throw TypeError(".binlogdata.VStreamRowsResponse.pkfields: object expected"); - message.pkfields[i] = $root.query.Field.fromObject(object.pkfields[i]); - } - } - if (object.gtid != null) - message.gtid = String(object.gtid); - if (object.rows) { - if (!Array.isArray(object.rows)) - throw TypeError(".binlogdata.VStreamRowsResponse.rows: array expected"); - message.rows = []; - for (let i = 0; i < object.rows.length; ++i) { - if (typeof object.rows[i] !== "object") - throw TypeError(".binlogdata.VStreamRowsResponse.rows: object expected"); - message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + let message = new $root.binlogdata.VEvent(); + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; } - } - if (object.lastpk != null) { - if (typeof object.lastpk !== "object") - throw TypeError(".binlogdata.VStreamRowsResponse.lastpk: object expected"); - message.lastpk = $root.query.Row.fromObject(object.lastpk); - } - if (object.throttled != null) - message.throttled = Boolean(object.throttled); - if (object.heartbeat != null) - message.heartbeat = Boolean(object.heartbeat); - return message; - }; - - /** - * Creates a plain object from a VStreamRowsResponse message. Also converts values to other types if specified. - * @function toObject - * @memberof binlogdata.VStreamRowsResponse - * @static - * @param {binlogdata.VStreamRowsResponse} message VStreamRowsResponse - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - VStreamRowsResponse.toObject = function toObject(message, options) { + break; + case "UNKNOWN": + case 0: + message.type = 0; + break; + case "GTID": + case 1: + message.type = 1; + break; + case "BEGIN": + case 2: + message.type = 2; + break; + case "COMMIT": + case 3: + message.type = 3; + break; + case "ROLLBACK": + case 4: + message.type = 4; + break; + case "DDL": + case 5: + message.type = 5; + break; + case "INSERT": + case 6: + message.type = 6; + break; + case "REPLACE": + case 7: + message.type = 7; + break; + case "UPDATE": + case 8: + message.type = 8; + break; + case "DELETE": + case 9: + message.type = 9; + break; + case "SET": + case 10: + message.type = 10; + break; + case "OTHER": + case 11: + message.type = 11; + break; + case "ROW": + case 12: + message.type = 12; + break; + case "FIELD": + case 13: + message.type = 13; + break; + case "HEARTBEAT": + case 14: + message.type = 14; + break; + case "VGTID": + case 15: + message.type = 15; + break; + case "JOURNAL": + case 16: + message.type = 16; + break; + case "VERSION": + case 17: + message.type = 17; + break; + case "LASTPK": + case 18: + message.type = 18; + break; + case "SAVEPOINT": + case 19: + message.type = 19; + break; + case "COPY_COMPLETED": + case 20: + message.type = 20; + break; + } + if (object.timestamp != null) + if ($util.Long) + (message.timestamp = $util.Long.fromValue(object.timestamp)).unsigned = false; + else if (typeof object.timestamp === "string") + message.timestamp = parseInt(object.timestamp, 10); + else if (typeof object.timestamp === "number") + message.timestamp = object.timestamp; + else if (typeof object.timestamp === "object") + message.timestamp = new $util.LongBits(object.timestamp.low >>> 0, object.timestamp.high >>> 0).toNumber(); + if (object.gtid != null) + message.gtid = String(object.gtid); + if (object.statement != null) + message.statement = String(object.statement); + if (object.row_event != null) { + if (typeof object.row_event !== "object") + throw TypeError(".binlogdata.VEvent.row_event: object expected"); + message.row_event = $root.binlogdata.RowEvent.fromObject(object.row_event); + } + if (object.field_event != null) { + if (typeof object.field_event !== "object") + throw TypeError(".binlogdata.VEvent.field_event: object expected"); + message.field_event = $root.binlogdata.FieldEvent.fromObject(object.field_event); + } + if (object.vgtid != null) { + if (typeof object.vgtid !== "object") + throw TypeError(".binlogdata.VEvent.vgtid: object expected"); + message.vgtid = $root.binlogdata.VGtid.fromObject(object.vgtid); + } + if (object.journal != null) { + if (typeof object.journal !== "object") + throw TypeError(".binlogdata.VEvent.journal: object expected"); + message.journal = $root.binlogdata.Journal.fromObject(object.journal); + } + if (object.dml != null) + message.dml = String(object.dml); + if (object.current_time != null) + if ($util.Long) + (message.current_time = $util.Long.fromValue(object.current_time)).unsigned = false; + else if (typeof object.current_time === "string") + message.current_time = parseInt(object.current_time, 10); + else if (typeof object.current_time === "number") + message.current_time = object.current_time; + else if (typeof object.current_time === "object") + message.current_time = new $util.LongBits(object.current_time.low >>> 0, object.current_time.high >>> 0).toNumber(); + if (object.last_p_k_event != null) { + if (typeof object.last_p_k_event !== "object") + throw TypeError(".binlogdata.VEvent.last_p_k_event: object expected"); + message.last_p_k_event = $root.binlogdata.LastPKEvent.fromObject(object.last_p_k_event); + } + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.throttled != null) + message.throttled = Boolean(object.throttled); + return message; + }; + + /** + * Creates a plain object from a VEvent message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.VEvent + * @static + * @param {binlogdata.VEvent} message VEvent + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VEvent.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.fields = []; - object.pkfields = []; - object.rows = []; - } if (options.defaults) { + object.type = options.enums === String ? "UNKNOWN" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.timestamp = options.longs === String ? "0" : 0; object.gtid = ""; - object.lastpk = null; + object.statement = ""; + object.row_event = null; + object.field_event = null; + object.vgtid = null; + object.journal = null; + object.dml = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.current_time = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.current_time = options.longs === String ? "0" : 0; + object.last_p_k_event = null; + object.keyspace = ""; + object.shard = ""; object.throttled = false; - object.heartbeat = false; - } - if (message.fields && message.fields.length) { - object.fields = []; - for (let j = 0; j < message.fields.length; ++j) - object.fields[j] = $root.query.Field.toObject(message.fields[j], options); - } - if (message.pkfields && message.pkfields.length) { - object.pkfields = []; - for (let j = 0; j < message.pkfields.length; ++j) - object.pkfields[j] = $root.query.Field.toObject(message.pkfields[j], options); } + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.binlogdata.VEventType[message.type] === undefined ? message.type : $root.binlogdata.VEventType[message.type] : message.type; + if (message.timestamp != null && message.hasOwnProperty("timestamp")) + if (typeof message.timestamp === "number") + object.timestamp = options.longs === String ? String(message.timestamp) : message.timestamp; + else + object.timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.timestamp) : options.longs === Number ? new $util.LongBits(message.timestamp.low >>> 0, message.timestamp.high >>> 0).toNumber() : message.timestamp; if (message.gtid != null && message.hasOwnProperty("gtid")) object.gtid = message.gtid; - if (message.rows && message.rows.length) { - object.rows = []; - for (let j = 0; j < message.rows.length; ++j) - object.rows[j] = $root.query.Row.toObject(message.rows[j], options); - } - if (message.lastpk != null && message.hasOwnProperty("lastpk")) - object.lastpk = $root.query.Row.toObject(message.lastpk, options); + if (message.statement != null && message.hasOwnProperty("statement")) + object.statement = message.statement; + if (message.row_event != null && message.hasOwnProperty("row_event")) + object.row_event = $root.binlogdata.RowEvent.toObject(message.row_event, options); + if (message.field_event != null && message.hasOwnProperty("field_event")) + object.field_event = $root.binlogdata.FieldEvent.toObject(message.field_event, options); + if (message.vgtid != null && message.hasOwnProperty("vgtid")) + object.vgtid = $root.binlogdata.VGtid.toObject(message.vgtid, options); + if (message.journal != null && message.hasOwnProperty("journal")) + object.journal = $root.binlogdata.Journal.toObject(message.journal, options); + if (message.dml != null && message.hasOwnProperty("dml")) + object.dml = message.dml; + if (message.current_time != null && message.hasOwnProperty("current_time")) + if (typeof message.current_time === "number") + object.current_time = options.longs === String ? String(message.current_time) : message.current_time; + else + object.current_time = options.longs === String ? $util.Long.prototype.toString.call(message.current_time) : options.longs === Number ? new $util.LongBits(message.current_time.low >>> 0, message.current_time.high >>> 0).toNumber() : message.current_time; + if (message.last_p_k_event != null && message.hasOwnProperty("last_p_k_event")) + object.last_p_k_event = $root.binlogdata.LastPKEvent.toObject(message.last_p_k_event, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; if (message.throttled != null && message.hasOwnProperty("throttled")) object.throttled = message.throttled; - if (message.heartbeat != null && message.hasOwnProperty("heartbeat")) - object.heartbeat = message.heartbeat; return object; }; /** - * Converts this VStreamRowsResponse to JSON. + * Converts this VEvent to JSON. * @function toJSON - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @instance * @returns {Object.} JSON object */ - VStreamRowsResponse.prototype.toJSON = function toJSON() { + VEvent.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamRowsResponse + * Gets the default type url for VEvent * @function getTypeUrl - * @memberof binlogdata.VStreamRowsResponse + * @memberof binlogdata.VEvent * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamRowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamRowsResponse"; + return typeUrlPrefix + "/binlogdata.VEvent"; }; - return VStreamRowsResponse; + return VEvent; })(); - binlogdata.LastPKEvent = (function() { + binlogdata.MinimalTable = (function() { /** - * Properties of a LastPKEvent. + * Properties of a MinimalTable. * @memberof binlogdata - * @interface ILastPKEvent - * @property {binlogdata.ITableLastPK|null} [table_last_p_k] LastPKEvent table_last_p_k - * @property {boolean|null} [completed] LastPKEvent completed + * @interface IMinimalTable + * @property {string|null} [name] MinimalTable name + * @property {Array.|null} [fields] MinimalTable fields + * @property {Array.|null} [p_k_columns] MinimalTable p_k_columns */ /** - * Constructs a new LastPKEvent. + * Constructs a new MinimalTable. * @memberof binlogdata - * @classdesc Represents a LastPKEvent. - * @implements ILastPKEvent + * @classdesc Represents a MinimalTable. + * @implements IMinimalTable * @constructor - * @param {binlogdata.ILastPKEvent=} [properties] Properties to set + * @param {binlogdata.IMinimalTable=} [properties] Properties to set */ - function LastPKEvent(properties) { + function MinimalTable(properties) { + this.fields = []; + this.p_k_columns = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -71327,89 +72373,117 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * LastPKEvent table_last_p_k. - * @member {binlogdata.ITableLastPK|null|undefined} table_last_p_k - * @memberof binlogdata.LastPKEvent + * MinimalTable name. + * @member {string} name + * @memberof binlogdata.MinimalTable * @instance */ - LastPKEvent.prototype.table_last_p_k = null; + MinimalTable.prototype.name = ""; /** - * LastPKEvent completed. - * @member {boolean} completed - * @memberof binlogdata.LastPKEvent + * MinimalTable fields. + * @member {Array.} fields + * @memberof binlogdata.MinimalTable * @instance */ - LastPKEvent.prototype.completed = false; + MinimalTable.prototype.fields = $util.emptyArray; /** - * Creates a new LastPKEvent instance using the specified properties. + * MinimalTable p_k_columns. + * @member {Array.} p_k_columns + * @memberof binlogdata.MinimalTable + * @instance + */ + MinimalTable.prototype.p_k_columns = $util.emptyArray; + + /** + * Creates a new MinimalTable instance using the specified properties. * @function create - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static - * @param {binlogdata.ILastPKEvent=} [properties] Properties to set - * @returns {binlogdata.LastPKEvent} LastPKEvent instance + * @param {binlogdata.IMinimalTable=} [properties] Properties to set + * @returns {binlogdata.MinimalTable} MinimalTable instance */ - LastPKEvent.create = function create(properties) { - return new LastPKEvent(properties); + MinimalTable.create = function create(properties) { + return new MinimalTable(properties); }; /** - * Encodes the specified LastPKEvent message. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. + * Encodes the specified MinimalTable message. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. * @function encode - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static - * @param {binlogdata.ILastPKEvent} message LastPKEvent message or plain object to encode + * @param {binlogdata.IMinimalTable} message MinimalTable message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - LastPKEvent.encode = function encode(message, writer) { + MinimalTable.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.table_last_p_k != null && Object.hasOwnProperty.call(message, "table_last_p_k")) - $root.binlogdata.TableLastPK.encode(message.table_last_p_k, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.completed != null && Object.hasOwnProperty.call(message, "completed")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.completed); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.fields != null && message.fields.length) + for (let i = 0; i < message.fields.length; ++i) + $root.query.Field.encode(message.fields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.p_k_columns != null && message.p_k_columns.length) { + writer.uint32(/* id 3, wireType 2 =*/26).fork(); + for (let i = 0; i < message.p_k_columns.length; ++i) + writer.int64(message.p_k_columns[i]); + writer.ldelim(); + } return writer; }; /** - * Encodes the specified LastPKEvent message, length delimited. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. + * Encodes the specified MinimalTable message, length delimited. Does not implicitly {@link binlogdata.MinimalTable.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static - * @param {binlogdata.ILastPKEvent} message LastPKEvent message or plain object to encode + * @param {binlogdata.IMinimalTable} message MinimalTable message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - LastPKEvent.encodeDelimited = function encodeDelimited(message, writer) { + MinimalTable.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a LastPKEvent message from the specified reader or buffer. + * Decodes a MinimalTable message from the specified reader or buffer. * @function decode - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.LastPKEvent} LastPKEvent + * @returns {binlogdata.MinimalTable} MinimalTable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - LastPKEvent.decode = function decode(reader, length) { + MinimalTable.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.LastPKEvent(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.MinimalTable(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.table_last_p_k = $root.binlogdata.TableLastPK.decode(reader, reader.uint32()); + message.name = reader.string(); break; } case 2: { - message.completed = reader.bool(); + if (!(message.fields && message.fields.length)) + message.fields = []; + message.fields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } + case 3: { + if (!(message.p_k_columns && message.p_k_columns.length)) + message.p_k_columns = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.p_k_columns.push(reader.int64()); + } else + message.p_k_columns.push(reader.int64()); break; } default: @@ -71421,137 +72495,180 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a LastPKEvent message from the specified reader or buffer, length delimited. + * Decodes a MinimalTable message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.LastPKEvent} LastPKEvent + * @returns {binlogdata.MinimalTable} MinimalTable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - LastPKEvent.decodeDelimited = function decodeDelimited(reader) { + MinimalTable.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a LastPKEvent message. + * Verifies a MinimalTable message. * @function verify - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - LastPKEvent.verify = function verify(message) { + MinimalTable.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.table_last_p_k != null && message.hasOwnProperty("table_last_p_k")) { - let error = $root.binlogdata.TableLastPK.verify(message.table_last_p_k); - if (error) - return "table_last_p_k." + error; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.fields != null && message.hasOwnProperty("fields")) { + if (!Array.isArray(message.fields)) + return "fields: array expected"; + for (let i = 0; i < message.fields.length; ++i) { + let error = $root.query.Field.verify(message.fields[i]); + if (error) + return "fields." + error; + } + } + if (message.p_k_columns != null && message.hasOwnProperty("p_k_columns")) { + if (!Array.isArray(message.p_k_columns)) + return "p_k_columns: array expected"; + for (let i = 0; i < message.p_k_columns.length; ++i) + if (!$util.isInteger(message.p_k_columns[i]) && !(message.p_k_columns[i] && $util.isInteger(message.p_k_columns[i].low) && $util.isInteger(message.p_k_columns[i].high))) + return "p_k_columns: integer|Long[] expected"; } - if (message.completed != null && message.hasOwnProperty("completed")) - if (typeof message.completed !== "boolean") - return "completed: boolean expected"; return null; }; /** - * Creates a LastPKEvent message from a plain object. Also converts values to their respective internal types. + * Creates a MinimalTable message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static * @param {Object.} object Plain object - * @returns {binlogdata.LastPKEvent} LastPKEvent + * @returns {binlogdata.MinimalTable} MinimalTable */ - LastPKEvent.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.LastPKEvent) + MinimalTable.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.MinimalTable) return object; - let message = new $root.binlogdata.LastPKEvent(); - if (object.table_last_p_k != null) { - if (typeof object.table_last_p_k !== "object") - throw TypeError(".binlogdata.LastPKEvent.table_last_p_k: object expected"); - message.table_last_p_k = $root.binlogdata.TableLastPK.fromObject(object.table_last_p_k); + let message = new $root.binlogdata.MinimalTable(); + if (object.name != null) + message.name = String(object.name); + if (object.fields) { + if (!Array.isArray(object.fields)) + throw TypeError(".binlogdata.MinimalTable.fields: array expected"); + message.fields = []; + for (let i = 0; i < object.fields.length; ++i) { + if (typeof object.fields[i] !== "object") + throw TypeError(".binlogdata.MinimalTable.fields: object expected"); + message.fields[i] = $root.query.Field.fromObject(object.fields[i]); + } + } + if (object.p_k_columns) { + if (!Array.isArray(object.p_k_columns)) + throw TypeError(".binlogdata.MinimalTable.p_k_columns: array expected"); + message.p_k_columns = []; + for (let i = 0; i < object.p_k_columns.length; ++i) + if ($util.Long) + (message.p_k_columns[i] = $util.Long.fromValue(object.p_k_columns[i])).unsigned = false; + else if (typeof object.p_k_columns[i] === "string") + message.p_k_columns[i] = parseInt(object.p_k_columns[i], 10); + else if (typeof object.p_k_columns[i] === "number") + message.p_k_columns[i] = object.p_k_columns[i]; + else if (typeof object.p_k_columns[i] === "object") + message.p_k_columns[i] = new $util.LongBits(object.p_k_columns[i].low >>> 0, object.p_k_columns[i].high >>> 0).toNumber(); } - if (object.completed != null) - message.completed = Boolean(object.completed); return message; }; /** - * Creates a plain object from a LastPKEvent message. Also converts values to other types if specified. + * Creates a plain object from a MinimalTable message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static - * @param {binlogdata.LastPKEvent} message LastPKEvent + * @param {binlogdata.MinimalTable} message MinimalTable * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - LastPKEvent.toObject = function toObject(message, options) { + MinimalTable.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.table_last_p_k = null; - object.completed = false; + if (options.arrays || options.defaults) { + object.fields = []; + object.p_k_columns = []; + } + if (options.defaults) + object.name = ""; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.fields && message.fields.length) { + object.fields = []; + for (let j = 0; j < message.fields.length; ++j) + object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + } + if (message.p_k_columns && message.p_k_columns.length) { + object.p_k_columns = []; + for (let j = 0; j < message.p_k_columns.length; ++j) + if (typeof message.p_k_columns[j] === "number") + object.p_k_columns[j] = options.longs === String ? String(message.p_k_columns[j]) : message.p_k_columns[j]; + else + object.p_k_columns[j] = options.longs === String ? $util.Long.prototype.toString.call(message.p_k_columns[j]) : options.longs === Number ? new $util.LongBits(message.p_k_columns[j].low >>> 0, message.p_k_columns[j].high >>> 0).toNumber() : message.p_k_columns[j]; } - if (message.table_last_p_k != null && message.hasOwnProperty("table_last_p_k")) - object.table_last_p_k = $root.binlogdata.TableLastPK.toObject(message.table_last_p_k, options); - if (message.completed != null && message.hasOwnProperty("completed")) - object.completed = message.completed; return object; }; /** - * Converts this LastPKEvent to JSON. + * Converts this MinimalTable to JSON. * @function toJSON - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @instance * @returns {Object.} JSON object */ - LastPKEvent.prototype.toJSON = function toJSON() { + MinimalTable.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for LastPKEvent + * Gets the default type url for MinimalTable * @function getTypeUrl - * @memberof binlogdata.LastPKEvent + * @memberof binlogdata.MinimalTable * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - LastPKEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MinimalTable.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.LastPKEvent"; + return typeUrlPrefix + "/binlogdata.MinimalTable"; }; - return LastPKEvent; + return MinimalTable; })(); - binlogdata.TableLastPK = (function() { + binlogdata.MinimalSchema = (function() { /** - * Properties of a TableLastPK. + * Properties of a MinimalSchema. * @memberof binlogdata - * @interface ITableLastPK - * @property {string|null} [table_name] TableLastPK table_name - * @property {query.IQueryResult|null} [lastpk] TableLastPK lastpk + * @interface IMinimalSchema + * @property {Array.|null} [tables] MinimalSchema tables */ /** - * Constructs a new TableLastPK. + * Constructs a new MinimalSchema. * @memberof binlogdata - * @classdesc Represents a TableLastPK. - * @implements ITableLastPK + * @classdesc Represents a MinimalSchema. + * @implements IMinimalSchema * @constructor - * @param {binlogdata.ITableLastPK=} [properties] Properties to set + * @param {binlogdata.IMinimalSchema=} [properties] Properties to set */ - function TableLastPK(properties) { + function MinimalSchema(properties) { + this.tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -71559,89 +72676,78 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * TableLastPK table_name. - * @member {string} table_name - * @memberof binlogdata.TableLastPK - * @instance - */ - TableLastPK.prototype.table_name = ""; - - /** - * TableLastPK lastpk. - * @member {query.IQueryResult|null|undefined} lastpk - * @memberof binlogdata.TableLastPK + * MinimalSchema tables. + * @member {Array.} tables + * @memberof binlogdata.MinimalSchema * @instance */ - TableLastPK.prototype.lastpk = null; + MinimalSchema.prototype.tables = $util.emptyArray; /** - * Creates a new TableLastPK instance using the specified properties. + * Creates a new MinimalSchema instance using the specified properties. * @function create - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static - * @param {binlogdata.ITableLastPK=} [properties] Properties to set - * @returns {binlogdata.TableLastPK} TableLastPK instance + * @param {binlogdata.IMinimalSchema=} [properties] Properties to set + * @returns {binlogdata.MinimalSchema} MinimalSchema instance */ - TableLastPK.create = function create(properties) { - return new TableLastPK(properties); + MinimalSchema.create = function create(properties) { + return new MinimalSchema(properties); }; /** - * Encodes the specified TableLastPK message. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. + * Encodes the specified MinimalSchema message. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. * @function encode - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static - * @param {binlogdata.ITableLastPK} message TableLastPK message or plain object to encode + * @param {binlogdata.IMinimalSchema} message MinimalSchema message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TableLastPK.encode = function encode(message, writer) { + MinimalSchema.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); - if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) - $root.query.QueryResult.encode(message.lastpk, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + $root.binlogdata.MinimalTable.encode(message.tables[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified TableLastPK message, length delimited. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. + * Encodes the specified MinimalSchema message, length delimited. Does not implicitly {@link binlogdata.MinimalSchema.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static - * @param {binlogdata.ITableLastPK} message TableLastPK message or plain object to encode + * @param {binlogdata.IMinimalSchema} message MinimalSchema message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TableLastPK.encodeDelimited = function encodeDelimited(message, writer) { + MinimalSchema.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TableLastPK message from the specified reader or buffer. + * Decodes a MinimalSchema message from the specified reader or buffer. * @function decode - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.TableLastPK} TableLastPK + * @returns {binlogdata.MinimalSchema} MinimalSchema * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TableLastPK.decode = function decode(reader, length) { + MinimalSchema.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.TableLastPK(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.MinimalSchema(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.table_name = reader.string(); - break; - } - case 3: { - message.lastpk = $root.query.QueryResult.decode(reader, reader.uint32()); + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push($root.binlogdata.MinimalTable.decode(reader, reader.uint32())); break; } default: @@ -71653,139 +72759,145 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a TableLastPK message from the specified reader or buffer, length delimited. + * Decodes a MinimalSchema message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.TableLastPK} TableLastPK + * @returns {binlogdata.MinimalSchema} MinimalSchema * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TableLastPK.decodeDelimited = function decodeDelimited(reader) { + MinimalSchema.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TableLastPK message. + * Verifies a MinimalSchema message. * @function verify - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TableLastPK.verify = function verify(message) { + MinimalSchema.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.table_name != null && message.hasOwnProperty("table_name")) - if (!$util.isString(message.table_name)) - return "table_name: string expected"; - if (message.lastpk != null && message.hasOwnProperty("lastpk")) { - let error = $root.query.QueryResult.verify(message.lastpk); - if (error) - return "lastpk." + error; + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) { + let error = $root.binlogdata.MinimalTable.verify(message.tables[i]); + if (error) + return "tables." + error; + } } return null; }; /** - * Creates a TableLastPK message from a plain object. Also converts values to their respective internal types. + * Creates a MinimalSchema message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static * @param {Object.} object Plain object - * @returns {binlogdata.TableLastPK} TableLastPK + * @returns {binlogdata.MinimalSchema} MinimalSchema */ - TableLastPK.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.TableLastPK) + MinimalSchema.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.MinimalSchema) return object; - let message = new $root.binlogdata.TableLastPK(); - if (object.table_name != null) - message.table_name = String(object.table_name); - if (object.lastpk != null) { - if (typeof object.lastpk !== "object") - throw TypeError(".binlogdata.TableLastPK.lastpk: object expected"); - message.lastpk = $root.query.QueryResult.fromObject(object.lastpk); + let message = new $root.binlogdata.MinimalSchema(); + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".binlogdata.MinimalSchema.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) { + if (typeof object.tables[i] !== "object") + throw TypeError(".binlogdata.MinimalSchema.tables: object expected"); + message.tables[i] = $root.binlogdata.MinimalTable.fromObject(object.tables[i]); + } } return message; }; /** - * Creates a plain object from a TableLastPK message. Also converts values to other types if specified. + * Creates a plain object from a MinimalSchema message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static - * @param {binlogdata.TableLastPK} message TableLastPK + * @param {binlogdata.MinimalSchema} message MinimalSchema * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TableLastPK.toObject = function toObject(message, options) { + MinimalSchema.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.table_name = ""; - object.lastpk = null; + if (options.arrays || options.defaults) + object.tables = []; + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = $root.binlogdata.MinimalTable.toObject(message.tables[j], options); } - if (message.table_name != null && message.hasOwnProperty("table_name")) - object.table_name = message.table_name; - if (message.lastpk != null && message.hasOwnProperty("lastpk")) - object.lastpk = $root.query.QueryResult.toObject(message.lastpk, options); return object; }; /** - * Converts this TableLastPK to JSON. + * Converts this MinimalSchema to JSON. * @function toJSON - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @instance * @returns {Object.} JSON object */ - TableLastPK.prototype.toJSON = function toJSON() { + MinimalSchema.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TableLastPK + * Gets the default type url for MinimalSchema * @function getTypeUrl - * @memberof binlogdata.TableLastPK + * @memberof binlogdata.MinimalSchema * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TableLastPK.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MinimalSchema.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.TableLastPK"; + return typeUrlPrefix + "/binlogdata.MinimalSchema"; }; - return TableLastPK; + return MinimalSchema; })(); - binlogdata.VStreamResultsRequest = (function() { + binlogdata.VStreamRequest = (function() { /** - * Properties of a VStreamResultsRequest. + * Properties of a VStreamRequest. * @memberof binlogdata - * @interface IVStreamResultsRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamResultsRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamResultsRequest immediate_caller_id - * @property {query.ITarget|null} [target] VStreamResultsRequest target - * @property {string|null} [query] VStreamResultsRequest query + * @interface IVStreamRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamRequest immediate_caller_id + * @property {query.ITarget|null} [target] VStreamRequest target + * @property {string|null} [position] VStreamRequest position + * @property {binlogdata.IFilter|null} [filter] VStreamRequest filter + * @property {Array.|null} [table_last_p_ks] VStreamRequest table_last_p_ks */ /** - * Constructs a new VStreamResultsRequest. + * Constructs a new VStreamRequest. * @memberof binlogdata - * @classdesc Represents a VStreamResultsRequest. - * @implements IVStreamResultsRequest + * @classdesc Represents a VStreamRequest. + * @implements IVStreamRequest * @constructor - * @param {binlogdata.IVStreamResultsRequest=} [properties] Properties to set + * @param {binlogdata.IVStreamRequest=} [properties] Properties to set */ - function VStreamResultsRequest(properties) { + function VStreamRequest(properties) { + this.table_last_p_ks = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -71793,59 +72905,75 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamResultsRequest effective_caller_id. + * VStreamRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @instance */ - VStreamResultsRequest.prototype.effective_caller_id = null; + VStreamRequest.prototype.effective_caller_id = null; /** - * VStreamResultsRequest immediate_caller_id. + * VStreamRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @instance */ - VStreamResultsRequest.prototype.immediate_caller_id = null; + VStreamRequest.prototype.immediate_caller_id = null; /** - * VStreamResultsRequest target. + * VStreamRequest target. * @member {query.ITarget|null|undefined} target - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @instance */ - VStreamResultsRequest.prototype.target = null; + VStreamRequest.prototype.target = null; /** - * VStreamResultsRequest query. - * @member {string} query - * @memberof binlogdata.VStreamResultsRequest + * VStreamRequest position. + * @member {string} position + * @memberof binlogdata.VStreamRequest * @instance */ - VStreamResultsRequest.prototype.query = ""; + VStreamRequest.prototype.position = ""; /** - * Creates a new VStreamResultsRequest instance using the specified properties. + * VStreamRequest filter. + * @member {binlogdata.IFilter|null|undefined} filter + * @memberof binlogdata.VStreamRequest + * @instance + */ + VStreamRequest.prototype.filter = null; + + /** + * VStreamRequest table_last_p_ks. + * @member {Array.} table_last_p_ks + * @memberof binlogdata.VStreamRequest + * @instance + */ + VStreamRequest.prototype.table_last_p_ks = $util.emptyArray; + + /** + * Creates a new VStreamRequest instance using the specified properties. * @function create - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static - * @param {binlogdata.IVStreamResultsRequest=} [properties] Properties to set - * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest instance + * @param {binlogdata.IVStreamRequest=} [properties] Properties to set + * @returns {binlogdata.VStreamRequest} VStreamRequest instance */ - VStreamResultsRequest.create = function create(properties) { - return new VStreamResultsRequest(properties); + VStreamRequest.create = function create(properties) { + return new VStreamRequest(properties); }; /** - * Encodes the specified VStreamResultsRequest message. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. + * Encodes the specified VStreamRequest message. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static - * @param {binlogdata.IVStreamResultsRequest} message VStreamResultsRequest message or plain object to encode + * @param {binlogdata.IVStreamRequest} message VStreamRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResultsRequest.encode = function encode(message, writer) { + VStreamRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -71854,39 +72982,44 @@ export const binlogdata = $root.binlogdata = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.query); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.position); + if (message.filter != null && Object.hasOwnProperty.call(message, "filter")) + $root.binlogdata.Filter.encode(message.filter, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.table_last_p_ks != null && message.table_last_p_ks.length) + for (let i = 0; i < message.table_last_p_ks.length; ++i) + $root.binlogdata.TableLastPK.encode(message.table_last_p_ks[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); return writer; }; /** - * Encodes the specified VStreamResultsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. + * Encodes the specified VStreamRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRequest.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static - * @param {binlogdata.IVStreamResultsRequest} message VStreamResultsRequest message or plain object to encode + * @param {binlogdata.IVStreamRequest} message VStreamRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResultsRequest.encodeDelimited = function encodeDelimited(message, writer) { + VStreamRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamResultsRequest message from the specified reader or buffer. + * Decodes a VStreamRequest message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest + * @returns {binlogdata.VStreamRequest} VStreamRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResultsRequest.decode = function decode(reader, length) { + VStreamRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResultsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -71903,7 +73036,17 @@ export const binlogdata = $root.binlogdata = (() => { break; } case 4: { - message.query = reader.string(); + message.position = reader.string(); + break; + } + case 5: { + message.filter = $root.binlogdata.Filter.decode(reader, reader.uint32()); + break; + } + case 6: { + if (!(message.table_last_p_ks && message.table_last_p_ks.length)) + message.table_last_p_ks = []; + message.table_last_p_ks.push($root.binlogdata.TableLastPK.decode(reader, reader.uint32())); break; } default: @@ -71915,30 +73058,30 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamResultsRequest message from the specified reader or buffer, length delimited. + * Decodes a VStreamRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest + * @returns {binlogdata.VStreamRequest} VStreamRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResultsRequest.decodeDelimited = function decodeDelimited(reader) { + VStreamRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamResultsRequest message. + * Verifies a VStreamRequest message. * @function verify - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamResultsRequest.verify = function verify(message) { + VStreamRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -71956,62 +73099,94 @@ export const binlogdata = $root.binlogdata = (() => { if (error) return "target." + error; } - if (message.query != null && message.hasOwnProperty("query")) - if (!$util.isString(message.query)) - return "query: string expected"; + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.filter != null && message.hasOwnProperty("filter")) { + let error = $root.binlogdata.Filter.verify(message.filter); + if (error) + return "filter." + error; + } + if (message.table_last_p_ks != null && message.hasOwnProperty("table_last_p_ks")) { + if (!Array.isArray(message.table_last_p_ks)) + return "table_last_p_ks: array expected"; + for (let i = 0; i < message.table_last_p_ks.length; ++i) { + let error = $root.binlogdata.TableLastPK.verify(message.table_last_p_ks[i]); + if (error) + return "table_last_p_ks." + error; + } + } return null; }; /** - * Creates a VStreamResultsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest + * @returns {binlogdata.VStreamRequest} VStreamRequest */ - VStreamResultsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamResultsRequest) + VStreamRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamRequest) return object; - let message = new $root.binlogdata.VStreamResultsRequest(); + let message = new $root.binlogdata.VStreamRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".binlogdata.VStreamResultsRequest.effective_caller_id: object expected"); + throw TypeError(".binlogdata.VStreamRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".binlogdata.VStreamResultsRequest.immediate_caller_id: object expected"); + throw TypeError(".binlogdata.VStreamRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".binlogdata.VStreamResultsRequest.target: object expected"); + throw TypeError(".binlogdata.VStreamRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.query != null) - message.query = String(object.query); + if (object.position != null) + message.position = String(object.position); + if (object.filter != null) { + if (typeof object.filter !== "object") + throw TypeError(".binlogdata.VStreamRequest.filter: object expected"); + message.filter = $root.binlogdata.Filter.fromObject(object.filter); + } + if (object.table_last_p_ks) { + if (!Array.isArray(object.table_last_p_ks)) + throw TypeError(".binlogdata.VStreamRequest.table_last_p_ks: array expected"); + message.table_last_p_ks = []; + for (let i = 0; i < object.table_last_p_ks.length; ++i) { + if (typeof object.table_last_p_ks[i] !== "object") + throw TypeError(".binlogdata.VStreamRequest.table_last_p_ks: object expected"); + message.table_last_p_ks[i] = $root.binlogdata.TableLastPK.fromObject(object.table_last_p_ks[i]); + } + } return message; }; /** - * Creates a plain object from a VStreamResultsRequest message. Also converts values to other types if specified. + * Creates a plain object from a VStreamRequest message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static - * @param {binlogdata.VStreamResultsRequest} message VStreamResultsRequest + * @param {binlogdata.VStreamRequest} message VStreamRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VStreamResultsRequest.toObject = function toObject(message, options) { + VStreamRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.table_last_p_ks = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.query = ""; + object.position = ""; + object.filter = null; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -72019,62 +73194,66 @@ export const binlogdata = $root.binlogdata = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = message.query; + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.filter != null && message.hasOwnProperty("filter")) + object.filter = $root.binlogdata.Filter.toObject(message.filter, options); + if (message.table_last_p_ks && message.table_last_p_ks.length) { + object.table_last_p_ks = []; + for (let j = 0; j < message.table_last_p_ks.length; ++j) + object.table_last_p_ks[j] = $root.binlogdata.TableLastPK.toObject(message.table_last_p_ks[j], options); + } return object; }; /** - * Converts this VStreamResultsRequest to JSON. + * Converts this VStreamRequest to JSON. * @function toJSON - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @instance * @returns {Object.} JSON object */ - VStreamResultsRequest.prototype.toJSON = function toJSON() { + VStreamRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamResultsRequest + * Gets the default type url for VStreamRequest * @function getTypeUrl - * @memberof binlogdata.VStreamResultsRequest + * @memberof binlogdata.VStreamRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamResultsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamResultsRequest"; + return typeUrlPrefix + "/binlogdata.VStreamRequest"; }; - return VStreamResultsRequest; + return VStreamRequest; })(); - binlogdata.VStreamResultsResponse = (function() { + binlogdata.VStreamResponse = (function() { /** - * Properties of a VStreamResultsResponse. + * Properties of a VStreamResponse. * @memberof binlogdata - * @interface IVStreamResultsResponse - * @property {Array.|null} [fields] VStreamResultsResponse fields - * @property {string|null} [gtid] VStreamResultsResponse gtid - * @property {Array.|null} [rows] VStreamResultsResponse rows + * @interface IVStreamResponse + * @property {Array.|null} [events] VStreamResponse events */ /** - * Constructs a new VStreamResultsResponse. + * Constructs a new VStreamResponse. * @memberof binlogdata - * @classdesc Represents a VStreamResultsResponse. - * @implements IVStreamResultsResponse + * @classdesc Represents a VStreamResponse. + * @implements IVStreamResponse * @constructor - * @param {binlogdata.IVStreamResultsResponse=} [properties] Properties to set + * @param {binlogdata.IVStreamResponse=} [properties] Properties to set */ - function VStreamResultsResponse(properties) { - this.fields = []; - this.rows = []; + function VStreamResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -72082,109 +73261,78 @@ export const binlogdata = $root.binlogdata = (() => { } /** - * VStreamResultsResponse fields. - * @member {Array.} fields - * @memberof binlogdata.VStreamResultsResponse - * @instance - */ - VStreamResultsResponse.prototype.fields = $util.emptyArray; - - /** - * VStreamResultsResponse gtid. - * @member {string} gtid - * @memberof binlogdata.VStreamResultsResponse - * @instance - */ - VStreamResultsResponse.prototype.gtid = ""; - - /** - * VStreamResultsResponse rows. - * @member {Array.} rows - * @memberof binlogdata.VStreamResultsResponse + * VStreamResponse events. + * @member {Array.} events + * @memberof binlogdata.VStreamResponse * @instance */ - VStreamResultsResponse.prototype.rows = $util.emptyArray; + VStreamResponse.prototype.events = $util.emptyArray; /** - * Creates a new VStreamResultsResponse instance using the specified properties. + * Creates a new VStreamResponse instance using the specified properties. * @function create - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static - * @param {binlogdata.IVStreamResultsResponse=} [properties] Properties to set - * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse instance + * @param {binlogdata.IVStreamResponse=} [properties] Properties to set + * @returns {binlogdata.VStreamResponse} VStreamResponse instance */ - VStreamResultsResponse.create = function create(properties) { - return new VStreamResultsResponse(properties); + VStreamResponse.create = function create(properties) { + return new VStreamResponse(properties); }; /** - * Encodes the specified VStreamResultsResponse message. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. + * Encodes the specified VStreamResponse message. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. * @function encode - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static - * @param {binlogdata.IVStreamResultsResponse} message VStreamResultsResponse message or plain object to encode + * @param {binlogdata.IVStreamResponse} message VStreamResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResultsResponse.encode = function encode(message, writer) { + VStreamResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.fields != null && message.fields.length) - for (let i = 0; i < message.fields.length; ++i) - $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); - if (message.rows != null && message.rows.length) - for (let i = 0; i < message.rows.length; ++i) - $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.binlogdata.VEvent.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified VStreamResultsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. + * Encodes the specified VStreamResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResponse.verify|verify} messages. * @function encodeDelimited - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static - * @param {binlogdata.IVStreamResultsResponse} message VStreamResultsResponse message or plain object to encode + * @param {binlogdata.IVStreamResponse} message VStreamResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VStreamResultsResponse.encodeDelimited = function encodeDelimited(message, writer) { + VStreamResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VStreamResultsResponse message from the specified reader or buffer. + * Decodes a VStreamResponse message from the specified reader or buffer. * @function decode - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse + * @returns {binlogdata.VStreamResponse} VStreamResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResultsResponse.decode = function decode(reader, length) { + VStreamResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResultsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.fields && message.fields.length)) - message.fields = []; - message.fields.push($root.query.Field.decode(reader, reader.uint32())); - break; - } - case 3: { - message.gtid = reader.string(); - break; - } - case 4: { - if (!(message.rows && message.rows.length)) - message.rows = []; - message.rows.push($root.query.Row.decode(reader, reader.uint32())); + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.binlogdata.VEvent.decode(reader, reader.uint32())); break; } default: @@ -72196,190 +73344,143 @@ export const binlogdata = $root.binlogdata = (() => { }; /** - * Decodes a VStreamResultsResponse message from the specified reader or buffer, length delimited. + * Decodes a VStreamResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse + * @returns {binlogdata.VStreamResponse} VStreamResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VStreamResultsResponse.decodeDelimited = function decodeDelimited(reader) { + VStreamResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VStreamResultsResponse message. + * Verifies a VStreamResponse message. * @function verify - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VStreamResultsResponse.verify = function verify(message) { + VStreamResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.fields != null && message.hasOwnProperty("fields")) { - if (!Array.isArray(message.fields)) - return "fields: array expected"; - for (let i = 0; i < message.fields.length; ++i) { - let error = $root.query.Field.verify(message.fields[i]); - if (error) - return "fields." + error; - } - } - if (message.gtid != null && message.hasOwnProperty("gtid")) - if (!$util.isString(message.gtid)) - return "gtid: string expected"; - if (message.rows != null && message.hasOwnProperty("rows")) { - if (!Array.isArray(message.rows)) - return "rows: array expected"; - for (let i = 0; i < message.rows.length; ++i) { - let error = $root.query.Row.verify(message.rows[i]); + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.binlogdata.VEvent.verify(message.events[i]); if (error) - return "rows." + error; + return "events." + error; } } return null; }; /** - * Creates a VStreamResultsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static * @param {Object.} object Plain object - * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse + * @returns {binlogdata.VStreamResponse} VStreamResponse */ - VStreamResultsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.binlogdata.VStreamResultsResponse) + VStreamResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamResponse) return object; - let message = new $root.binlogdata.VStreamResultsResponse(); - if (object.fields) { - if (!Array.isArray(object.fields)) - throw TypeError(".binlogdata.VStreamResultsResponse.fields: array expected"); - message.fields = []; - for (let i = 0; i < object.fields.length; ++i) { - if (typeof object.fields[i] !== "object") - throw TypeError(".binlogdata.VStreamResultsResponse.fields: object expected"); - message.fields[i] = $root.query.Field.fromObject(object.fields[i]); - } - } - if (object.gtid != null) - message.gtid = String(object.gtid); - if (object.rows) { - if (!Array.isArray(object.rows)) - throw TypeError(".binlogdata.VStreamResultsResponse.rows: array expected"); - message.rows = []; - for (let i = 0; i < object.rows.length; ++i) { - if (typeof object.rows[i] !== "object") - throw TypeError(".binlogdata.VStreamResultsResponse.rows: object expected"); - message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + let message = new $root.binlogdata.VStreamResponse(); + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".binlogdata.VStreamResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".binlogdata.VStreamResponse.events: object expected"); + message.events[i] = $root.binlogdata.VEvent.fromObject(object.events[i]); } } return message; }; /** - * Creates a plain object from a VStreamResultsResponse message. Also converts values to other types if specified. + * Creates a plain object from a VStreamResponse message. Also converts values to other types if specified. * @function toObject - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static - * @param {binlogdata.VStreamResultsResponse} message VStreamResultsResponse + * @param {binlogdata.VStreamResponse} message VStreamResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VStreamResultsResponse.toObject = function toObject(message, options) { + VStreamResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.fields = []; - object.rows = []; - } - if (options.defaults) - object.gtid = ""; - if (message.fields && message.fields.length) { - object.fields = []; - for (let j = 0; j < message.fields.length; ++j) - object.fields[j] = $root.query.Field.toObject(message.fields[j], options); - } - if (message.gtid != null && message.hasOwnProperty("gtid")) - object.gtid = message.gtid; - if (message.rows && message.rows.length) { - object.rows = []; - for (let j = 0; j < message.rows.length; ++j) - object.rows[j] = $root.query.Row.toObject(message.rows[j], options); + if (options.arrays || options.defaults) + object.events = []; + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.binlogdata.VEvent.toObject(message.events[j], options); } return object; }; /** - * Converts this VStreamResultsResponse to JSON. + * Converts this VStreamResponse to JSON. * @function toJSON - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @instance * @returns {Object.} JSON object */ - VStreamResultsResponse.prototype.toJSON = function toJSON() { + VStreamResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VStreamResultsResponse + * Gets the default type url for VStreamResponse * @function getTypeUrl - * @memberof binlogdata.VStreamResultsResponse + * @memberof binlogdata.VStreamResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VStreamResultsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/binlogdata.VStreamResultsResponse"; + return typeUrlPrefix + "/binlogdata.VStreamResponse"; }; - return VStreamResultsResponse; + return VStreamResponse; })(); - return binlogdata; -})(); - -export const vtrpc = $root.vtrpc = (() => { - - /** - * Namespace vtrpc. - * @exports vtrpc - * @namespace - */ - const vtrpc = {}; - - vtrpc.CallerID = (function() { + binlogdata.VStreamRowsRequest = (function() { /** - * Properties of a CallerID. - * @memberof vtrpc - * @interface ICallerID - * @property {string|null} [principal] CallerID principal - * @property {string|null} [component] CallerID component - * @property {string|null} [subcomponent] CallerID subcomponent - * @property {Array.|null} [groups] CallerID groups + * Properties of a VStreamRowsRequest. + * @memberof binlogdata + * @interface IVStreamRowsRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamRowsRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamRowsRequest immediate_caller_id + * @property {query.ITarget|null} [target] VStreamRowsRequest target + * @property {string|null} [query] VStreamRowsRequest query + * @property {query.IQueryResult|null} [lastpk] VStreamRowsRequest lastpk */ /** - * Constructs a new CallerID. - * @memberof vtrpc - * @classdesc Represents a CallerID. - * @implements ICallerID + * Constructs a new VStreamRowsRequest. + * @memberof binlogdata + * @classdesc Represents a VStreamRowsRequest. + * @implements IVStreamRowsRequest * @constructor - * @param {vtrpc.ICallerID=} [properties] Properties to set + * @param {binlogdata.IVStreamRowsRequest=} [properties] Properties to set */ - function CallerID(properties) { - this.groups = []; + function VStreamRowsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -72387,120 +73488,131 @@ export const vtrpc = $root.vtrpc = (() => { } /** - * CallerID principal. - * @member {string} principal - * @memberof vtrpc.CallerID + * VStreamRowsRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof binlogdata.VStreamRowsRequest * @instance */ - CallerID.prototype.principal = ""; + VStreamRowsRequest.prototype.effective_caller_id = null; /** - * CallerID component. - * @member {string} component - * @memberof vtrpc.CallerID + * VStreamRowsRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof binlogdata.VStreamRowsRequest * @instance */ - CallerID.prototype.component = ""; + VStreamRowsRequest.prototype.immediate_caller_id = null; /** - * CallerID subcomponent. - * @member {string} subcomponent - * @memberof vtrpc.CallerID + * VStreamRowsRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof binlogdata.VStreamRowsRequest * @instance */ - CallerID.prototype.subcomponent = ""; + VStreamRowsRequest.prototype.target = null; /** - * CallerID groups. - * @member {Array.} groups - * @memberof vtrpc.CallerID + * VStreamRowsRequest query. + * @member {string} query + * @memberof binlogdata.VStreamRowsRequest * @instance */ - CallerID.prototype.groups = $util.emptyArray; + VStreamRowsRequest.prototype.query = ""; /** - * Creates a new CallerID instance using the specified properties. + * VStreamRowsRequest lastpk. + * @member {query.IQueryResult|null|undefined} lastpk + * @memberof binlogdata.VStreamRowsRequest + * @instance + */ + VStreamRowsRequest.prototype.lastpk = null; + + /** + * Creates a new VStreamRowsRequest instance using the specified properties. * @function create - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static - * @param {vtrpc.ICallerID=} [properties] Properties to set - * @returns {vtrpc.CallerID} CallerID instance + * @param {binlogdata.IVStreamRowsRequest=} [properties] Properties to set + * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest instance */ - CallerID.create = function create(properties) { - return new CallerID(properties); + VStreamRowsRequest.create = function create(properties) { + return new VStreamRowsRequest(properties); }; /** - * Encodes the specified CallerID message. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. + * Encodes the specified VStreamRowsRequest message. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. * @function encode - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static - * @param {vtrpc.ICallerID} message CallerID message or plain object to encode + * @param {binlogdata.IVStreamRowsRequest} message VStreamRowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CallerID.encode = function encode(message, writer) { + VStreamRowsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.principal != null && Object.hasOwnProperty.call(message, "principal")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.principal); - if (message.component != null && Object.hasOwnProperty.call(message, "component")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.component); - if (message.subcomponent != null && Object.hasOwnProperty.call(message, "subcomponent")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.subcomponent); - if (message.groups != null && message.groups.length) - for (let i = 0; i < message.groups.length; ++i) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.groups[i]); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.query); + if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) + $root.query.QueryResult.encode(message.lastpk, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified CallerID message, length delimited. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. + * Encodes the specified VStreamRowsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static - * @param {vtrpc.ICallerID} message CallerID message or plain object to encode + * @param {binlogdata.IVStreamRowsRequest} message VStreamRowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CallerID.encodeDelimited = function encodeDelimited(message, writer) { + VStreamRowsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CallerID message from the specified reader or buffer. + * Decodes a VStreamRowsRequest message from the specified reader or buffer. * @function decode - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtrpc.CallerID} CallerID + * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CallerID.decode = function decode(reader, length) { + VStreamRowsRequest.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.vtrpc.CallerID(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRowsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.principal = reader.string(); + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); break; } case 2: { - message.component = reader.string(); + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); break; } case 3: { - message.subcomponent = reader.string(); + message.target = $root.query.Target.decode(reader, reader.uint32()); break; } case 4: { - if (!(message.groups && message.groups.length)) - message.groups = []; - message.groups.push(reader.string()); + message.query = reader.string(); + break; + } + case 5: { + message.lastpk = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -72512,299 +73624,353 @@ export const vtrpc = $root.vtrpc = (() => { }; /** - * Decodes a CallerID message from the specified reader or buffer, length delimited. + * Decodes a VStreamRowsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtrpc.CallerID} CallerID + * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CallerID.decodeDelimited = function decodeDelimited(reader) { + VStreamRowsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CallerID message. + * Verifies a VStreamRowsRequest message. * @function verify - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CallerID.verify = function verify(message) { + VStreamRowsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.principal != null && message.hasOwnProperty("principal")) - if (!$util.isString(message.principal)) - return "principal: string expected"; - if (message.component != null && message.hasOwnProperty("component")) - if (!$util.isString(message.component)) - return "component: string expected"; - if (message.subcomponent != null && message.hasOwnProperty("subcomponent")) - if (!$util.isString(message.subcomponent)) - return "subcomponent: string expected"; - if (message.groups != null && message.hasOwnProperty("groups")) { - if (!Array.isArray(message.groups)) - return "groups: array expected"; - for (let i = 0; i < message.groups.length; ++i) - if (!$util.isString(message.groups[i])) - return "groups: string[] expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) + if (!$util.isString(message.query)) + return "query: string expected"; + if (message.lastpk != null && message.hasOwnProperty("lastpk")) { + let error = $root.query.QueryResult.verify(message.lastpk); + if (error) + return "lastpk." + error; } return null; }; /** - * Creates a CallerID message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRowsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static * @param {Object.} object Plain object - * @returns {vtrpc.CallerID} CallerID + * @returns {binlogdata.VStreamRowsRequest} VStreamRowsRequest */ - CallerID.fromObject = function fromObject(object) { - if (object instanceof $root.vtrpc.CallerID) + VStreamRowsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamRowsRequest) return object; - let message = new $root.vtrpc.CallerID(); - if (object.principal != null) - message.principal = String(object.principal); - if (object.component != null) - message.component = String(object.component); - if (object.subcomponent != null) - message.subcomponent = String(object.subcomponent); - if (object.groups) { - if (!Array.isArray(object.groups)) - throw TypeError(".vtrpc.CallerID.groups: array expected"); - message.groups = []; - for (let i = 0; i < object.groups.length; ++i) - message.groups[i] = String(object.groups[i]); + let message = new $root.binlogdata.VStreamRowsRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".binlogdata.VStreamRowsRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".binlogdata.VStreamRowsRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".binlogdata.VStreamRowsRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) + message.query = String(object.query); + if (object.lastpk != null) { + if (typeof object.lastpk !== "object") + throw TypeError(".binlogdata.VStreamRowsRequest.lastpk: object expected"); + message.lastpk = $root.query.QueryResult.fromObject(object.lastpk); } return message; }; /** - * Creates a plain object from a CallerID message. Also converts values to other types if specified. + * Creates a plain object from a VStreamRowsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static - * @param {vtrpc.CallerID} message CallerID + * @param {binlogdata.VStreamRowsRequest} message VStreamRowsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CallerID.toObject = function toObject(message, options) { + VStreamRowsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.groups = []; if (options.defaults) { - object.principal = ""; - object.component = ""; - object.subcomponent = ""; - } - if (message.principal != null && message.hasOwnProperty("principal")) - object.principal = message.principal; - if (message.component != null && message.hasOwnProperty("component")) - object.component = message.component; - if (message.subcomponent != null && message.hasOwnProperty("subcomponent")) - object.subcomponent = message.subcomponent; - if (message.groups && message.groups.length) { - object.groups = []; - for (let j = 0; j < message.groups.length; ++j) - object.groups[j] = message.groups[j]; + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = ""; + object.lastpk = null; } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = message.query; + if (message.lastpk != null && message.hasOwnProperty("lastpk")) + object.lastpk = $root.query.QueryResult.toObject(message.lastpk, options); return object; }; /** - * Converts this CallerID to JSON. + * Converts this VStreamRowsRequest to JSON. * @function toJSON - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @instance * @returns {Object.} JSON object */ - CallerID.prototype.toJSON = function toJSON() { + VStreamRowsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CallerID + * Gets the default type url for VStreamRowsRequest * @function getTypeUrl - * @memberof vtrpc.CallerID + * @memberof binlogdata.VStreamRowsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CallerID.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamRowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtrpc.CallerID"; + return typeUrlPrefix + "/binlogdata.VStreamRowsRequest"; }; - return CallerID; - })(); - - /** - * Code enum. - * @name vtrpc.Code - * @enum {number} - * @property {number} OK=0 OK value - * @property {number} CANCELED=1 CANCELED value - * @property {number} UNKNOWN=2 UNKNOWN value - * @property {number} INVALID_ARGUMENT=3 INVALID_ARGUMENT value - * @property {number} DEADLINE_EXCEEDED=4 DEADLINE_EXCEEDED value - * @property {number} NOT_FOUND=5 NOT_FOUND value - * @property {number} ALREADY_EXISTS=6 ALREADY_EXISTS value - * @property {number} PERMISSION_DENIED=7 PERMISSION_DENIED value - * @property {number} RESOURCE_EXHAUSTED=8 RESOURCE_EXHAUSTED value - * @property {number} FAILED_PRECONDITION=9 FAILED_PRECONDITION value - * @property {number} ABORTED=10 ABORTED value - * @property {number} OUT_OF_RANGE=11 OUT_OF_RANGE value - * @property {number} UNIMPLEMENTED=12 UNIMPLEMENTED value - * @property {number} INTERNAL=13 INTERNAL value - * @property {number} UNAVAILABLE=14 UNAVAILABLE value - * @property {number} DATA_LOSS=15 DATA_LOSS value - * @property {number} UNAUTHENTICATED=16 UNAUTHENTICATED value - * @property {number} CLUSTER_EVENT=17 CLUSTER_EVENT value - * @property {number} READ_ONLY=18 READ_ONLY value - */ - vtrpc.Code = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "OK"] = 0; - values[valuesById[1] = "CANCELED"] = 1; - values[valuesById[2] = "UNKNOWN"] = 2; - values[valuesById[3] = "INVALID_ARGUMENT"] = 3; - values[valuesById[4] = "DEADLINE_EXCEEDED"] = 4; - values[valuesById[5] = "NOT_FOUND"] = 5; - values[valuesById[6] = "ALREADY_EXISTS"] = 6; - values[valuesById[7] = "PERMISSION_DENIED"] = 7; - values[valuesById[8] = "RESOURCE_EXHAUSTED"] = 8; - values[valuesById[9] = "FAILED_PRECONDITION"] = 9; - values[valuesById[10] = "ABORTED"] = 10; - values[valuesById[11] = "OUT_OF_RANGE"] = 11; - values[valuesById[12] = "UNIMPLEMENTED"] = 12; - values[valuesById[13] = "INTERNAL"] = 13; - values[valuesById[14] = "UNAVAILABLE"] = 14; - values[valuesById[15] = "DATA_LOSS"] = 15; - values[valuesById[16] = "UNAUTHENTICATED"] = 16; - values[valuesById[17] = "CLUSTER_EVENT"] = 17; - values[valuesById[18] = "READ_ONLY"] = 18; - return values; + return VStreamRowsRequest; })(); - vtrpc.RPCError = (function() { + binlogdata.VStreamRowsResponse = (function() { /** - * Properties of a RPCError. - * @memberof vtrpc - * @interface IRPCError - * @property {string|null} [message] RPCError message - * @property {vtrpc.Code|null} [code] RPCError code + * Properties of a VStreamRowsResponse. + * @memberof binlogdata + * @interface IVStreamRowsResponse + * @property {Array.|null} [fields] VStreamRowsResponse fields + * @property {Array.|null} [pkfields] VStreamRowsResponse pkfields + * @property {string|null} [gtid] VStreamRowsResponse gtid + * @property {Array.|null} [rows] VStreamRowsResponse rows + * @property {query.IRow|null} [lastpk] VStreamRowsResponse lastpk + * @property {boolean|null} [throttled] VStreamRowsResponse throttled + * @property {boolean|null} [heartbeat] VStreamRowsResponse heartbeat */ /** - * Constructs a new RPCError. - * @memberof vtrpc - * @classdesc Represents a RPCError. - * @implements IRPCError + * Constructs a new VStreamRowsResponse. + * @memberof binlogdata + * @classdesc Represents a VStreamRowsResponse. + * @implements IVStreamRowsResponse * @constructor - * @param {vtrpc.IRPCError=} [properties] Properties to set + * @param {binlogdata.IVStreamRowsResponse=} [properties] Properties to set */ - function RPCError(properties) { - if (properties) + function VStreamRowsResponse(properties) { + this.fields = []; + this.pkfields = []; + this.rows = []; + if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) this[keys[i]] = properties[keys[i]]; } /** - * RPCError message. - * @member {string} message - * @memberof vtrpc.RPCError + * VStreamRowsResponse fields. + * @member {Array.} fields + * @memberof binlogdata.VStreamRowsResponse * @instance */ - RPCError.prototype.message = ""; + VStreamRowsResponse.prototype.fields = $util.emptyArray; /** - * RPCError code. - * @member {vtrpc.Code} code - * @memberof vtrpc.RPCError + * VStreamRowsResponse pkfields. + * @member {Array.} pkfields + * @memberof binlogdata.VStreamRowsResponse * @instance */ - RPCError.prototype.code = 0; + VStreamRowsResponse.prototype.pkfields = $util.emptyArray; /** - * Creates a new RPCError instance using the specified properties. + * VStreamRowsResponse gtid. + * @member {string} gtid + * @memberof binlogdata.VStreamRowsResponse + * @instance + */ + VStreamRowsResponse.prototype.gtid = ""; + + /** + * VStreamRowsResponse rows. + * @member {Array.} rows + * @memberof binlogdata.VStreamRowsResponse + * @instance + */ + VStreamRowsResponse.prototype.rows = $util.emptyArray; + + /** + * VStreamRowsResponse lastpk. + * @member {query.IRow|null|undefined} lastpk + * @memberof binlogdata.VStreamRowsResponse + * @instance + */ + VStreamRowsResponse.prototype.lastpk = null; + + /** + * VStreamRowsResponse throttled. + * @member {boolean} throttled + * @memberof binlogdata.VStreamRowsResponse + * @instance + */ + VStreamRowsResponse.prototype.throttled = false; + + /** + * VStreamRowsResponse heartbeat. + * @member {boolean} heartbeat + * @memberof binlogdata.VStreamRowsResponse + * @instance + */ + VStreamRowsResponse.prototype.heartbeat = false; + + /** + * Creates a new VStreamRowsResponse instance using the specified properties. * @function create - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static - * @param {vtrpc.IRPCError=} [properties] Properties to set - * @returns {vtrpc.RPCError} RPCError instance + * @param {binlogdata.IVStreamRowsResponse=} [properties] Properties to set + * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse instance */ - RPCError.create = function create(properties) { - return new RPCError(properties); + VStreamRowsResponse.create = function create(properties) { + return new VStreamRowsResponse(properties); }; /** - * Encodes the specified RPCError message. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. + * Encodes the specified VStreamRowsResponse message. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. * @function encode - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static - * @param {vtrpc.IRPCError} message RPCError message or plain object to encode + * @param {binlogdata.IVStreamRowsResponse} message VStreamRowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RPCError.encode = function encode(message, writer) { + VStreamRowsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.message != null && Object.hasOwnProperty.call(message, "message")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.message); - if (message.code != null && Object.hasOwnProperty.call(message, "code")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.code); + if (message.fields != null && message.fields.length) + for (let i = 0; i < message.fields.length; ++i) + $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.pkfields != null && message.pkfields.length) + for (let i = 0; i < message.pkfields.length; ++i) + $root.query.Field.encode(message.pkfields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); + if (message.rows != null && message.rows.length) + for (let i = 0; i < message.rows.length; ++i) + $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) + $root.query.Row.encode(message.lastpk, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.throttled != null && Object.hasOwnProperty.call(message, "throttled")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.throttled); + if (message.heartbeat != null && Object.hasOwnProperty.call(message, "heartbeat")) + writer.uint32(/* id 7, wireType 0 =*/56).bool(message.heartbeat); return writer; }; /** - * Encodes the specified RPCError message, length delimited. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. + * Encodes the specified VStreamRowsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamRowsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static - * @param {vtrpc.IRPCError} message RPCError message or plain object to encode + * @param {binlogdata.IVStreamRowsResponse} message VStreamRowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RPCError.encodeDelimited = function encodeDelimited(message, writer) { + VStreamRowsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RPCError message from the specified reader or buffer. + * Decodes a VStreamRowsResponse message from the specified reader or buffer. * @function decode - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtrpc.RPCError} RPCError + * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RPCError.decode = function decode(reader, length) { + VStreamRowsResponse.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.vtrpc.RPCError(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamRowsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + if (!(message.fields && message.fields.length)) + message.fields = []; + message.fields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } case 2: { - message.message = reader.string(); + if (!(message.pkfields && message.pkfields.length)) + message.pkfields = []; + message.pkfields.push($root.query.Field.decode(reader, reader.uint32())); break; } case 3: { - message.code = reader.int32(); + message.gtid = reader.string(); + break; + } + case 4: { + if (!(message.rows && message.rows.length)) + message.rows = []; + message.rows.push($root.query.Row.decode(reader, reader.uint32())); + break; + } + case 5: { + message.lastpk = $root.query.Row.decode(reader, reader.uint32()); + break; + } + case 6: { + message.throttled = reader.bool(); + break; + } + case 7: { + message.heartbeat = reader.bool(); break; } default: @@ -72816,250 +73982,230 @@ export const vtrpc = $root.vtrpc = (() => { }; /** - * Decodes a RPCError message from the specified reader or buffer, length delimited. + * Decodes a VStreamRowsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtrpc.RPCError} RPCError + * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RPCError.decodeDelimited = function decodeDelimited(reader) { + VStreamRowsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RPCError message. + * Verifies a VStreamRowsResponse message. * @function verify - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RPCError.verify = function verify(message) { + VStreamRowsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.message != null && message.hasOwnProperty("message")) - if (!$util.isString(message.message)) - return "message: string expected"; - if (message.code != null && message.hasOwnProperty("code")) - switch (message.code) { - default: - return "code: enum value expected"; - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - case 9: - case 10: - case 11: - case 12: - case 13: - case 14: - case 15: - case 16: - case 17: - case 18: - break; + if (message.fields != null && message.hasOwnProperty("fields")) { + if (!Array.isArray(message.fields)) + return "fields: array expected"; + for (let i = 0; i < message.fields.length; ++i) { + let error = $root.query.Field.verify(message.fields[i]); + if (error) + return "fields." + error; + } + } + if (message.pkfields != null && message.hasOwnProperty("pkfields")) { + if (!Array.isArray(message.pkfields)) + return "pkfields: array expected"; + for (let i = 0; i < message.pkfields.length; ++i) { + let error = $root.query.Field.verify(message.pkfields[i]); + if (error) + return "pkfields." + error; + } + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + if (!$util.isString(message.gtid)) + return "gtid: string expected"; + if (message.rows != null && message.hasOwnProperty("rows")) { + if (!Array.isArray(message.rows)) + return "rows: array expected"; + for (let i = 0; i < message.rows.length; ++i) { + let error = $root.query.Row.verify(message.rows[i]); + if (error) + return "rows." + error; } + } + if (message.lastpk != null && message.hasOwnProperty("lastpk")) { + let error = $root.query.Row.verify(message.lastpk); + if (error) + return "lastpk." + error; + } + if (message.throttled != null && message.hasOwnProperty("throttled")) + if (typeof message.throttled !== "boolean") + return "throttled: boolean expected"; + if (message.heartbeat != null && message.hasOwnProperty("heartbeat")) + if (typeof message.heartbeat !== "boolean") + return "heartbeat: boolean expected"; return null; }; /** - * Creates a RPCError message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamRowsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static * @param {Object.} object Plain object - * @returns {vtrpc.RPCError} RPCError + * @returns {binlogdata.VStreamRowsResponse} VStreamRowsResponse */ - RPCError.fromObject = function fromObject(object) { - if (object instanceof $root.vtrpc.RPCError) + VStreamRowsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamRowsResponse) return object; - let message = new $root.vtrpc.RPCError(); - if (object.message != null) - message.message = String(object.message); - switch (object.code) { - default: - if (typeof object.code === "number") { - message.code = object.code; - break; + let message = new $root.binlogdata.VStreamRowsResponse(); + if (object.fields) { + if (!Array.isArray(object.fields)) + throw TypeError(".binlogdata.VStreamRowsResponse.fields: array expected"); + message.fields = []; + for (let i = 0; i < object.fields.length; ++i) { + if (typeof object.fields[i] !== "object") + throw TypeError(".binlogdata.VStreamRowsResponse.fields: object expected"); + message.fields[i] = $root.query.Field.fromObject(object.fields[i]); } - break; - case "OK": - case 0: - message.code = 0; - break; - case "CANCELED": - case 1: - message.code = 1; - break; - case "UNKNOWN": - case 2: - message.code = 2; - break; - case "INVALID_ARGUMENT": - case 3: - message.code = 3; - break; - case "DEADLINE_EXCEEDED": - case 4: - message.code = 4; - break; - case "NOT_FOUND": - case 5: - message.code = 5; - break; - case "ALREADY_EXISTS": - case 6: - message.code = 6; - break; - case "PERMISSION_DENIED": - case 7: - message.code = 7; - break; - case "RESOURCE_EXHAUSTED": - case 8: - message.code = 8; - break; - case "FAILED_PRECONDITION": - case 9: - message.code = 9; - break; - case "ABORTED": - case 10: - message.code = 10; - break; - case "OUT_OF_RANGE": - case 11: - message.code = 11; - break; - case "UNIMPLEMENTED": - case 12: - message.code = 12; - break; - case "INTERNAL": - case 13: - message.code = 13; - break; - case "UNAVAILABLE": - case 14: - message.code = 14; - break; - case "DATA_LOSS": - case 15: - message.code = 15; - break; - case "UNAUTHENTICATED": - case 16: - message.code = 16; - break; - case "CLUSTER_EVENT": - case 17: - message.code = 17; - break; - case "READ_ONLY": - case 18: - message.code = 18; - break; } + if (object.pkfields) { + if (!Array.isArray(object.pkfields)) + throw TypeError(".binlogdata.VStreamRowsResponse.pkfields: array expected"); + message.pkfields = []; + for (let i = 0; i < object.pkfields.length; ++i) { + if (typeof object.pkfields[i] !== "object") + throw TypeError(".binlogdata.VStreamRowsResponse.pkfields: object expected"); + message.pkfields[i] = $root.query.Field.fromObject(object.pkfields[i]); + } + } + if (object.gtid != null) + message.gtid = String(object.gtid); + if (object.rows) { + if (!Array.isArray(object.rows)) + throw TypeError(".binlogdata.VStreamRowsResponse.rows: array expected"); + message.rows = []; + for (let i = 0; i < object.rows.length; ++i) { + if (typeof object.rows[i] !== "object") + throw TypeError(".binlogdata.VStreamRowsResponse.rows: object expected"); + message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + } + } + if (object.lastpk != null) { + if (typeof object.lastpk !== "object") + throw TypeError(".binlogdata.VStreamRowsResponse.lastpk: object expected"); + message.lastpk = $root.query.Row.fromObject(object.lastpk); + } + if (object.throttled != null) + message.throttled = Boolean(object.throttled); + if (object.heartbeat != null) + message.heartbeat = Boolean(object.heartbeat); return message; }; /** - * Creates a plain object from a RPCError message. Also converts values to other types if specified. + * Creates a plain object from a VStreamRowsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static - * @param {vtrpc.RPCError} message RPCError + * @param {binlogdata.VStreamRowsResponse} message VStreamRowsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RPCError.toObject = function toObject(message, options) { + VStreamRowsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.fields = []; + object.pkfields = []; + object.rows = []; + } if (options.defaults) { - object.message = ""; - object.code = options.enums === String ? "OK" : 0; + object.gtid = ""; + object.lastpk = null; + object.throttled = false; + object.heartbeat = false; } - if (message.message != null && message.hasOwnProperty("message")) - object.message = message.message; - if (message.code != null && message.hasOwnProperty("code")) - object.code = options.enums === String ? $root.vtrpc.Code[message.code] === undefined ? message.code : $root.vtrpc.Code[message.code] : message.code; + if (message.fields && message.fields.length) { + object.fields = []; + for (let j = 0; j < message.fields.length; ++j) + object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + } + if (message.pkfields && message.pkfields.length) { + object.pkfields = []; + for (let j = 0; j < message.pkfields.length; ++j) + object.pkfields[j] = $root.query.Field.toObject(message.pkfields[j], options); + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + object.gtid = message.gtid; + if (message.rows && message.rows.length) { + object.rows = []; + for (let j = 0; j < message.rows.length; ++j) + object.rows[j] = $root.query.Row.toObject(message.rows[j], options); + } + if (message.lastpk != null && message.hasOwnProperty("lastpk")) + object.lastpk = $root.query.Row.toObject(message.lastpk, options); + if (message.throttled != null && message.hasOwnProperty("throttled")) + object.throttled = message.throttled; + if (message.heartbeat != null && message.hasOwnProperty("heartbeat")) + object.heartbeat = message.heartbeat; return object; }; /** - * Converts this RPCError to JSON. + * Converts this VStreamRowsResponse to JSON. * @function toJSON - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @instance * @returns {Object.} JSON object */ - RPCError.prototype.toJSON = function toJSON() { + VStreamRowsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RPCError + * Gets the default type url for VStreamRowsResponse * @function getTypeUrl - * @memberof vtrpc.RPCError + * @memberof binlogdata.VStreamRowsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RPCError.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamRowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtrpc.RPCError"; + return typeUrlPrefix + "/binlogdata.VStreamRowsResponse"; }; - return RPCError; + return VStreamRowsResponse; })(); - return vtrpc; -})(); - -export const query = $root.query = (() => { - - /** - * Namespace query. - * @exports query - * @namespace - */ - const query = {}; - - query.Target = (function() { + binlogdata.LastPKEvent = (function() { /** - * Properties of a Target. - * @memberof query - * @interface ITarget - * @property {string|null} [keyspace] Target keyspace - * @property {string|null} [shard] Target shard - * @property {topodata.TabletType|null} [tablet_type] Target tablet_type - * @property {string|null} [cell] Target cell + * Properties of a LastPKEvent. + * @memberof binlogdata + * @interface ILastPKEvent + * @property {binlogdata.ITableLastPK|null} [table_last_p_k] LastPKEvent table_last_p_k + * @property {boolean|null} [completed] LastPKEvent completed */ /** - * Constructs a new Target. - * @memberof query - * @classdesc Represents a Target. - * @implements ITarget + * Constructs a new LastPKEvent. + * @memberof binlogdata + * @classdesc Represents a LastPKEvent. + * @implements ILastPKEvent * @constructor - * @param {query.ITarget=} [properties] Properties to set + * @param {binlogdata.ILastPKEvent=} [properties] Properties to set */ - function Target(properties) { + function LastPKEvent(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -73067,117 +74213,89 @@ export const query = $root.query = (() => { } /** - * Target keyspace. - * @member {string} keyspace - * @memberof query.Target - * @instance - */ - Target.prototype.keyspace = ""; - - /** - * Target shard. - * @member {string} shard - * @memberof query.Target - * @instance - */ - Target.prototype.shard = ""; - - /** - * Target tablet_type. - * @member {topodata.TabletType} tablet_type - * @memberof query.Target + * LastPKEvent table_last_p_k. + * @member {binlogdata.ITableLastPK|null|undefined} table_last_p_k + * @memberof binlogdata.LastPKEvent * @instance */ - Target.prototype.tablet_type = 0; + LastPKEvent.prototype.table_last_p_k = null; /** - * Target cell. - * @member {string} cell - * @memberof query.Target + * LastPKEvent completed. + * @member {boolean} completed + * @memberof binlogdata.LastPKEvent * @instance */ - Target.prototype.cell = ""; + LastPKEvent.prototype.completed = false; /** - * Creates a new Target instance using the specified properties. + * Creates a new LastPKEvent instance using the specified properties. * @function create - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static - * @param {query.ITarget=} [properties] Properties to set - * @returns {query.Target} Target instance + * @param {binlogdata.ILastPKEvent=} [properties] Properties to set + * @returns {binlogdata.LastPKEvent} LastPKEvent instance */ - Target.create = function create(properties) { - return new Target(properties); + LastPKEvent.create = function create(properties) { + return new LastPKEvent(properties); }; /** - * Encodes the specified Target message. Does not implicitly {@link query.Target.verify|verify} messages. + * Encodes the specified LastPKEvent message. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. * @function encode - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static - * @param {query.ITarget} message Target message or plain object to encode + * @param {binlogdata.ILastPKEvent} message LastPKEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Target.encode = function encode(message, writer) { + LastPKEvent.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.cell); + if (message.table_last_p_k != null && Object.hasOwnProperty.call(message, "table_last_p_k")) + $root.binlogdata.TableLastPK.encode(message.table_last_p_k, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.completed != null && Object.hasOwnProperty.call(message, "completed")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.completed); return writer; }; /** - * Encodes the specified Target message, length delimited. Does not implicitly {@link query.Target.verify|verify} messages. + * Encodes the specified LastPKEvent message, length delimited. Does not implicitly {@link binlogdata.LastPKEvent.verify|verify} messages. * @function encodeDelimited - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static - * @param {query.ITarget} message Target message or plain object to encode + * @param {binlogdata.ILastPKEvent} message LastPKEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Target.encodeDelimited = function encodeDelimited(message, writer) { + LastPKEvent.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Target message from the specified reader or buffer. + * Decodes a LastPKEvent message from the specified reader or buffer. * @function decode - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.Target} Target + * @returns {binlogdata.LastPKEvent} LastPKEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Target.decode = function decode(reader, length) { + LastPKEvent.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.query.Target(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.LastPKEvent(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.table_last_p_k = $root.binlogdata.TableLastPK.decode(reader, reader.uint32()); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.tablet_type = reader.int32(); - break; - } - case 4: { - message.cell = reader.string(); + message.completed = reader.bool(); break; } default: @@ -73189,213 +74307,137 @@ export const query = $root.query = (() => { }; /** - * Decodes a Target message from the specified reader or buffer, length delimited. + * Decodes a LastPKEvent message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.Target} Target + * @returns {binlogdata.LastPKEvent} LastPKEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Target.decodeDelimited = function decodeDelimited(reader) { + LastPKEvent.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Target message. + * Verifies a LastPKEvent message. * @function verify - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Target.verify = function verify(message) { + LastPKEvent.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - switch (message.tablet_type) { - default: - return "tablet_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; + if (message.table_last_p_k != null && message.hasOwnProperty("table_last_p_k")) { + let error = $root.binlogdata.TableLastPK.verify(message.table_last_p_k); + if (error) + return "table_last_p_k." + error; + } + if (message.completed != null && message.hasOwnProperty("completed")) + if (typeof message.completed !== "boolean") + return "completed: boolean expected"; return null; }; /** - * Creates a Target message from a plain object. Also converts values to their respective internal types. + * Creates a LastPKEvent message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static * @param {Object.} object Plain object - * @returns {query.Target} Target + * @returns {binlogdata.LastPKEvent} LastPKEvent */ - Target.fromObject = function fromObject(object) { - if (object instanceof $root.query.Target) + LastPKEvent.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.LastPKEvent) return object; - let message = new $root.query.Target(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - switch (object.tablet_type) { - default: - if (typeof object.tablet_type === "number") { - message.tablet_type = object.tablet_type; - break; - } - break; - case "UNKNOWN": - case 0: - message.tablet_type = 0; - break; - case "PRIMARY": - case 1: - message.tablet_type = 1; - break; - case "MASTER": - case 1: - message.tablet_type = 1; - break; - case "REPLICA": - case 2: - message.tablet_type = 2; - break; - case "RDONLY": - case 3: - message.tablet_type = 3; - break; - case "BATCH": - case 3: - message.tablet_type = 3; - break; - case "SPARE": - case 4: - message.tablet_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_type = 5; - break; - case "BACKUP": - case 6: - message.tablet_type = 6; - break; - case "RESTORE": - case 7: - message.tablet_type = 7; - break; - case "DRAINED": - case 8: - message.tablet_type = 8; - break; + let message = new $root.binlogdata.LastPKEvent(); + if (object.table_last_p_k != null) { + if (typeof object.table_last_p_k !== "object") + throw TypeError(".binlogdata.LastPKEvent.table_last_p_k: object expected"); + message.table_last_p_k = $root.binlogdata.TableLastPK.fromObject(object.table_last_p_k); } - if (object.cell != null) - message.cell = String(object.cell); + if (object.completed != null) + message.completed = Boolean(object.completed); return message; }; /** - * Creates a plain object from a Target message. Also converts values to other types if specified. + * Creates a plain object from a LastPKEvent message. Also converts values to other types if specified. * @function toObject - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static - * @param {query.Target} message Target + * @param {binlogdata.LastPKEvent} message LastPKEvent * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Target.toObject = function toObject(message, options) { + LastPKEvent.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.tablet_type = options.enums === String ? "UNKNOWN" : 0; - object.cell = ""; + object.table_last_p_k = null; + object.completed = false; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; + if (message.table_last_p_k != null && message.hasOwnProperty("table_last_p_k")) + object.table_last_p_k = $root.binlogdata.TableLastPK.toObject(message.table_last_p_k, options); + if (message.completed != null && message.hasOwnProperty("completed")) + object.completed = message.completed; return object; }; /** - * Converts this Target to JSON. + * Converts this LastPKEvent to JSON. * @function toJSON - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @instance * @returns {Object.} JSON object */ - Target.prototype.toJSON = function toJSON() { + LastPKEvent.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Target + * Gets the default type url for LastPKEvent * @function getTypeUrl - * @memberof query.Target + * @memberof binlogdata.LastPKEvent * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Target.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + LastPKEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.Target"; + return typeUrlPrefix + "/binlogdata.LastPKEvent"; }; - return Target; + return LastPKEvent; })(); - query.VTGateCallerID = (function() { + binlogdata.TableLastPK = (function() { /** - * Properties of a VTGateCallerID. - * @memberof query - * @interface IVTGateCallerID - * @property {string|null} [username] VTGateCallerID username - * @property {Array.|null} [groups] VTGateCallerID groups + * Properties of a TableLastPK. + * @memberof binlogdata + * @interface ITableLastPK + * @property {string|null} [table_name] TableLastPK table_name + * @property {query.IQueryResult|null} [lastpk] TableLastPK lastpk */ /** - * Constructs a new VTGateCallerID. - * @memberof query - * @classdesc Represents a VTGateCallerID. - * @implements IVTGateCallerID + * Constructs a new TableLastPK. + * @memberof binlogdata + * @classdesc Represents a TableLastPK. + * @implements ITableLastPK * @constructor - * @param {query.IVTGateCallerID=} [properties] Properties to set + * @param {binlogdata.ITableLastPK=} [properties] Properties to set */ - function VTGateCallerID(properties) { - this.groups = []; + function TableLastPK(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -73403,92 +74445,89 @@ export const query = $root.query = (() => { } /** - * VTGateCallerID username. - * @member {string} username - * @memberof query.VTGateCallerID + * TableLastPK table_name. + * @member {string} table_name + * @memberof binlogdata.TableLastPK * @instance */ - VTGateCallerID.prototype.username = ""; + TableLastPK.prototype.table_name = ""; /** - * VTGateCallerID groups. - * @member {Array.} groups - * @memberof query.VTGateCallerID + * TableLastPK lastpk. + * @member {query.IQueryResult|null|undefined} lastpk + * @memberof binlogdata.TableLastPK * @instance */ - VTGateCallerID.prototype.groups = $util.emptyArray; + TableLastPK.prototype.lastpk = null; /** - * Creates a new VTGateCallerID instance using the specified properties. + * Creates a new TableLastPK instance using the specified properties. * @function create - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static - * @param {query.IVTGateCallerID=} [properties] Properties to set - * @returns {query.VTGateCallerID} VTGateCallerID instance + * @param {binlogdata.ITableLastPK=} [properties] Properties to set + * @returns {binlogdata.TableLastPK} TableLastPK instance */ - VTGateCallerID.create = function create(properties) { - return new VTGateCallerID(properties); + TableLastPK.create = function create(properties) { + return new TableLastPK(properties); }; /** - * Encodes the specified VTGateCallerID message. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. + * Encodes the specified TableLastPK message. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. * @function encode - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static - * @param {query.IVTGateCallerID} message VTGateCallerID message or plain object to encode + * @param {binlogdata.ITableLastPK} message TableLastPK message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VTGateCallerID.encode = function encode(message, writer) { + TableLastPK.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.username != null && Object.hasOwnProperty.call(message, "username")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.username); - if (message.groups != null && message.groups.length) - for (let i = 0; i < message.groups.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.groups[i]); + if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); + if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) + $root.query.QueryResult.encode(message.lastpk, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified VTGateCallerID message, length delimited. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. + * Encodes the specified TableLastPK message, length delimited. Does not implicitly {@link binlogdata.TableLastPK.verify|verify} messages. * @function encodeDelimited - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static - * @param {query.IVTGateCallerID} message VTGateCallerID message or plain object to encode + * @param {binlogdata.ITableLastPK} message TableLastPK message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VTGateCallerID.encodeDelimited = function encodeDelimited(message, writer) { + TableLastPK.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VTGateCallerID message from the specified reader or buffer. + * Decodes a TableLastPK message from the specified reader or buffer. * @function decode - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.VTGateCallerID} VTGateCallerID + * @returns {binlogdata.TableLastPK} TableLastPK * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VTGateCallerID.decode = function decode(reader, length) { + TableLastPK.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.query.VTGateCallerID(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.TableLastPK(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.username = reader.string(); + message.table_name = reader.string(); break; } - case 2: { - if (!(message.groups && message.groups.length)) - message.groups = []; - message.groups.push(reader.string()); + case 3: { + message.lastpk = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -73500,145 +74539,139 @@ export const query = $root.query = (() => { }; /** - * Decodes a VTGateCallerID message from the specified reader or buffer, length delimited. + * Decodes a TableLastPK message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.VTGateCallerID} VTGateCallerID + * @returns {binlogdata.TableLastPK} TableLastPK * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VTGateCallerID.decodeDelimited = function decodeDelimited(reader) { + TableLastPK.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VTGateCallerID message. + * Verifies a TableLastPK message. * @function verify - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VTGateCallerID.verify = function verify(message) { + TableLastPK.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.username != null && message.hasOwnProperty("username")) - if (!$util.isString(message.username)) - return "username: string expected"; - if (message.groups != null && message.hasOwnProperty("groups")) { - if (!Array.isArray(message.groups)) - return "groups: array expected"; - for (let i = 0; i < message.groups.length; ++i) - if (!$util.isString(message.groups[i])) - return "groups: string[] expected"; + if (message.table_name != null && message.hasOwnProperty("table_name")) + if (!$util.isString(message.table_name)) + return "table_name: string expected"; + if (message.lastpk != null && message.hasOwnProperty("lastpk")) { + let error = $root.query.QueryResult.verify(message.lastpk); + if (error) + return "lastpk." + error; } return null; }; /** - * Creates a VTGateCallerID message from a plain object. Also converts values to their respective internal types. + * Creates a TableLastPK message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static * @param {Object.} object Plain object - * @returns {query.VTGateCallerID} VTGateCallerID + * @returns {binlogdata.TableLastPK} TableLastPK */ - VTGateCallerID.fromObject = function fromObject(object) { - if (object instanceof $root.query.VTGateCallerID) + TableLastPK.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.TableLastPK) return object; - let message = new $root.query.VTGateCallerID(); - if (object.username != null) - message.username = String(object.username); - if (object.groups) { - if (!Array.isArray(object.groups)) - throw TypeError(".query.VTGateCallerID.groups: array expected"); - message.groups = []; - for (let i = 0; i < object.groups.length; ++i) - message.groups[i] = String(object.groups[i]); + let message = new $root.binlogdata.TableLastPK(); + if (object.table_name != null) + message.table_name = String(object.table_name); + if (object.lastpk != null) { + if (typeof object.lastpk !== "object") + throw TypeError(".binlogdata.TableLastPK.lastpk: object expected"); + message.lastpk = $root.query.QueryResult.fromObject(object.lastpk); } return message; }; /** - * Creates a plain object from a VTGateCallerID message. Also converts values to other types if specified. + * Creates a plain object from a TableLastPK message. Also converts values to other types if specified. * @function toObject - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static - * @param {query.VTGateCallerID} message VTGateCallerID + * @param {binlogdata.TableLastPK} message TableLastPK * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VTGateCallerID.toObject = function toObject(message, options) { + TableLastPK.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.groups = []; - if (options.defaults) - object.username = ""; - if (message.username != null && message.hasOwnProperty("username")) - object.username = message.username; - if (message.groups && message.groups.length) { - object.groups = []; - for (let j = 0; j < message.groups.length; ++j) - object.groups[j] = message.groups[j]; + if (options.defaults) { + object.table_name = ""; + object.lastpk = null; } + if (message.table_name != null && message.hasOwnProperty("table_name")) + object.table_name = message.table_name; + if (message.lastpk != null && message.hasOwnProperty("lastpk")) + object.lastpk = $root.query.QueryResult.toObject(message.lastpk, options); return object; }; /** - * Converts this VTGateCallerID to JSON. + * Converts this TableLastPK to JSON. * @function toJSON - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @instance * @returns {Object.} JSON object */ - VTGateCallerID.prototype.toJSON = function toJSON() { + TableLastPK.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VTGateCallerID + * Gets the default type url for TableLastPK * @function getTypeUrl - * @memberof query.VTGateCallerID + * @memberof binlogdata.TableLastPK * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VTGateCallerID.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + TableLastPK.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.VTGateCallerID"; + return typeUrlPrefix + "/binlogdata.TableLastPK"; }; - return VTGateCallerID; + return TableLastPK; })(); - query.EventToken = (function() { + binlogdata.VStreamResultsRequest = (function() { /** - * Properties of an EventToken. - * @memberof query - * @interface IEventToken - * @property {number|Long|null} [timestamp] EventToken timestamp - * @property {string|null} [shard] EventToken shard - * @property {string|null} [position] EventToken position + * Properties of a VStreamResultsRequest. + * @memberof binlogdata + * @interface IVStreamResultsRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamResultsRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamResultsRequest immediate_caller_id + * @property {query.ITarget|null} [target] VStreamResultsRequest target + * @property {string|null} [query] VStreamResultsRequest query */ /** - * Constructs a new EventToken. - * @memberof query - * @classdesc Represents an EventToken. - * @implements IEventToken + * Constructs a new VStreamResultsRequest. + * @memberof binlogdata + * @classdesc Represents a VStreamResultsRequest. + * @implements IVStreamResultsRequest * @constructor - * @param {query.IEventToken=} [properties] Properties to set + * @param {binlogdata.IVStreamResultsRequest=} [properties] Properties to set */ - function EventToken(properties) { + function VStreamResultsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -73646,103 +74679,117 @@ export const query = $root.query = (() => { } /** - * EventToken timestamp. - * @member {number|Long} timestamp - * @memberof query.EventToken + * VStreamResultsRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof binlogdata.VStreamResultsRequest * @instance */ - EventToken.prototype.timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + VStreamResultsRequest.prototype.effective_caller_id = null; /** - * EventToken shard. - * @member {string} shard - * @memberof query.EventToken + * VStreamResultsRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof binlogdata.VStreamResultsRequest * @instance */ - EventToken.prototype.shard = ""; + VStreamResultsRequest.prototype.immediate_caller_id = null; /** - * EventToken position. - * @member {string} position - * @memberof query.EventToken + * VStreamResultsRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof binlogdata.VStreamResultsRequest * @instance */ - EventToken.prototype.position = ""; + VStreamResultsRequest.prototype.target = null; /** - * Creates a new EventToken instance using the specified properties. + * VStreamResultsRequest query. + * @member {string} query + * @memberof binlogdata.VStreamResultsRequest + * @instance + */ + VStreamResultsRequest.prototype.query = ""; + + /** + * Creates a new VStreamResultsRequest instance using the specified properties. * @function create - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static - * @param {query.IEventToken=} [properties] Properties to set - * @returns {query.EventToken} EventToken instance + * @param {binlogdata.IVStreamResultsRequest=} [properties] Properties to set + * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest instance */ - EventToken.create = function create(properties) { - return new EventToken(properties); + VStreamResultsRequest.create = function create(properties) { + return new VStreamResultsRequest(properties); }; /** - * Encodes the specified EventToken message. Does not implicitly {@link query.EventToken.verify|verify} messages. + * Encodes the specified VStreamResultsRequest message. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. * @function encode - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static - * @param {query.IEventToken} message EventToken message or plain object to encode + * @param {binlogdata.IVStreamResultsRequest} message VStreamResultsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EventToken.encode = function encode(message, writer) { + VStreamResultsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.timestamp != null && Object.hasOwnProperty.call(message, "timestamp")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.timestamp); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.position); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.query); return writer; }; /** - * Encodes the specified EventToken message, length delimited. Does not implicitly {@link query.EventToken.verify|verify} messages. + * Encodes the specified VStreamResultsRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static - * @param {query.IEventToken} message EventToken message or plain object to encode + * @param {binlogdata.IVStreamResultsRequest} message VStreamResultsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EventToken.encodeDelimited = function encodeDelimited(message, writer) { + VStreamResultsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an EventToken message from the specified reader or buffer. + * Decodes a VStreamResultsRequest message from the specified reader or buffer. * @function decode - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.EventToken} EventToken + * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EventToken.decode = function decode(reader, length) { + VStreamResultsRequest.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.query.EventToken(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResultsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.timestamp = reader.int64(); + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); break; } case 2: { - message.shard = reader.string(); + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); break; } case 3: { - message.position = reader.string(); + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + case 4: { + message.query = reader.string(); break; } default: @@ -73754,308 +74801,166 @@ export const query = $root.query = (() => { }; /** - * Decodes an EventToken message from the specified reader or buffer, length delimited. + * Decodes a VStreamResultsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.EventToken} EventToken + * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EventToken.decodeDelimited = function decodeDelimited(reader) { + VStreamResultsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an EventToken message. + * Verifies a VStreamResultsRequest message. * @function verify - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - EventToken.verify = function verify(message) { + VStreamResultsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.timestamp != null && message.hasOwnProperty("timestamp")) - if (!$util.isInteger(message.timestamp) && !(message.timestamp && $util.isInteger(message.timestamp.low) && $util.isInteger(message.timestamp.high))) - return "timestamp: integer|Long expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) + if (!$util.isString(message.query)) + return "query: string expected"; return null; }; /** - * Creates an EventToken message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResultsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static * @param {Object.} object Plain object - * @returns {query.EventToken} EventToken + * @returns {binlogdata.VStreamResultsRequest} VStreamResultsRequest */ - EventToken.fromObject = function fromObject(object) { - if (object instanceof $root.query.EventToken) + VStreamResultsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamResultsRequest) return object; - let message = new $root.query.EventToken(); - if (object.timestamp != null) - if ($util.Long) - (message.timestamp = $util.Long.fromValue(object.timestamp)).unsigned = false; - else if (typeof object.timestamp === "string") - message.timestamp = parseInt(object.timestamp, 10); - else if (typeof object.timestamp === "number") - message.timestamp = object.timestamp; - else if (typeof object.timestamp === "object") - message.timestamp = new $util.LongBits(object.timestamp.low >>> 0, object.timestamp.high >>> 0).toNumber(); - if (object.shard != null) - message.shard = String(object.shard); - if (object.position != null) - message.position = String(object.position); + let message = new $root.binlogdata.VStreamResultsRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".binlogdata.VStreamResultsRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".binlogdata.VStreamResultsRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".binlogdata.VStreamResultsRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) + message.query = String(object.query); return message; }; /** - * Creates a plain object from an EventToken message. Also converts values to other types if specified. + * Creates a plain object from a VStreamResultsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static - * @param {query.EventToken} message EventToken + * @param {binlogdata.VStreamResultsRequest} message VStreamResultsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - EventToken.toObject = function toObject(message, options) { + VStreamResultsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.timestamp = options.longs === String ? "0" : 0; - object.shard = ""; - object.position = ""; + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = ""; } - if (message.timestamp != null && message.hasOwnProperty("timestamp")) - if (typeof message.timestamp === "number") - object.timestamp = options.longs === String ? String(message.timestamp) : message.timestamp; - else - object.timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.timestamp) : options.longs === Number ? new $util.LongBits(message.timestamp.low >>> 0, message.timestamp.high >>> 0).toNumber() : message.timestamp; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = message.query; return object; }; /** - * Converts this EventToken to JSON. + * Converts this VStreamResultsRequest to JSON. * @function toJSON - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @instance * @returns {Object.} JSON object */ - EventToken.prototype.toJSON = function toJSON() { + VStreamResultsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for EventToken + * Gets the default type url for VStreamResultsRequest * @function getTypeUrl - * @memberof query.EventToken + * @memberof binlogdata.VStreamResultsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - EventToken.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamResultsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.EventToken"; + return typeUrlPrefix + "/binlogdata.VStreamResultsRequest"; }; - return EventToken; - })(); - - /** - * MySqlFlag enum. - * @name query.MySqlFlag - * @enum {number} - * @property {number} EMPTY=0 EMPTY value - * @property {number} NOT_NULL_FLAG=1 NOT_NULL_FLAG value - * @property {number} PRI_KEY_FLAG=2 PRI_KEY_FLAG value - * @property {number} UNIQUE_KEY_FLAG=4 UNIQUE_KEY_FLAG value - * @property {number} MULTIPLE_KEY_FLAG=8 MULTIPLE_KEY_FLAG value - * @property {number} BLOB_FLAG=16 BLOB_FLAG value - * @property {number} UNSIGNED_FLAG=32 UNSIGNED_FLAG value - * @property {number} ZEROFILL_FLAG=64 ZEROFILL_FLAG value - * @property {number} BINARY_FLAG=128 BINARY_FLAG value - * @property {number} ENUM_FLAG=256 ENUM_FLAG value - * @property {number} AUTO_INCREMENT_FLAG=512 AUTO_INCREMENT_FLAG value - * @property {number} TIMESTAMP_FLAG=1024 TIMESTAMP_FLAG value - * @property {number} SET_FLAG=2048 SET_FLAG value - * @property {number} NO_DEFAULT_VALUE_FLAG=4096 NO_DEFAULT_VALUE_FLAG value - * @property {number} ON_UPDATE_NOW_FLAG=8192 ON_UPDATE_NOW_FLAG value - * @property {number} NUM_FLAG=32768 NUM_FLAG value - * @property {number} PART_KEY_FLAG=16384 PART_KEY_FLAG value - * @property {number} GROUP_FLAG=32768 GROUP_FLAG value - * @property {number} UNIQUE_FLAG=65536 UNIQUE_FLAG value - * @property {number} BINCMP_FLAG=131072 BINCMP_FLAG value - */ - query.MySqlFlag = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "EMPTY"] = 0; - values[valuesById[1] = "NOT_NULL_FLAG"] = 1; - values[valuesById[2] = "PRI_KEY_FLAG"] = 2; - values[valuesById[4] = "UNIQUE_KEY_FLAG"] = 4; - values[valuesById[8] = "MULTIPLE_KEY_FLAG"] = 8; - values[valuesById[16] = "BLOB_FLAG"] = 16; - values[valuesById[32] = "UNSIGNED_FLAG"] = 32; - values[valuesById[64] = "ZEROFILL_FLAG"] = 64; - values[valuesById[128] = "BINARY_FLAG"] = 128; - values[valuesById[256] = "ENUM_FLAG"] = 256; - values[valuesById[512] = "AUTO_INCREMENT_FLAG"] = 512; - values[valuesById[1024] = "TIMESTAMP_FLAG"] = 1024; - values[valuesById[2048] = "SET_FLAG"] = 2048; - values[valuesById[4096] = "NO_DEFAULT_VALUE_FLAG"] = 4096; - values[valuesById[8192] = "ON_UPDATE_NOW_FLAG"] = 8192; - values[valuesById[32768] = "NUM_FLAG"] = 32768; - values[valuesById[16384] = "PART_KEY_FLAG"] = 16384; - values["GROUP_FLAG"] = 32768; - values[valuesById[65536] = "UNIQUE_FLAG"] = 65536; - values[valuesById[131072] = "BINCMP_FLAG"] = 131072; - return values; - })(); - - /** - * Flag enum. - * @name query.Flag - * @enum {number} - * @property {number} NONE=0 NONE value - * @property {number} ISINTEGRAL=256 ISINTEGRAL value - * @property {number} ISUNSIGNED=512 ISUNSIGNED value - * @property {number} ISFLOAT=1024 ISFLOAT value - * @property {number} ISQUOTED=2048 ISQUOTED value - * @property {number} ISTEXT=4096 ISTEXT value - * @property {number} ISBINARY=8192 ISBINARY value - */ - query.Flag = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "NONE"] = 0; - values[valuesById[256] = "ISINTEGRAL"] = 256; - values[valuesById[512] = "ISUNSIGNED"] = 512; - values[valuesById[1024] = "ISFLOAT"] = 1024; - values[valuesById[2048] = "ISQUOTED"] = 2048; - values[valuesById[4096] = "ISTEXT"] = 4096; - values[valuesById[8192] = "ISBINARY"] = 8192; - return values; - })(); - - /** - * Type enum. - * @name query.Type - * @enum {number} - * @property {number} NULL_TYPE=0 NULL_TYPE value - * @property {number} INT8=257 INT8 value - * @property {number} UINT8=770 UINT8 value - * @property {number} INT16=259 INT16 value - * @property {number} UINT16=772 UINT16 value - * @property {number} INT24=261 INT24 value - * @property {number} UINT24=774 UINT24 value - * @property {number} INT32=263 INT32 value - * @property {number} UINT32=776 UINT32 value - * @property {number} INT64=265 INT64 value - * @property {number} UINT64=778 UINT64 value - * @property {number} FLOAT32=1035 FLOAT32 value - * @property {number} FLOAT64=1036 FLOAT64 value - * @property {number} TIMESTAMP=2061 TIMESTAMP value - * @property {number} DATE=2062 DATE value - * @property {number} TIME=2063 TIME value - * @property {number} DATETIME=2064 DATETIME value - * @property {number} YEAR=785 YEAR value - * @property {number} DECIMAL=18 DECIMAL value - * @property {number} TEXT=6163 TEXT value - * @property {number} BLOB=10260 BLOB value - * @property {number} VARCHAR=6165 VARCHAR value - * @property {number} VARBINARY=10262 VARBINARY value - * @property {number} CHAR=6167 CHAR value - * @property {number} BINARY=10264 BINARY value - * @property {number} BIT=2073 BIT value - * @property {number} ENUM=2074 ENUM value - * @property {number} SET=2075 SET value - * @property {number} TUPLE=28 TUPLE value - * @property {number} GEOMETRY=2077 GEOMETRY value - * @property {number} JSON=2078 JSON value - * @property {number} EXPRESSION=31 EXPRESSION value - * @property {number} HEXNUM=4128 HEXNUM value - * @property {number} HEXVAL=4129 HEXVAL value - * @property {number} BITNUM=4130 BITNUM value - */ - query.Type = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "NULL_TYPE"] = 0; - values[valuesById[257] = "INT8"] = 257; - values[valuesById[770] = "UINT8"] = 770; - values[valuesById[259] = "INT16"] = 259; - values[valuesById[772] = "UINT16"] = 772; - values[valuesById[261] = "INT24"] = 261; - values[valuesById[774] = "UINT24"] = 774; - values[valuesById[263] = "INT32"] = 263; - values[valuesById[776] = "UINT32"] = 776; - values[valuesById[265] = "INT64"] = 265; - values[valuesById[778] = "UINT64"] = 778; - values[valuesById[1035] = "FLOAT32"] = 1035; - values[valuesById[1036] = "FLOAT64"] = 1036; - values[valuesById[2061] = "TIMESTAMP"] = 2061; - values[valuesById[2062] = "DATE"] = 2062; - values[valuesById[2063] = "TIME"] = 2063; - values[valuesById[2064] = "DATETIME"] = 2064; - values[valuesById[785] = "YEAR"] = 785; - values[valuesById[18] = "DECIMAL"] = 18; - values[valuesById[6163] = "TEXT"] = 6163; - values[valuesById[10260] = "BLOB"] = 10260; - values[valuesById[6165] = "VARCHAR"] = 6165; - values[valuesById[10262] = "VARBINARY"] = 10262; - values[valuesById[6167] = "CHAR"] = 6167; - values[valuesById[10264] = "BINARY"] = 10264; - values[valuesById[2073] = "BIT"] = 2073; - values[valuesById[2074] = "ENUM"] = 2074; - values[valuesById[2075] = "SET"] = 2075; - values[valuesById[28] = "TUPLE"] = 28; - values[valuesById[2077] = "GEOMETRY"] = 2077; - values[valuesById[2078] = "JSON"] = 2078; - values[valuesById[31] = "EXPRESSION"] = 31; - values[valuesById[4128] = "HEXNUM"] = 4128; - values[valuesById[4129] = "HEXVAL"] = 4129; - values[valuesById[4130] = "BITNUM"] = 4130; - return values; + return VStreamResultsRequest; })(); - query.Value = (function() { + binlogdata.VStreamResultsResponse = (function() { /** - * Properties of a Value. - * @memberof query - * @interface IValue - * @property {query.Type|null} [type] Value type - * @property {Uint8Array|null} [value] Value value + * Properties of a VStreamResultsResponse. + * @memberof binlogdata + * @interface IVStreamResultsResponse + * @property {Array.|null} [fields] VStreamResultsResponse fields + * @property {string|null} [gtid] VStreamResultsResponse gtid + * @property {Array.|null} [rows] VStreamResultsResponse rows */ /** - * Constructs a new Value. - * @memberof query - * @classdesc Represents a Value. - * @implements IValue + * Constructs a new VStreamResultsResponse. + * @memberof binlogdata + * @classdesc Represents a VStreamResultsResponse. + * @implements IVStreamResultsResponse * @constructor - * @param {query.IValue=} [properties] Properties to set + * @param {binlogdata.IVStreamResultsResponse=} [properties] Properties to set */ - function Value(properties) { + function VStreamResultsResponse(properties) { + this.fields = []; + this.rows = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -74063,89 +74968,109 @@ export const query = $root.query = (() => { } /** - * Value type. - * @member {query.Type} type - * @memberof query.Value + * VStreamResultsResponse fields. + * @member {Array.} fields + * @memberof binlogdata.VStreamResultsResponse * @instance */ - Value.prototype.type = 0; + VStreamResultsResponse.prototype.fields = $util.emptyArray; /** - * Value value. - * @member {Uint8Array} value - * @memberof query.Value + * VStreamResultsResponse gtid. + * @member {string} gtid + * @memberof binlogdata.VStreamResultsResponse * @instance */ - Value.prototype.value = $util.newBuffer([]); + VStreamResultsResponse.prototype.gtid = ""; /** - * Creates a new Value instance using the specified properties. + * VStreamResultsResponse rows. + * @member {Array.} rows + * @memberof binlogdata.VStreamResultsResponse + * @instance + */ + VStreamResultsResponse.prototype.rows = $util.emptyArray; + + /** + * Creates a new VStreamResultsResponse instance using the specified properties. * @function create - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static - * @param {query.IValue=} [properties] Properties to set - * @returns {query.Value} Value instance + * @param {binlogdata.IVStreamResultsResponse=} [properties] Properties to set + * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse instance */ - Value.create = function create(properties) { - return new Value(properties); + VStreamResultsResponse.create = function create(properties) { + return new VStreamResultsResponse(properties); }; /** - * Encodes the specified Value message. Does not implicitly {@link query.Value.verify|verify} messages. + * Encodes the specified VStreamResultsResponse message. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. * @function encode - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static - * @param {query.IValue} message Value message or plain object to encode + * @param {binlogdata.IVStreamResultsResponse} message VStreamResultsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Value.encode = function encode(message, writer) { + VStreamResultsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); - if (message.value != null && Object.hasOwnProperty.call(message, "value")) - writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.value); + if (message.fields != null && message.fields.length) + for (let i = 0; i < message.fields.length; ++i) + $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.gtid); + if (message.rows != null && message.rows.length) + for (let i = 0; i < message.rows.length; ++i) + $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified Value message, length delimited. Does not implicitly {@link query.Value.verify|verify} messages. + * Encodes the specified VStreamResultsResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamResultsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static - * @param {query.IValue} message Value message or plain object to encode + * @param {binlogdata.IVStreamResultsResponse} message VStreamResultsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Value.encodeDelimited = function encodeDelimited(message, writer) { + VStreamResultsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Value message from the specified reader or buffer. + * Decodes a VStreamResultsResponse message from the specified reader or buffer. * @function decode - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.Value} Value + * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Value.decode = function decode(reader, length) { + VStreamResultsResponse.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.query.Value(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamResultsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.type = reader.int32(); + if (!(message.fields && message.fields.length)) + message.fields = []; + message.fields.push($root.query.Field.decode(reader, reader.uint32())); break; } - case 2: { - message.value = reader.bytes(); + case 3: { + message.gtid = reader.string(); + break; + } + case 4: { + if (!(message.rows && message.rows.length)) + message.rows = []; + message.rows.push($root.query.Row.decode(reader, reader.uint32())); break; } default: @@ -74157,327 +75082,190 @@ export const query = $root.query = (() => { }; /** - * Decodes a Value message from the specified reader or buffer, length delimited. + * Decodes a VStreamResultsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.Value} Value + * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Value.decodeDelimited = function decodeDelimited(reader) { + VStreamResultsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Value message. + * Verifies a VStreamResultsResponse message. * @function verify - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Value.verify = function verify(message) { + VStreamResultsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "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; + if (message.fields != null && message.hasOwnProperty("fields")) { + if (!Array.isArray(message.fields)) + return "fields: array expected"; + for (let i = 0; i < message.fields.length; ++i) { + let error = $root.query.Field.verify(message.fields[i]); + if (error) + return "fields." + error; } - if (message.value != null && message.hasOwnProperty("value")) - if (!(message.value && typeof message.value.length === "number" || $util.isString(message.value))) - return "value: buffer expected"; + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + if (!$util.isString(message.gtid)) + return "gtid: string expected"; + if (message.rows != null && message.hasOwnProperty("rows")) { + if (!Array.isArray(message.rows)) + return "rows: array expected"; + for (let i = 0; i < message.rows.length; ++i) { + let error = $root.query.Row.verify(message.rows[i]); + if (error) + return "rows." + error; + } + } return null; }; /** - * Creates a Value message from a plain object. Also converts values to their respective internal types. + * Creates a VStreamResultsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static * @param {Object.} object Plain object - * @returns {query.Value} Value + * @returns {binlogdata.VStreamResultsResponse} VStreamResultsResponse */ - Value.fromObject = function fromObject(object) { - if (object instanceof $root.query.Value) + VStreamResultsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamResultsResponse) return object; - let message = new $root.query.Value(); - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; + let message = new $root.binlogdata.VStreamResultsResponse(); + if (object.fields) { + if (!Array.isArray(object.fields)) + throw TypeError(".binlogdata.VStreamResultsResponse.fields: array expected"); + message.fields = []; + for (let i = 0; i < object.fields.length; ++i) { + if (typeof object.fields[i] !== "object") + throw TypeError(".binlogdata.VStreamResultsResponse.fields: object expected"); + message.fields[i] = $root.query.Field.fromObject(object.fields[i]); + } + } + if (object.gtid != null) + message.gtid = String(object.gtid); + if (object.rows) { + if (!Array.isArray(object.rows)) + throw TypeError(".binlogdata.VStreamResultsResponse.rows: array expected"); + message.rows = []; + for (let i = 0; i < object.rows.length; ++i) { + if (typeof object.rows[i] !== "object") + throw TypeError(".binlogdata.VStreamResultsResponse.rows: object expected"); + message.rows[i] = $root.query.Row.fromObject(object.rows[i]); } - break; - case "NULL_TYPE": - case 0: - message.type = 0; - break; - case "INT8": - case 257: - message.type = 257; - break; - case "UINT8": - case 770: - message.type = 770; - break; - case "INT16": - case 259: - message.type = 259; - break; - case "UINT16": - case 772: - message.type = 772; - break; - case "INT24": - case 261: - message.type = 261; - break; - case "UINT24": - case 774: - message.type = 774; - break; - case "INT32": - case 263: - message.type = 263; - break; - case "UINT32": - case 776: - message.type = 776; - break; - case "INT64": - case 265: - message.type = 265; - break; - case "UINT64": - case 778: - message.type = 778; - break; - case "FLOAT32": - case 1035: - message.type = 1035; - break; - case "FLOAT64": - case 1036: - message.type = 1036; - break; - case "TIMESTAMP": - case 2061: - message.type = 2061; - break; - case "DATE": - case 2062: - message.type = 2062; - break; - case "TIME": - case 2063: - message.type = 2063; - break; - case "DATETIME": - case 2064: - message.type = 2064; - break; - case "YEAR": - case 785: - message.type = 785; - break; - case "DECIMAL": - case 18: - message.type = 18; - break; - case "TEXT": - case 6163: - message.type = 6163; - break; - case "BLOB": - case 10260: - message.type = 10260; - break; - case "VARCHAR": - case 6165: - message.type = 6165; - break; - case "VARBINARY": - case 10262: - message.type = 10262; - break; - case "CHAR": - case 6167: - message.type = 6167; - break; - case "BINARY": - case 10264: - message.type = 10264; - break; - case "BIT": - case 2073: - message.type = 2073; - break; - case "ENUM": - case 2074: - message.type = 2074; - break; - case "SET": - case 2075: - message.type = 2075; - break; - case "TUPLE": - case 28: - message.type = 28; - break; - case "GEOMETRY": - case 2077: - message.type = 2077; - break; - case "JSON": - case 2078: - message.type = 2078; - break; - case "EXPRESSION": - case 31: - message.type = 31; - break; - case "HEXNUM": - case 4128: - message.type = 4128; - break; - case "HEXVAL": - case 4129: - message.type = 4129; - break; - case "BITNUM": - case 4130: - message.type = 4130; - break; } - if (object.value != null) - if (typeof object.value === "string") - $util.base64.decode(object.value, message.value = $util.newBuffer($util.base64.length(object.value)), 0); - else if (object.value.length >= 0) - message.value = object.value; return message; }; /** - * Creates a plain object from a Value message. Also converts values to other types if specified. + * Creates a plain object from a VStreamResultsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static - * @param {query.Value} message Value + * @param {binlogdata.VStreamResultsResponse} message VStreamResultsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Value.toObject = function toObject(message, options) { + VStreamResultsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.type = options.enums === String ? "NULL_TYPE" : 0; - if (options.bytes === String) - object.value = ""; - else { - object.value = []; - if (options.bytes !== Array) - object.value = $util.newBuffer(object.value); - } + if (options.arrays || options.defaults) { + object.fields = []; + object.rows = []; + } + if (options.defaults) + object.gtid = ""; + if (message.fields && message.fields.length) { + object.fields = []; + for (let j = 0; j < message.fields.length; ++j) + object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + object.gtid = message.gtid; + if (message.rows && message.rows.length) { + object.rows = []; + for (let j = 0; j < message.rows.length; ++j) + object.rows[j] = $root.query.Row.toObject(message.rows[j], options); } - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; - if (message.value != null && message.hasOwnProperty("value")) - object.value = options.bytes === String ? $util.base64.encode(message.value, 0, message.value.length) : options.bytes === Array ? Array.prototype.slice.call(message.value) : message.value; return object; }; /** - * Converts this Value to JSON. + * Converts this VStreamResultsResponse to JSON. * @function toJSON - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @instance * @returns {Object.} JSON object */ - Value.prototype.toJSON = function toJSON() { + VStreamResultsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Value + * Gets the default type url for VStreamResultsResponse * @function getTypeUrl - * @memberof query.Value + * @memberof binlogdata.VStreamResultsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Value.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VStreamResultsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.Value"; + return typeUrlPrefix + "/binlogdata.VStreamResultsResponse"; }; - return Value; + return VStreamResultsResponse; })(); - query.BindVariable = (function() { + return binlogdata; +})(); + +export const vtrpc = $root.vtrpc = (() => { + + /** + * Namespace vtrpc. + * @exports vtrpc + * @namespace + */ + const vtrpc = {}; + + vtrpc.CallerID = (function() { /** - * Properties of a BindVariable. - * @memberof query - * @interface IBindVariable - * @property {query.Type|null} [type] BindVariable type - * @property {Uint8Array|null} [value] BindVariable value - * @property {Array.|null} [values] BindVariable values + * Properties of a CallerID. + * @memberof vtrpc + * @interface ICallerID + * @property {string|null} [principal] CallerID principal + * @property {string|null} [component] CallerID component + * @property {string|null} [subcomponent] CallerID subcomponent + * @property {Array.|null} [groups] CallerID groups */ /** - * Constructs a new BindVariable. - * @memberof query - * @classdesc Represents a BindVariable. - * @implements IBindVariable + * Constructs a new CallerID. + * @memberof vtrpc + * @classdesc Represents a CallerID. + * @implements ICallerID * @constructor - * @param {query.IBindVariable=} [properties] Properties to set + * @param {vtrpc.ICallerID=} [properties] Properties to set */ - function BindVariable(properties) { - this.values = []; + function CallerID(properties) { + this.groups = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -74485,106 +75273,120 @@ export const query = $root.query = (() => { } /** - * BindVariable type. - * @member {query.Type} type - * @memberof query.BindVariable + * CallerID principal. + * @member {string} principal + * @memberof vtrpc.CallerID * @instance */ - BindVariable.prototype.type = 0; + CallerID.prototype.principal = ""; /** - * BindVariable value. - * @member {Uint8Array} value - * @memberof query.BindVariable + * CallerID component. + * @member {string} component + * @memberof vtrpc.CallerID * @instance */ - BindVariable.prototype.value = $util.newBuffer([]); + CallerID.prototype.component = ""; /** - * BindVariable values. - * @member {Array.} values - * @memberof query.BindVariable + * CallerID subcomponent. + * @member {string} subcomponent + * @memberof vtrpc.CallerID * @instance */ - BindVariable.prototype.values = $util.emptyArray; + CallerID.prototype.subcomponent = ""; /** - * Creates a new BindVariable instance using the specified properties. + * CallerID groups. + * @member {Array.} groups + * @memberof vtrpc.CallerID + * @instance + */ + CallerID.prototype.groups = $util.emptyArray; + + /** + * Creates a new CallerID instance using the specified properties. * @function create - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static - * @param {query.IBindVariable=} [properties] Properties to set - * @returns {query.BindVariable} BindVariable instance + * @param {vtrpc.ICallerID=} [properties] Properties to set + * @returns {vtrpc.CallerID} CallerID instance */ - BindVariable.create = function create(properties) { - return new BindVariable(properties); + CallerID.create = function create(properties) { + return new CallerID(properties); }; /** - * Encodes the specified BindVariable message. Does not implicitly {@link query.BindVariable.verify|verify} messages. + * Encodes the specified CallerID message. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. * @function encode - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static - * @param {query.IBindVariable} message BindVariable message or plain object to encode + * @param {vtrpc.ICallerID} message CallerID message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BindVariable.encode = function encode(message, writer) { + CallerID.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); - if (message.value != null && Object.hasOwnProperty.call(message, "value")) - writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.value); - if (message.values != null && message.values.length) - for (let i = 0; i < message.values.length; ++i) - $root.query.Value.encode(message.values[i], writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.principal != null && Object.hasOwnProperty.call(message, "principal")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.principal); + if (message.component != null && Object.hasOwnProperty.call(message, "component")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.component); + if (message.subcomponent != null && Object.hasOwnProperty.call(message, "subcomponent")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.subcomponent); + if (message.groups != null && message.groups.length) + for (let i = 0; i < message.groups.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.groups[i]); return writer; }; /** - * Encodes the specified BindVariable message, length delimited. Does not implicitly {@link query.BindVariable.verify|verify} messages. + * Encodes the specified CallerID message, length delimited. Does not implicitly {@link vtrpc.CallerID.verify|verify} messages. * @function encodeDelimited - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static - * @param {query.IBindVariable} message BindVariable message or plain object to encode + * @param {vtrpc.ICallerID} message CallerID message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BindVariable.encodeDelimited = function encodeDelimited(message, writer) { + CallerID.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BindVariable message from the specified reader or buffer. + * Decodes a CallerID message from the specified reader or buffer. * @function decode - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BindVariable} BindVariable + * @returns {vtrpc.CallerID} CallerID * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BindVariable.decode = function decode(reader, length) { + CallerID.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.query.BindVariable(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtrpc.CallerID(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.type = reader.int32(); + message.principal = reader.string(); break; } case 2: { - message.value = reader.bytes(); + message.component = reader.string(); break; } case 3: { - if (!(message.values && message.values.length)) - message.values = []; - message.values.push($root.query.Value.decode(reader, reader.uint32())); + message.subcomponent = reader.string(); + break; + } + case 4: { + if (!(message.groups && message.groups.length)) + message.groups = []; + message.groups.push(reader.string()); break; } default: @@ -74596,352 +75398,209 @@ export const query = $root.query = (() => { }; /** - * Decodes a BindVariable message from the specified reader or buffer, length delimited. + * Decodes a CallerID message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BindVariable} BindVariable + * @returns {vtrpc.CallerID} CallerID * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BindVariable.decodeDelimited = function decodeDelimited(reader) { + CallerID.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BindVariable message. + * Verifies a CallerID message. * @function verify - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BindVariable.verify = function verify(message) { + CallerID.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "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; - } - if (message.value != null && message.hasOwnProperty("value")) - if (!(message.value && typeof message.value.length === "number" || $util.isString(message.value))) - return "value: buffer expected"; - if (message.values != null && message.hasOwnProperty("values")) { - if (!Array.isArray(message.values)) - return "values: array expected"; - for (let i = 0; i < message.values.length; ++i) { - let error = $root.query.Value.verify(message.values[i]); - if (error) - return "values." + error; - } + if (message.principal != null && message.hasOwnProperty("principal")) + if (!$util.isString(message.principal)) + return "principal: string expected"; + if (message.component != null && message.hasOwnProperty("component")) + if (!$util.isString(message.component)) + return "component: string expected"; + if (message.subcomponent != null && message.hasOwnProperty("subcomponent")) + if (!$util.isString(message.subcomponent)) + return "subcomponent: string expected"; + if (message.groups != null && message.hasOwnProperty("groups")) { + if (!Array.isArray(message.groups)) + return "groups: array expected"; + for (let i = 0; i < message.groups.length; ++i) + if (!$util.isString(message.groups[i])) + return "groups: string[] expected"; } return null; }; /** - * Creates a BindVariable message from a plain object. Also converts values to their respective internal types. + * Creates a CallerID message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static * @param {Object.} object Plain object - * @returns {query.BindVariable} BindVariable + * @returns {vtrpc.CallerID} CallerID */ - BindVariable.fromObject = function fromObject(object) { - if (object instanceof $root.query.BindVariable) + CallerID.fromObject = function fromObject(object) { + if (object instanceof $root.vtrpc.CallerID) return object; - let message = new $root.query.BindVariable(); - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; - } - break; - case "NULL_TYPE": - case 0: - message.type = 0; - break; - case "INT8": - case 257: - message.type = 257; - break; - case "UINT8": - case 770: - message.type = 770; - break; - case "INT16": - case 259: - message.type = 259; - break; - case "UINT16": - case 772: - message.type = 772; - break; - case "INT24": - case 261: - message.type = 261; - break; - case "UINT24": - case 774: - message.type = 774; - break; - case "INT32": - case 263: - message.type = 263; - break; - case "UINT32": - case 776: - message.type = 776; - break; - case "INT64": - case 265: - message.type = 265; - break; - case "UINT64": - case 778: - message.type = 778; - break; - case "FLOAT32": - case 1035: - message.type = 1035; - break; - case "FLOAT64": - case 1036: - message.type = 1036; - break; - case "TIMESTAMP": - case 2061: - message.type = 2061; - break; - case "DATE": - case 2062: - message.type = 2062; - break; - case "TIME": - case 2063: - message.type = 2063; - break; - case "DATETIME": - case 2064: - message.type = 2064; - break; - case "YEAR": - case 785: - message.type = 785; - break; - case "DECIMAL": - case 18: - message.type = 18; - break; - case "TEXT": - case 6163: - message.type = 6163; - break; - case "BLOB": - case 10260: - message.type = 10260; - break; - case "VARCHAR": - case 6165: - message.type = 6165; - break; - case "VARBINARY": - case 10262: - message.type = 10262; - break; - case "CHAR": - case 6167: - message.type = 6167; - break; - case "BINARY": - case 10264: - message.type = 10264; - break; - case "BIT": - case 2073: - message.type = 2073; - break; - case "ENUM": - case 2074: - message.type = 2074; - break; - case "SET": - case 2075: - message.type = 2075; - break; - case "TUPLE": - case 28: - message.type = 28; - break; - case "GEOMETRY": - case 2077: - message.type = 2077; - break; - case "JSON": - case 2078: - message.type = 2078; - break; - case "EXPRESSION": - case 31: - message.type = 31; - break; - case "HEXNUM": - case 4128: - message.type = 4128; - break; - case "HEXVAL": - case 4129: - message.type = 4129; - break; - case "BITNUM": - case 4130: - message.type = 4130; - break; - } - if (object.value != null) - if (typeof object.value === "string") - $util.base64.decode(object.value, message.value = $util.newBuffer($util.base64.length(object.value)), 0); - else if (object.value.length >= 0) - message.value = object.value; - if (object.values) { - if (!Array.isArray(object.values)) - throw TypeError(".query.BindVariable.values: array expected"); - message.values = []; - for (let i = 0; i < object.values.length; ++i) { - if (typeof object.values[i] !== "object") - throw TypeError(".query.BindVariable.values: object expected"); - message.values[i] = $root.query.Value.fromObject(object.values[i]); - } + let message = new $root.vtrpc.CallerID(); + if (object.principal != null) + message.principal = String(object.principal); + if (object.component != null) + message.component = String(object.component); + if (object.subcomponent != null) + message.subcomponent = String(object.subcomponent); + if (object.groups) { + if (!Array.isArray(object.groups)) + throw TypeError(".vtrpc.CallerID.groups: array expected"); + message.groups = []; + for (let i = 0; i < object.groups.length; ++i) + message.groups[i] = String(object.groups[i]); } return message; }; /** - * Creates a plain object from a BindVariable message. Also converts values to other types if specified. + * Creates a plain object from a CallerID message. Also converts values to other types if specified. * @function toObject - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static - * @param {query.BindVariable} message BindVariable + * @param {vtrpc.CallerID} message CallerID * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BindVariable.toObject = function toObject(message, options) { + CallerID.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.values = []; + object.groups = []; if (options.defaults) { - object.type = options.enums === String ? "NULL_TYPE" : 0; - if (options.bytes === String) - object.value = ""; - else { - object.value = []; - if (options.bytes !== Array) - object.value = $util.newBuffer(object.value); - } + object.principal = ""; + object.component = ""; + object.subcomponent = ""; } - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; - if (message.value != null && message.hasOwnProperty("value")) - object.value = options.bytes === String ? $util.base64.encode(message.value, 0, message.value.length) : options.bytes === Array ? Array.prototype.slice.call(message.value) : message.value; - if (message.values && message.values.length) { - object.values = []; - for (let j = 0; j < message.values.length; ++j) - object.values[j] = $root.query.Value.toObject(message.values[j], options); + if (message.principal != null && message.hasOwnProperty("principal")) + object.principal = message.principal; + if (message.component != null && message.hasOwnProperty("component")) + object.component = message.component; + if (message.subcomponent != null && message.hasOwnProperty("subcomponent")) + object.subcomponent = message.subcomponent; + if (message.groups && message.groups.length) { + object.groups = []; + for (let j = 0; j < message.groups.length; ++j) + object.groups[j] = message.groups[j]; } return object; }; /** - * Converts this BindVariable to JSON. + * Converts this CallerID to JSON. * @function toJSON - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @instance * @returns {Object.} JSON object */ - BindVariable.prototype.toJSON = function toJSON() { + CallerID.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BindVariable + * Gets the default type url for CallerID * @function getTypeUrl - * @memberof query.BindVariable + * @memberof vtrpc.CallerID * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BindVariable.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CallerID.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BindVariable"; + return typeUrlPrefix + "/vtrpc.CallerID"; }; - return BindVariable; + return CallerID; })(); - query.BoundQuery = (function() { + /** + * Code enum. + * @name vtrpc.Code + * @enum {number} + * @property {number} OK=0 OK value + * @property {number} CANCELED=1 CANCELED value + * @property {number} UNKNOWN=2 UNKNOWN value + * @property {number} INVALID_ARGUMENT=3 INVALID_ARGUMENT value + * @property {number} DEADLINE_EXCEEDED=4 DEADLINE_EXCEEDED value + * @property {number} NOT_FOUND=5 NOT_FOUND value + * @property {number} ALREADY_EXISTS=6 ALREADY_EXISTS value + * @property {number} PERMISSION_DENIED=7 PERMISSION_DENIED value + * @property {number} RESOURCE_EXHAUSTED=8 RESOURCE_EXHAUSTED value + * @property {number} FAILED_PRECONDITION=9 FAILED_PRECONDITION value + * @property {number} ABORTED=10 ABORTED value + * @property {number} OUT_OF_RANGE=11 OUT_OF_RANGE value + * @property {number} UNIMPLEMENTED=12 UNIMPLEMENTED value + * @property {number} INTERNAL=13 INTERNAL value + * @property {number} UNAVAILABLE=14 UNAVAILABLE value + * @property {number} DATA_LOSS=15 DATA_LOSS value + * @property {number} UNAUTHENTICATED=16 UNAUTHENTICATED value + * @property {number} CLUSTER_EVENT=17 CLUSTER_EVENT value + * @property {number} READ_ONLY=18 READ_ONLY value + */ + vtrpc.Code = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "OK"] = 0; + values[valuesById[1] = "CANCELED"] = 1; + values[valuesById[2] = "UNKNOWN"] = 2; + values[valuesById[3] = "INVALID_ARGUMENT"] = 3; + values[valuesById[4] = "DEADLINE_EXCEEDED"] = 4; + values[valuesById[5] = "NOT_FOUND"] = 5; + values[valuesById[6] = "ALREADY_EXISTS"] = 6; + values[valuesById[7] = "PERMISSION_DENIED"] = 7; + values[valuesById[8] = "RESOURCE_EXHAUSTED"] = 8; + values[valuesById[9] = "FAILED_PRECONDITION"] = 9; + values[valuesById[10] = "ABORTED"] = 10; + values[valuesById[11] = "OUT_OF_RANGE"] = 11; + values[valuesById[12] = "UNIMPLEMENTED"] = 12; + values[valuesById[13] = "INTERNAL"] = 13; + values[valuesById[14] = "UNAVAILABLE"] = 14; + values[valuesById[15] = "DATA_LOSS"] = 15; + values[valuesById[16] = "UNAUTHENTICATED"] = 16; + values[valuesById[17] = "CLUSTER_EVENT"] = 17; + values[valuesById[18] = "READ_ONLY"] = 18; + return values; + })(); + + vtrpc.RPCError = (function() { /** - * Properties of a BoundQuery. - * @memberof query - * @interface IBoundQuery - * @property {string|null} [sql] BoundQuery sql - * @property {Object.|null} [bind_variables] BoundQuery bind_variables + * Properties of a RPCError. + * @memberof vtrpc + * @interface IRPCError + * @property {string|null} [message] RPCError message + * @property {vtrpc.Code|null} [code] RPCError code */ /** - * Constructs a new BoundQuery. - * @memberof query - * @classdesc Represents a BoundQuery. - * @implements IBoundQuery + * Constructs a new RPCError. + * @memberof vtrpc + * @classdesc Represents a RPCError. + * @implements IRPCError * @constructor - * @param {query.IBoundQuery=} [properties] Properties to set + * @param {vtrpc.IRPCError=} [properties] Properties to set */ - function BoundQuery(properties) { - this.bind_variables = {}; + function RPCError(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -74949,111 +75608,89 @@ export const query = $root.query = (() => { } /** - * BoundQuery sql. - * @member {string} sql - * @memberof query.BoundQuery + * RPCError message. + * @member {string} message + * @memberof vtrpc.RPCError * @instance */ - BoundQuery.prototype.sql = ""; + RPCError.prototype.message = ""; /** - * BoundQuery bind_variables. - * @member {Object.} bind_variables - * @memberof query.BoundQuery + * RPCError code. + * @member {vtrpc.Code} code + * @memberof vtrpc.RPCError * @instance */ - BoundQuery.prototype.bind_variables = $util.emptyObject; + RPCError.prototype.code = 0; /** - * Creates a new BoundQuery instance using the specified properties. + * Creates a new RPCError instance using the specified properties. * @function create - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static - * @param {query.IBoundQuery=} [properties] Properties to set - * @returns {query.BoundQuery} BoundQuery instance + * @param {vtrpc.IRPCError=} [properties] Properties to set + * @returns {vtrpc.RPCError} RPCError instance */ - BoundQuery.create = function create(properties) { - return new BoundQuery(properties); + RPCError.create = function create(properties) { + return new RPCError(properties); }; /** - * Encodes the specified BoundQuery message. Does not implicitly {@link query.BoundQuery.verify|verify} messages. + * Encodes the specified RPCError message. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. * @function encode - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static - * @param {query.IBoundQuery} message BoundQuery message or plain object to encode + * @param {vtrpc.IRPCError} message RPCError message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BoundQuery.encode = function encode(message, writer) { + RPCError.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.sql); - if (message.bind_variables != null && Object.hasOwnProperty.call(message, "bind_variables")) - for (let keys = Object.keys(message.bind_variables), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.query.BindVariable.encode(message.bind_variables[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.message); + if (message.code != null && Object.hasOwnProperty.call(message, "code")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.code); return writer; }; /** - * Encodes the specified BoundQuery message, length delimited. Does not implicitly {@link query.BoundQuery.verify|verify} messages. + * Encodes the specified RPCError message, length delimited. Does not implicitly {@link vtrpc.RPCError.verify|verify} messages. * @function encodeDelimited - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static - * @param {query.IBoundQuery} message BoundQuery message or plain object to encode + * @param {vtrpc.IRPCError} message RPCError message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BoundQuery.encodeDelimited = function encodeDelimited(message, writer) { + RPCError.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BoundQuery message from the specified reader or buffer. + * Decodes a RPCError message from the specified reader or buffer. * @function decode - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BoundQuery} BoundQuery + * @returns {vtrpc.RPCError} RPCError * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BoundQuery.decode = function decode(reader, length) { + RPCError.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.query.BoundQuery(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtrpc.RPCError(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.sql = reader.string(); + case 2: { + message.message = reader.string(); break; } - case 2: { - if (message.bind_variables === $util.emptyObject) - message.bind_variables = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.query.BindVariable.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.bind_variables[key] = value; + case 3: { + message.code = reader.int32(); break; } default: @@ -75065,162 +75702,250 @@ export const query = $root.query = (() => { }; /** - * Decodes a BoundQuery message from the specified reader or buffer, length delimited. + * Decodes a RPCError message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BoundQuery} BoundQuery + * @returns {vtrpc.RPCError} RPCError * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BoundQuery.decodeDelimited = function decodeDelimited(reader) { + RPCError.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BoundQuery message. + * Verifies a RPCError message. * @function verify - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BoundQuery.verify = function verify(message) { + RPCError.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.sql != null && message.hasOwnProperty("sql")) - if (!$util.isString(message.sql)) - return "sql: string expected"; - if (message.bind_variables != null && message.hasOwnProperty("bind_variables")) { - if (!$util.isObject(message.bind_variables)) - return "bind_variables: object expected"; - let key = Object.keys(message.bind_variables); - for (let i = 0; i < key.length; ++i) { - let error = $root.query.BindVariable.verify(message.bind_variables[key[i]]); - if (error) - return "bind_variables." + error; + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.code != null && message.hasOwnProperty("code")) + switch (message.code) { + default: + return "code: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + case 9: + case 10: + case 11: + case 12: + case 13: + case 14: + case 15: + case 16: + case 17: + case 18: + break; } - } return null; }; /** - * Creates a BoundQuery message from a plain object. Also converts values to their respective internal types. + * Creates a RPCError message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static * @param {Object.} object Plain object - * @returns {query.BoundQuery} BoundQuery + * @returns {vtrpc.RPCError} RPCError */ - BoundQuery.fromObject = function fromObject(object) { - if (object instanceof $root.query.BoundQuery) + RPCError.fromObject = function fromObject(object) { + if (object instanceof $root.vtrpc.RPCError) return object; - let message = new $root.query.BoundQuery(); - if (object.sql != null) - message.sql = String(object.sql); - if (object.bind_variables) { - if (typeof object.bind_variables !== "object") - throw TypeError(".query.BoundQuery.bind_variables: object expected"); - message.bind_variables = {}; - for (let keys = Object.keys(object.bind_variables), i = 0; i < keys.length; ++i) { - if (typeof object.bind_variables[keys[i]] !== "object") - throw TypeError(".query.BoundQuery.bind_variables: object expected"); - message.bind_variables[keys[i]] = $root.query.BindVariable.fromObject(object.bind_variables[keys[i]]); + let message = new $root.vtrpc.RPCError(); + if (object.message != null) + message.message = String(object.message); + switch (object.code) { + default: + if (typeof object.code === "number") { + message.code = object.code; + break; } + break; + case "OK": + case 0: + message.code = 0; + break; + case "CANCELED": + case 1: + message.code = 1; + break; + case "UNKNOWN": + case 2: + message.code = 2; + break; + case "INVALID_ARGUMENT": + case 3: + message.code = 3; + break; + case "DEADLINE_EXCEEDED": + case 4: + message.code = 4; + break; + case "NOT_FOUND": + case 5: + message.code = 5; + break; + case "ALREADY_EXISTS": + case 6: + message.code = 6; + break; + case "PERMISSION_DENIED": + case 7: + message.code = 7; + break; + case "RESOURCE_EXHAUSTED": + case 8: + message.code = 8; + break; + case "FAILED_PRECONDITION": + case 9: + message.code = 9; + break; + case "ABORTED": + case 10: + message.code = 10; + break; + case "OUT_OF_RANGE": + case 11: + message.code = 11; + break; + case "UNIMPLEMENTED": + case 12: + message.code = 12; + break; + case "INTERNAL": + case 13: + message.code = 13; + break; + case "UNAVAILABLE": + case 14: + message.code = 14; + break; + case "DATA_LOSS": + case 15: + message.code = 15; + break; + case "UNAUTHENTICATED": + case 16: + message.code = 16; + break; + case "CLUSTER_EVENT": + case 17: + message.code = 17; + break; + case "READ_ONLY": + case 18: + message.code = 18; + break; } return message; }; /** - * Creates a plain object from a BoundQuery message. Also converts values to other types if specified. + * Creates a plain object from a RPCError message. Also converts values to other types if specified. * @function toObject - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static - * @param {query.BoundQuery} message BoundQuery + * @param {vtrpc.RPCError} message RPCError * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BoundQuery.toObject = function toObject(message, options) { + RPCError.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.bind_variables = {}; - if (options.defaults) - object.sql = ""; - if (message.sql != null && message.hasOwnProperty("sql")) - object.sql = message.sql; - let keys2; - if (message.bind_variables && (keys2 = Object.keys(message.bind_variables)).length) { - object.bind_variables = {}; - for (let j = 0; j < keys2.length; ++j) - object.bind_variables[keys2[j]] = $root.query.BindVariable.toObject(message.bind_variables[keys2[j]], options); + if (options.defaults) { + object.message = ""; + object.code = options.enums === String ? "OK" : 0; } + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; + if (message.code != null && message.hasOwnProperty("code")) + object.code = options.enums === String ? $root.vtrpc.Code[message.code] === undefined ? message.code : $root.vtrpc.Code[message.code] : message.code; return object; }; /** - * Converts this BoundQuery to JSON. + * Converts this RPCError to JSON. * @function toJSON - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @instance * @returns {Object.} JSON object */ - BoundQuery.prototype.toJSON = function toJSON() { + RPCError.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BoundQuery + * Gets the default type url for RPCError * @function getTypeUrl - * @memberof query.BoundQuery + * @memberof vtrpc.RPCError * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BoundQuery.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RPCError.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BoundQuery"; + return typeUrlPrefix + "/vtrpc.RPCError"; }; - return BoundQuery; + return RPCError; })(); - query.ExecuteOptions = (function() { + return vtrpc; +})(); + +export const query = $root.query = (() => { + + /** + * Namespace query. + * @exports query + * @namespace + */ + const query = {}; + + query.Target = (function() { /** - * Properties of an ExecuteOptions. + * Properties of a Target. * @memberof query - * @interface IExecuteOptions - * @property {query.ExecuteOptions.IncludedFields|null} [included_fields] ExecuteOptions included_fields - * @property {boolean|null} [client_found_rows] ExecuteOptions client_found_rows - * @property {query.ExecuteOptions.Workload|null} [workload] ExecuteOptions workload - * @property {number|Long|null} [sql_select_limit] ExecuteOptions sql_select_limit - * @property {query.ExecuteOptions.TransactionIsolation|null} [transaction_isolation] ExecuteOptions transaction_isolation - * @property {boolean|null} [skip_query_plan_cache] ExecuteOptions skip_query_plan_cache - * @property {query.ExecuteOptions.PlannerVersion|null} [planner_version] ExecuteOptions planner_version - * @property {boolean|null} [has_created_temp_tables] ExecuteOptions has_created_temp_tables - * @property {query.ExecuteOptions.Consolidator|null} [consolidator] ExecuteOptions consolidator - * @property {Array.|null} [transaction_access_mode] ExecuteOptions transaction_access_mode - * @property {string|null} [WorkloadName] ExecuteOptions WorkloadName - * @property {string|null} [priority] ExecuteOptions priority + * @interface ITarget + * @property {string|null} [keyspace] Target keyspace + * @property {string|null} [shard] Target shard + * @property {topodata.TabletType|null} [tablet_type] Target tablet_type + * @property {string|null} [cell] Target cell */ /** - * Constructs a new ExecuteOptions. + * Constructs a new Target. * @memberof query - * @classdesc Represents an ExecuteOptions. - * @implements IExecuteOptions + * @classdesc Represents a Target. + * @implements ITarget * @constructor - * @param {query.IExecuteOptions=} [properties] Properties to set + * @param {query.ITarget=} [properties] Properties to set */ - function ExecuteOptions(properties) { - this.transaction_access_mode = []; + function Target(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -75228,240 +75953,117 @@ export const query = $root.query = (() => { } /** - * ExecuteOptions included_fields. - * @member {query.ExecuteOptions.IncludedFields} included_fields - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.included_fields = 0; - - /** - * ExecuteOptions client_found_rows. - * @member {boolean} client_found_rows - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.client_found_rows = false; - - /** - * ExecuteOptions workload. - * @member {query.ExecuteOptions.Workload} workload - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.workload = 0; - - /** - * ExecuteOptions sql_select_limit. - * @member {number|Long} sql_select_limit - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.sql_select_limit = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * ExecuteOptions transaction_isolation. - * @member {query.ExecuteOptions.TransactionIsolation} transaction_isolation - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.transaction_isolation = 0; - - /** - * ExecuteOptions skip_query_plan_cache. - * @member {boolean} skip_query_plan_cache - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.skip_query_plan_cache = false; - - /** - * ExecuteOptions planner_version. - * @member {query.ExecuteOptions.PlannerVersion} planner_version - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.planner_version = 0; - - /** - * ExecuteOptions has_created_temp_tables. - * @member {boolean} has_created_temp_tables - * @memberof query.ExecuteOptions - * @instance - */ - ExecuteOptions.prototype.has_created_temp_tables = false; - - /** - * ExecuteOptions consolidator. - * @member {query.ExecuteOptions.Consolidator} consolidator - * @memberof query.ExecuteOptions + * Target keyspace. + * @member {string} keyspace + * @memberof query.Target * @instance */ - ExecuteOptions.prototype.consolidator = 0; + Target.prototype.keyspace = ""; /** - * ExecuteOptions transaction_access_mode. - * @member {Array.} transaction_access_mode - * @memberof query.ExecuteOptions + * Target shard. + * @member {string} shard + * @memberof query.Target * @instance */ - ExecuteOptions.prototype.transaction_access_mode = $util.emptyArray; + Target.prototype.shard = ""; /** - * ExecuteOptions WorkloadName. - * @member {string} WorkloadName - * @memberof query.ExecuteOptions + * Target tablet_type. + * @member {topodata.TabletType} tablet_type + * @memberof query.Target * @instance */ - ExecuteOptions.prototype.WorkloadName = ""; + Target.prototype.tablet_type = 0; /** - * ExecuteOptions priority. - * @member {string} priority - * @memberof query.ExecuteOptions + * Target cell. + * @member {string} cell + * @memberof query.Target * @instance */ - ExecuteOptions.prototype.priority = ""; + Target.prototype.cell = ""; /** - * Creates a new ExecuteOptions instance using the specified properties. + * Creates a new Target instance using the specified properties. * @function create - * @memberof query.ExecuteOptions + * @memberof query.Target * @static - * @param {query.IExecuteOptions=} [properties] Properties to set - * @returns {query.ExecuteOptions} ExecuteOptions instance + * @param {query.ITarget=} [properties] Properties to set + * @returns {query.Target} Target instance */ - ExecuteOptions.create = function create(properties) { - return new ExecuteOptions(properties); + Target.create = function create(properties) { + return new Target(properties); }; /** - * Encodes the specified ExecuteOptions message. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. + * Encodes the specified Target message. Does not implicitly {@link query.Target.verify|verify} messages. * @function encode - * @memberof query.ExecuteOptions + * @memberof query.Target * @static - * @param {query.IExecuteOptions} message ExecuteOptions message or plain object to encode + * @param {query.ITarget} message Target message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteOptions.encode = function encode(message, writer) { + Target.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.included_fields != null && Object.hasOwnProperty.call(message, "included_fields")) - writer.uint32(/* id 4, wireType 0 =*/32).int32(message.included_fields); - if (message.client_found_rows != null && Object.hasOwnProperty.call(message, "client_found_rows")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.client_found_rows); - if (message.workload != null && Object.hasOwnProperty.call(message, "workload")) - writer.uint32(/* id 6, wireType 0 =*/48).int32(message.workload); - if (message.sql_select_limit != null && Object.hasOwnProperty.call(message, "sql_select_limit")) - writer.uint32(/* id 8, wireType 0 =*/64).int64(message.sql_select_limit); - if (message.transaction_isolation != null && Object.hasOwnProperty.call(message, "transaction_isolation")) - writer.uint32(/* id 9, wireType 0 =*/72).int32(message.transaction_isolation); - if (message.skip_query_plan_cache != null && Object.hasOwnProperty.call(message, "skip_query_plan_cache")) - writer.uint32(/* id 10, wireType 0 =*/80).bool(message.skip_query_plan_cache); - if (message.planner_version != null && Object.hasOwnProperty.call(message, "planner_version")) - writer.uint32(/* id 11, wireType 0 =*/88).int32(message.planner_version); - if (message.has_created_temp_tables != null && Object.hasOwnProperty.call(message, "has_created_temp_tables")) - writer.uint32(/* id 12, wireType 0 =*/96).bool(message.has_created_temp_tables); - if (message.consolidator != null && Object.hasOwnProperty.call(message, "consolidator")) - writer.uint32(/* id 13, wireType 0 =*/104).int32(message.consolidator); - if (message.transaction_access_mode != null && message.transaction_access_mode.length) { - writer.uint32(/* id 14, wireType 2 =*/114).fork(); - for (let i = 0; i < message.transaction_access_mode.length; ++i) - writer.int32(message.transaction_access_mode[i]); - writer.ldelim(); - } - if (message.WorkloadName != null && Object.hasOwnProperty.call(message, "WorkloadName")) - writer.uint32(/* id 15, wireType 2 =*/122).string(message.WorkloadName); - if (message.priority != null && Object.hasOwnProperty.call(message, "priority")) - writer.uint32(/* id 16, wireType 2 =*/130).string(message.priority); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.cell); return writer; }; /** - * Encodes the specified ExecuteOptions message, length delimited. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. + * Encodes the specified Target message, length delimited. Does not implicitly {@link query.Target.verify|verify} messages. * @function encodeDelimited - * @memberof query.ExecuteOptions + * @memberof query.Target * @static - * @param {query.IExecuteOptions} message ExecuteOptions message or plain object to encode + * @param {query.ITarget} message Target message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteOptions.encodeDelimited = function encodeDelimited(message, writer) { + Target.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteOptions message from the specified reader or buffer. + * Decodes a Target message from the specified reader or buffer. * @function decode - * @memberof query.ExecuteOptions + * @memberof query.Target * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ExecuteOptions} ExecuteOptions + * @returns {query.Target} Target * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteOptions.decode = function decode(reader, length) { + Target.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.query.ExecuteOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.Target(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 4: { - message.included_fields = reader.int32(); - break; - } - case 5: { - message.client_found_rows = reader.bool(); - break; - } - case 6: { - message.workload = reader.int32(); - break; - } - case 8: { - message.sql_select_limit = reader.int64(); - break; - } - case 9: { - message.transaction_isolation = reader.int32(); - break; - } - case 10: { - message.skip_query_plan_cache = reader.bool(); - break; - } - case 11: { - message.planner_version = reader.int32(); - break; - } - case 12: { - message.has_created_temp_tables = reader.bool(); - break; - } - case 13: { - message.consolidator = reader.int32(); + case 1: { + message.keyspace = reader.string(); break; } - case 14: { - if (!(message.transaction_access_mode && message.transaction_access_mode.length)) - message.transaction_access_mode = []; - if ((tag & 7) === 2) { - let end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) - message.transaction_access_mode.push(reader.int32()); - } else - message.transaction_access_mode.push(reader.int32()); + case 2: { + message.shard = reader.string(); break; } - case 15: { - message.WorkloadName = reader.string(); + case 3: { + message.tablet_type = reader.int32(); break; } - case 16: { - message.priority = reader.string(); + case 4: { + message.cell = reader.string(); break; } default: @@ -75473,565 +76075,213 @@ export const query = $root.query = (() => { }; /** - * Decodes an ExecuteOptions message from the specified reader or buffer, length delimited. + * Decodes a Target message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ExecuteOptions + * @memberof query.Target * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ExecuteOptions} ExecuteOptions + * @returns {query.Target} Target * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteOptions.decodeDelimited = function decodeDelimited(reader) { + Target.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteOptions message. + * Verifies a Target message. * @function verify - * @memberof query.ExecuteOptions + * @memberof query.Target * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteOptions.verify = function verify(message) { + Target.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.included_fields != null && message.hasOwnProperty("included_fields")) - switch (message.included_fields) { - default: - return "included_fields: enum value expected"; - case 0: - case 1: - case 2: - break; - } - if (message.client_found_rows != null && message.hasOwnProperty("client_found_rows")) - if (typeof message.client_found_rows !== "boolean") - return "client_found_rows: boolean expected"; - if (message.workload != null && message.hasOwnProperty("workload")) - switch (message.workload) { + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + switch (message.tablet_type) { default: - return "workload: enum value expected"; + return "tablet_type: enum value expected"; case 0: case 1: - case 2: - case 3: - break; - } - if (message.sql_select_limit != null && message.hasOwnProperty("sql_select_limit")) - if (!$util.isInteger(message.sql_select_limit) && !(message.sql_select_limit && $util.isInteger(message.sql_select_limit.low) && $util.isInteger(message.sql_select_limit.high))) - return "sql_select_limit: integer|Long expected"; - if (message.transaction_isolation != null && message.hasOwnProperty("transaction_isolation")) - switch (message.transaction_isolation) { - default: - return "transaction_isolation: enum value expected"; - case 0: case 1: case 2: case 3: - case 4: - case 5: - case 6: - break; - } - if (message.skip_query_plan_cache != null && message.hasOwnProperty("skip_query_plan_cache")) - if (typeof message.skip_query_plan_cache !== "boolean") - return "skip_query_plan_cache: boolean expected"; - if (message.planner_version != null && message.hasOwnProperty("planner_version")) - switch (message.planner_version) { - default: - return "planner_version: enum value expected"; - case 0: - case 1: - case 2: case 3: case 4: case 5: case 6: case 7: + case 8: break; } - if (message.has_created_temp_tables != null && message.hasOwnProperty("has_created_temp_tables")) - if (typeof message.has_created_temp_tables !== "boolean") - return "has_created_temp_tables: boolean expected"; - if (message.consolidator != null && message.hasOwnProperty("consolidator")) - switch (message.consolidator) { - default: - return "consolidator: enum value expected"; - case 0: - case 1: - case 2: - case 3: - break; - } - if (message.transaction_access_mode != null && message.hasOwnProperty("transaction_access_mode")) { - if (!Array.isArray(message.transaction_access_mode)) - return "transaction_access_mode: array expected"; - for (let i = 0; i < message.transaction_access_mode.length; ++i) - switch (message.transaction_access_mode[i]) { - default: - return "transaction_access_mode: enum value[] expected"; - case 0: - case 1: - case 2: - break; - } - } - if (message.WorkloadName != null && message.hasOwnProperty("WorkloadName")) - if (!$util.isString(message.WorkloadName)) - return "WorkloadName: string expected"; - if (message.priority != null && message.hasOwnProperty("priority")) - if (!$util.isString(message.priority)) - return "priority: string expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; return null; }; /** - * Creates an ExecuteOptions message from a plain object. Also converts values to their respective internal types. + * Creates a Target message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ExecuteOptions + * @memberof query.Target * @static * @param {Object.} object Plain object - * @returns {query.ExecuteOptions} ExecuteOptions + * @returns {query.Target} Target */ - ExecuteOptions.fromObject = function fromObject(object) { - if (object instanceof $root.query.ExecuteOptions) + Target.fromObject = function fromObject(object) { + if (object instanceof $root.query.Target) return object; - let message = new $root.query.ExecuteOptions(); - switch (object.included_fields) { + let message = new $root.query.Target(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + switch (object.tablet_type) { default: - if (typeof object.included_fields === "number") { - message.included_fields = object.included_fields; + if (typeof object.tablet_type === "number") { + message.tablet_type = object.tablet_type; break; } break; - case "TYPE_AND_NAME": + case "UNKNOWN": case 0: - message.included_fields = 0; + message.tablet_type = 0; break; - case "TYPE_ONLY": + case "PRIMARY": case 1: - message.included_fields = 1; - break; - case "ALL": - case 2: - message.included_fields = 2; - break; - } - if (object.client_found_rows != null) - message.client_found_rows = Boolean(object.client_found_rows); - switch (object.workload) { - default: - if (typeof object.workload === "number") { - message.workload = object.workload; - break; - } - break; - case "UNSPECIFIED": - case 0: - message.workload = 0; + message.tablet_type = 1; break; - case "OLTP": + case "MASTER": case 1: - message.workload = 1; + message.tablet_type = 1; break; - case "OLAP": + case "REPLICA": case 2: - message.workload = 2; + message.tablet_type = 2; break; - case "DBA": + case "RDONLY": case 3: - message.workload = 3; - break; - } - if (object.sql_select_limit != null) - if ($util.Long) - (message.sql_select_limit = $util.Long.fromValue(object.sql_select_limit)).unsigned = false; - else if (typeof object.sql_select_limit === "string") - message.sql_select_limit = parseInt(object.sql_select_limit, 10); - else if (typeof object.sql_select_limit === "number") - message.sql_select_limit = object.sql_select_limit; - else if (typeof object.sql_select_limit === "object") - message.sql_select_limit = new $util.LongBits(object.sql_select_limit.low >>> 0, object.sql_select_limit.high >>> 0).toNumber(); - switch (object.transaction_isolation) { - default: - if (typeof object.transaction_isolation === "number") { - message.transaction_isolation = object.transaction_isolation; - break; - } - break; - case "DEFAULT": - case 0: - message.transaction_isolation = 0; - break; - case "REPEATABLE_READ": - case 1: - message.transaction_isolation = 1; - break; - case "READ_COMMITTED": - case 2: - message.transaction_isolation = 2; + message.tablet_type = 3; break; - case "READ_UNCOMMITTED": + case "BATCH": case 3: - message.transaction_isolation = 3; + message.tablet_type = 3; break; - case "SERIALIZABLE": + case "SPARE": case 4: - message.transaction_isolation = 4; + message.tablet_type = 4; break; - case "CONSISTENT_SNAPSHOT_READ_ONLY": + case "EXPERIMENTAL": case 5: - message.transaction_isolation = 5; + message.tablet_type = 5; break; - case "AUTOCOMMIT": + case "BACKUP": case 6: - message.transaction_isolation = 6; + message.tablet_type = 6; + break; + case "RESTORE": + case 7: + message.tablet_type = 7; + break; + case "DRAINED": + case 8: + message.tablet_type = 8; break; } - if (object.skip_query_plan_cache != null) - message.skip_query_plan_cache = Boolean(object.skip_query_plan_cache); - switch (object.planner_version) { - default: - if (typeof object.planner_version === "number") { - message.planner_version = object.planner_version; - break; - } - break; - case "DEFAULT_PLANNER": - case 0: - message.planner_version = 0; - break; - case "V3": - case 1: - message.planner_version = 1; - break; - case "Gen4": - case 2: - message.planner_version = 2; - break; - case "Gen4Greedy": - case 3: - message.planner_version = 3; - break; - case "Gen4Left2Right": - case 4: - message.planner_version = 4; - break; - case "Gen4WithFallback": - case 5: - message.planner_version = 5; - break; - case "Gen4CompareV3": - case 6: - message.planner_version = 6; - break; - case "V3Insert": - case 7: - message.planner_version = 7; - break; - } - if (object.has_created_temp_tables != null) - message.has_created_temp_tables = Boolean(object.has_created_temp_tables); - switch (object.consolidator) { - default: - if (typeof object.consolidator === "number") { - message.consolidator = object.consolidator; - break; - } - break; - case "CONSOLIDATOR_UNSPECIFIED": - case 0: - message.consolidator = 0; - break; - case "CONSOLIDATOR_DISABLED": - case 1: - message.consolidator = 1; - break; - case "CONSOLIDATOR_ENABLED": - case 2: - message.consolidator = 2; - break; - case "CONSOLIDATOR_ENABLED_REPLICAS": - case 3: - message.consolidator = 3; - break; - } - if (object.transaction_access_mode) { - if (!Array.isArray(object.transaction_access_mode)) - throw TypeError(".query.ExecuteOptions.transaction_access_mode: array expected"); - message.transaction_access_mode = []; - for (let i = 0; i < object.transaction_access_mode.length; ++i) - switch (object.transaction_access_mode[i]) { - default: - if (typeof object.transaction_access_mode[i] === "number") { - message.transaction_access_mode[i] = object.transaction_access_mode[i]; - break; - } - case "CONSISTENT_SNAPSHOT": - case 0: - message.transaction_access_mode[i] = 0; - break; - case "READ_WRITE": - case 1: - message.transaction_access_mode[i] = 1; - break; - case "READ_ONLY": - case 2: - message.transaction_access_mode[i] = 2; - break; - } - } - if (object.WorkloadName != null) - message.WorkloadName = String(object.WorkloadName); - if (object.priority != null) - message.priority = String(object.priority); + if (object.cell != null) + message.cell = String(object.cell); return message; }; /** - * Creates a plain object from an ExecuteOptions message. Also converts values to other types if specified. + * Creates a plain object from a Target message. Also converts values to other types if specified. * @function toObject - * @memberof query.ExecuteOptions + * @memberof query.Target * @static - * @param {query.ExecuteOptions} message ExecuteOptions + * @param {query.Target} message Target * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteOptions.toObject = function toObject(message, options) { + Target.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.transaction_access_mode = []; if (options.defaults) { - object.included_fields = options.enums === String ? "TYPE_AND_NAME" : 0; - object.client_found_rows = false; - object.workload = options.enums === String ? "UNSPECIFIED" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.sql_select_limit = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.sql_select_limit = options.longs === String ? "0" : 0; - object.transaction_isolation = options.enums === String ? "DEFAULT" : 0; - object.skip_query_plan_cache = false; - object.planner_version = options.enums === String ? "DEFAULT_PLANNER" : 0; - object.has_created_temp_tables = false; - object.consolidator = options.enums === String ? "CONSOLIDATOR_UNSPECIFIED" : 0; - object.WorkloadName = ""; - object.priority = ""; - } - if (message.included_fields != null && message.hasOwnProperty("included_fields")) - object.included_fields = options.enums === String ? $root.query.ExecuteOptions.IncludedFields[message.included_fields] === undefined ? message.included_fields : $root.query.ExecuteOptions.IncludedFields[message.included_fields] : message.included_fields; - if (message.client_found_rows != null && message.hasOwnProperty("client_found_rows")) - object.client_found_rows = message.client_found_rows; - if (message.workload != null && message.hasOwnProperty("workload")) - object.workload = options.enums === String ? $root.query.ExecuteOptions.Workload[message.workload] === undefined ? message.workload : $root.query.ExecuteOptions.Workload[message.workload] : message.workload; - if (message.sql_select_limit != null && message.hasOwnProperty("sql_select_limit")) - if (typeof message.sql_select_limit === "number") - object.sql_select_limit = options.longs === String ? String(message.sql_select_limit) : message.sql_select_limit; - else - object.sql_select_limit = options.longs === String ? $util.Long.prototype.toString.call(message.sql_select_limit) : options.longs === Number ? new $util.LongBits(message.sql_select_limit.low >>> 0, message.sql_select_limit.high >>> 0).toNumber() : message.sql_select_limit; - if (message.transaction_isolation != null && message.hasOwnProperty("transaction_isolation")) - object.transaction_isolation = options.enums === String ? $root.query.ExecuteOptions.TransactionIsolation[message.transaction_isolation] === undefined ? message.transaction_isolation : $root.query.ExecuteOptions.TransactionIsolation[message.transaction_isolation] : message.transaction_isolation; - if (message.skip_query_plan_cache != null && message.hasOwnProperty("skip_query_plan_cache")) - object.skip_query_plan_cache = message.skip_query_plan_cache; - if (message.planner_version != null && message.hasOwnProperty("planner_version")) - object.planner_version = options.enums === String ? $root.query.ExecuteOptions.PlannerVersion[message.planner_version] === undefined ? message.planner_version : $root.query.ExecuteOptions.PlannerVersion[message.planner_version] : message.planner_version; - if (message.has_created_temp_tables != null && message.hasOwnProperty("has_created_temp_tables")) - object.has_created_temp_tables = message.has_created_temp_tables; - if (message.consolidator != null && message.hasOwnProperty("consolidator")) - object.consolidator = options.enums === String ? $root.query.ExecuteOptions.Consolidator[message.consolidator] === undefined ? message.consolidator : $root.query.ExecuteOptions.Consolidator[message.consolidator] : message.consolidator; - if (message.transaction_access_mode && message.transaction_access_mode.length) { - object.transaction_access_mode = []; - for (let j = 0; j < message.transaction_access_mode.length; ++j) - object.transaction_access_mode[j] = options.enums === String ? $root.query.ExecuteOptions.TransactionAccessMode[message.transaction_access_mode[j]] === undefined ? message.transaction_access_mode[j] : $root.query.ExecuteOptions.TransactionAccessMode[message.transaction_access_mode[j]] : message.transaction_access_mode[j]; + object.keyspace = ""; + object.shard = ""; + object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + object.cell = ""; } - if (message.WorkloadName != null && message.hasOwnProperty("WorkloadName")) - object.WorkloadName = message.WorkloadName; - if (message.priority != null && message.hasOwnProperty("priority")) - object.priority = message.priority; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; return object; }; /** - * Converts this ExecuteOptions to JSON. + * Converts this Target to JSON. * @function toJSON - * @memberof query.ExecuteOptions + * @memberof query.Target * @instance * @returns {Object.} JSON object */ - ExecuteOptions.prototype.toJSON = function toJSON() { + Target.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteOptions + * Gets the default type url for Target * @function getTypeUrl - * @memberof query.ExecuteOptions + * @memberof query.Target * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Target.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ExecuteOptions"; + return typeUrlPrefix + "/query.Target"; }; - /** - * IncludedFields enum. - * @name query.ExecuteOptions.IncludedFields - * @enum {number} - * @property {number} TYPE_AND_NAME=0 TYPE_AND_NAME value - * @property {number} TYPE_ONLY=1 TYPE_ONLY value - * @property {number} ALL=2 ALL value - */ - ExecuteOptions.IncludedFields = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "TYPE_AND_NAME"] = 0; - values[valuesById[1] = "TYPE_ONLY"] = 1; - values[valuesById[2] = "ALL"] = 2; - return values; - })(); - - /** - * Workload enum. - * @name query.ExecuteOptions.Workload - * @enum {number} - * @property {number} UNSPECIFIED=0 UNSPECIFIED value - * @property {number} OLTP=1 OLTP value - * @property {number} OLAP=2 OLAP value - * @property {number} DBA=3 DBA value - */ - ExecuteOptions.Workload = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "UNSPECIFIED"] = 0; - values[valuesById[1] = "OLTP"] = 1; - values[valuesById[2] = "OLAP"] = 2; - values[valuesById[3] = "DBA"] = 3; - return values; - })(); - - /** - * TransactionIsolation enum. - * @name query.ExecuteOptions.TransactionIsolation - * @enum {number} - * @property {number} DEFAULT=0 DEFAULT value - * @property {number} REPEATABLE_READ=1 REPEATABLE_READ value - * @property {number} READ_COMMITTED=2 READ_COMMITTED value - * @property {number} READ_UNCOMMITTED=3 READ_UNCOMMITTED value - * @property {number} SERIALIZABLE=4 SERIALIZABLE value - * @property {number} CONSISTENT_SNAPSHOT_READ_ONLY=5 CONSISTENT_SNAPSHOT_READ_ONLY value - * @property {number} AUTOCOMMIT=6 AUTOCOMMIT value - */ - ExecuteOptions.TransactionIsolation = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "DEFAULT"] = 0; - values[valuesById[1] = "REPEATABLE_READ"] = 1; - values[valuesById[2] = "READ_COMMITTED"] = 2; - values[valuesById[3] = "READ_UNCOMMITTED"] = 3; - values[valuesById[4] = "SERIALIZABLE"] = 4; - values[valuesById[5] = "CONSISTENT_SNAPSHOT_READ_ONLY"] = 5; - values[valuesById[6] = "AUTOCOMMIT"] = 6; - return values; - })(); - - /** - * PlannerVersion enum. - * @name query.ExecuteOptions.PlannerVersion - * @enum {number} - * @property {number} DEFAULT_PLANNER=0 DEFAULT_PLANNER value - * @property {number} V3=1 V3 value - * @property {number} Gen4=2 Gen4 value - * @property {number} Gen4Greedy=3 Gen4Greedy value - * @property {number} Gen4Left2Right=4 Gen4Left2Right value - * @property {number} Gen4WithFallback=5 Gen4WithFallback value - * @property {number} Gen4CompareV3=6 Gen4CompareV3 value - * @property {number} V3Insert=7 V3Insert value - */ - ExecuteOptions.PlannerVersion = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "DEFAULT_PLANNER"] = 0; - values[valuesById[1] = "V3"] = 1; - values[valuesById[2] = "Gen4"] = 2; - values[valuesById[3] = "Gen4Greedy"] = 3; - values[valuesById[4] = "Gen4Left2Right"] = 4; - values[valuesById[5] = "Gen4WithFallback"] = 5; - values[valuesById[6] = "Gen4CompareV3"] = 6; - values[valuesById[7] = "V3Insert"] = 7; - return values; - })(); - - /** - * Consolidator enum. - * @name query.ExecuteOptions.Consolidator - * @enum {number} - * @property {number} CONSOLIDATOR_UNSPECIFIED=0 CONSOLIDATOR_UNSPECIFIED value - * @property {number} CONSOLIDATOR_DISABLED=1 CONSOLIDATOR_DISABLED value - * @property {number} CONSOLIDATOR_ENABLED=2 CONSOLIDATOR_ENABLED value - * @property {number} CONSOLIDATOR_ENABLED_REPLICAS=3 CONSOLIDATOR_ENABLED_REPLICAS value - */ - ExecuteOptions.Consolidator = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "CONSOLIDATOR_UNSPECIFIED"] = 0; - values[valuesById[1] = "CONSOLIDATOR_DISABLED"] = 1; - values[valuesById[2] = "CONSOLIDATOR_ENABLED"] = 2; - values[valuesById[3] = "CONSOLIDATOR_ENABLED_REPLICAS"] = 3; - return values; - })(); - - /** - * TransactionAccessMode enum. - * @name query.ExecuteOptions.TransactionAccessMode - * @enum {number} - * @property {number} CONSISTENT_SNAPSHOT=0 CONSISTENT_SNAPSHOT value - * @property {number} READ_WRITE=1 READ_WRITE value - * @property {number} READ_ONLY=2 READ_ONLY value - */ - ExecuteOptions.TransactionAccessMode = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "CONSISTENT_SNAPSHOT"] = 0; - values[valuesById[1] = "READ_WRITE"] = 1; - values[valuesById[2] = "READ_ONLY"] = 2; - return values; - })(); - - return ExecuteOptions; + return Target; })(); - query.Field = (function() { + query.VTGateCallerID = (function() { /** - * Properties of a Field. + * Properties of a VTGateCallerID. * @memberof query - * @interface IField - * @property {string|null} [name] Field name - * @property {query.Type|null} [type] Field type - * @property {string|null} [table] Field table - * @property {string|null} [org_table] Field org_table - * @property {string|null} [database] Field database - * @property {string|null} [org_name] Field org_name - * @property {number|null} [column_length] Field column_length - * @property {number|null} [charset] Field charset - * @property {number|null} [decimals] Field decimals - * @property {number|null} [flags] Field flags - * @property {string|null} [column_type] Field column_type + * @interface IVTGateCallerID + * @property {string|null} [username] VTGateCallerID username + * @property {Array.|null} [groups] VTGateCallerID groups */ /** - * Constructs a new Field. + * Constructs a new VTGateCallerID. * @memberof query - * @classdesc Represents a Field. - * @implements IField + * @classdesc Represents a VTGateCallerID. + * @implements IVTGateCallerID * @constructor - * @param {query.IField=} [properties] Properties to set + * @param {query.IVTGateCallerID=} [properties] Properties to set */ - function Field(properties) { + function VTGateCallerID(properties) { + this.groups = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -76039,215 +76289,92 @@ export const query = $root.query = (() => { } /** - * Field name. - * @member {string} name - * @memberof query.Field - * @instance - */ - Field.prototype.name = ""; - - /** - * Field type. - * @member {query.Type} type - * @memberof query.Field - * @instance - */ - Field.prototype.type = 0; - - /** - * Field table. - * @member {string} table - * @memberof query.Field - * @instance - */ - Field.prototype.table = ""; - - /** - * Field org_table. - * @member {string} org_table - * @memberof query.Field - * @instance - */ - Field.prototype.org_table = ""; - - /** - * Field database. - * @member {string} database - * @memberof query.Field - * @instance - */ - Field.prototype.database = ""; - - /** - * Field org_name. - * @member {string} org_name - * @memberof query.Field - * @instance - */ - Field.prototype.org_name = ""; - - /** - * Field column_length. - * @member {number} column_length - * @memberof query.Field - * @instance - */ - Field.prototype.column_length = 0; - - /** - * Field charset. - * @member {number} charset - * @memberof query.Field - * @instance - */ - Field.prototype.charset = 0; - - /** - * Field decimals. - * @member {number} decimals - * @memberof query.Field - * @instance - */ - Field.prototype.decimals = 0; - - /** - * Field flags. - * @member {number} flags - * @memberof query.Field + * VTGateCallerID username. + * @member {string} username + * @memberof query.VTGateCallerID * @instance */ - Field.prototype.flags = 0; + VTGateCallerID.prototype.username = ""; /** - * Field column_type. - * @member {string} column_type - * @memberof query.Field + * VTGateCallerID groups. + * @member {Array.} groups + * @memberof query.VTGateCallerID * @instance */ - Field.prototype.column_type = ""; + VTGateCallerID.prototype.groups = $util.emptyArray; /** - * Creates a new Field instance using the specified properties. + * Creates a new VTGateCallerID instance using the specified properties. * @function create - * @memberof query.Field + * @memberof query.VTGateCallerID * @static - * @param {query.IField=} [properties] Properties to set - * @returns {query.Field} Field instance + * @param {query.IVTGateCallerID=} [properties] Properties to set + * @returns {query.VTGateCallerID} VTGateCallerID instance */ - Field.create = function create(properties) { - return new Field(properties); + VTGateCallerID.create = function create(properties) { + return new VTGateCallerID(properties); }; /** - * Encodes the specified Field message. Does not implicitly {@link query.Field.verify|verify} messages. + * Encodes the specified VTGateCallerID message. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. * @function encode - * @memberof query.Field + * @memberof query.VTGateCallerID * @static - * @param {query.IField} message Field message or plain object to encode + * @param {query.IVTGateCallerID} message VTGateCallerID message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Field.encode = function encode(message, writer) { + VTGateCallerID.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.type); - if (message.table != null && Object.hasOwnProperty.call(message, "table")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.table); - if (message.org_table != null && Object.hasOwnProperty.call(message, "org_table")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.org_table); - if (message.database != null && Object.hasOwnProperty.call(message, "database")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.database); - if (message.org_name != null && Object.hasOwnProperty.call(message, "org_name")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.org_name); - if (message.column_length != null && Object.hasOwnProperty.call(message, "column_length")) - writer.uint32(/* id 7, wireType 0 =*/56).uint32(message.column_length); - if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) - writer.uint32(/* id 8, wireType 0 =*/64).uint32(message.charset); - if (message.decimals != null && Object.hasOwnProperty.call(message, "decimals")) - writer.uint32(/* id 9, wireType 0 =*/72).uint32(message.decimals); - if (message.flags != null && Object.hasOwnProperty.call(message, "flags")) - writer.uint32(/* id 10, wireType 0 =*/80).uint32(message.flags); - if (message.column_type != null && Object.hasOwnProperty.call(message, "column_type")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.column_type); + if (message.username != null && Object.hasOwnProperty.call(message, "username")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.username); + if (message.groups != null && message.groups.length) + for (let i = 0; i < message.groups.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.groups[i]); return writer; }; /** - * Encodes the specified Field message, length delimited. Does not implicitly {@link query.Field.verify|verify} messages. + * Encodes the specified VTGateCallerID message, length delimited. Does not implicitly {@link query.VTGateCallerID.verify|verify} messages. * @function encodeDelimited - * @memberof query.Field + * @memberof query.VTGateCallerID * @static - * @param {query.IField} message Field message or plain object to encode + * @param {query.IVTGateCallerID} message VTGateCallerID message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Field.encodeDelimited = function encodeDelimited(message, writer) { + VTGateCallerID.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Field message from the specified reader or buffer. + * Decodes a VTGateCallerID message from the specified reader or buffer. * @function decode - * @memberof query.Field + * @memberof query.VTGateCallerID * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.Field} Field + * @returns {query.VTGateCallerID} VTGateCallerID * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Field.decode = function decode(reader, length) { + VTGateCallerID.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.query.Field(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.VTGateCallerID(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.username = reader.string(); break; } case 2: { - message.type = reader.int32(); - break; - } - case 3: { - message.table = reader.string(); - break; - } - case 4: { - message.org_table = reader.string(); - break; - } - case 5: { - message.database = reader.string(); - break; - } - case 6: { - message.org_name = reader.string(); - break; - } - case 7: { - message.column_length = reader.uint32(); - break; - } - case 8: { - message.charset = reader.uint32(); - break; - } - case 9: { - message.decimals = reader.uint32(); - break; - } - case 10: { - message.flags = reader.uint32(); - break; - } - case 11: { - message.column_type = reader.string(); + if (!(message.groups && message.groups.length)) + message.groups = []; + message.groups.push(reader.string()); break; } default: @@ -76259,389 +76386,145 @@ export const query = $root.query = (() => { }; /** - * Decodes a Field message from the specified reader or buffer, length delimited. + * Decodes a VTGateCallerID message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.Field + * @memberof query.VTGateCallerID * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.Field} Field + * @returns {query.VTGateCallerID} VTGateCallerID * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Field.decodeDelimited = function decodeDelimited(reader) { + VTGateCallerID.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Field message. + * Verifies a VTGateCallerID message. * @function verify - * @memberof query.Field + * @memberof query.VTGateCallerID * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Field.verify = function verify(message) { + VTGateCallerID.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "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; - } - if (message.table != null && message.hasOwnProperty("table")) - if (!$util.isString(message.table)) - return "table: string expected"; - if (message.org_table != null && message.hasOwnProperty("org_table")) - if (!$util.isString(message.org_table)) - return "org_table: string expected"; - if (message.database != null && message.hasOwnProperty("database")) - if (!$util.isString(message.database)) - return "database: string expected"; - if (message.org_name != null && message.hasOwnProperty("org_name")) - if (!$util.isString(message.org_name)) - return "org_name: string expected"; - if (message.column_length != null && message.hasOwnProperty("column_length")) - if (!$util.isInteger(message.column_length)) - return "column_length: integer expected"; - if (message.charset != null && message.hasOwnProperty("charset")) - if (!$util.isInteger(message.charset)) - return "charset: integer expected"; - if (message.decimals != null && message.hasOwnProperty("decimals")) - if (!$util.isInteger(message.decimals)) - return "decimals: integer expected"; - if (message.flags != null && message.hasOwnProperty("flags")) - if (!$util.isInteger(message.flags)) - return "flags: integer expected"; - if (message.column_type != null && message.hasOwnProperty("column_type")) - if (!$util.isString(message.column_type)) - return "column_type: string expected"; + if (message.username != null && message.hasOwnProperty("username")) + if (!$util.isString(message.username)) + return "username: string expected"; + if (message.groups != null && message.hasOwnProperty("groups")) { + if (!Array.isArray(message.groups)) + return "groups: array expected"; + for (let i = 0; i < message.groups.length; ++i) + if (!$util.isString(message.groups[i])) + return "groups: string[] expected"; + } return null; }; /** - * Creates a Field message from a plain object. Also converts values to their respective internal types. + * Creates a VTGateCallerID message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.Field + * @memberof query.VTGateCallerID * @static * @param {Object.} object Plain object - * @returns {query.Field} Field + * @returns {query.VTGateCallerID} VTGateCallerID */ - Field.fromObject = function fromObject(object) { - if (object instanceof $root.query.Field) + VTGateCallerID.fromObject = function fromObject(object) { + if (object instanceof $root.query.VTGateCallerID) return object; - let message = new $root.query.Field(); - if (object.name != null) - message.name = String(object.name); - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; - } - break; - case "NULL_TYPE": - case 0: - message.type = 0; - break; - case "INT8": - case 257: - message.type = 257; - break; - case "UINT8": - case 770: - message.type = 770; - break; - case "INT16": - case 259: - message.type = 259; - break; - case "UINT16": - case 772: - message.type = 772; - break; - case "INT24": - case 261: - message.type = 261; - break; - case "UINT24": - case 774: - message.type = 774; - break; - case "INT32": - case 263: - message.type = 263; - break; - case "UINT32": - case 776: - message.type = 776; - break; - case "INT64": - case 265: - message.type = 265; - break; - case "UINT64": - case 778: - message.type = 778; - break; - case "FLOAT32": - case 1035: - message.type = 1035; - break; - case "FLOAT64": - case 1036: - message.type = 1036; - break; - case "TIMESTAMP": - case 2061: - message.type = 2061; - break; - case "DATE": - case 2062: - message.type = 2062; - break; - case "TIME": - case 2063: - message.type = 2063; - break; - case "DATETIME": - case 2064: - message.type = 2064; - break; - case "YEAR": - case 785: - message.type = 785; - break; - case "DECIMAL": - case 18: - message.type = 18; - break; - case "TEXT": - case 6163: - message.type = 6163; - break; - case "BLOB": - case 10260: - message.type = 10260; - break; - case "VARCHAR": - case 6165: - message.type = 6165; - break; - case "VARBINARY": - case 10262: - message.type = 10262; - break; - case "CHAR": - case 6167: - message.type = 6167; - break; - case "BINARY": - case 10264: - message.type = 10264; - break; - case "BIT": - case 2073: - message.type = 2073; - break; - case "ENUM": - case 2074: - message.type = 2074; - break; - case "SET": - case 2075: - message.type = 2075; - break; - case "TUPLE": - case 28: - message.type = 28; - break; - case "GEOMETRY": - case 2077: - message.type = 2077; - break; - case "JSON": - case 2078: - message.type = 2078; - break; - case "EXPRESSION": - case 31: - message.type = 31; - break; - case "HEXNUM": - case 4128: - message.type = 4128; - break; - case "HEXVAL": - case 4129: - message.type = 4129; - break; - case "BITNUM": - case 4130: - message.type = 4130; - break; + let message = new $root.query.VTGateCallerID(); + if (object.username != null) + message.username = String(object.username); + if (object.groups) { + if (!Array.isArray(object.groups)) + throw TypeError(".query.VTGateCallerID.groups: array expected"); + message.groups = []; + for (let i = 0; i < object.groups.length; ++i) + message.groups[i] = String(object.groups[i]); } - if (object.table != null) - message.table = String(object.table); - if (object.org_table != null) - message.org_table = String(object.org_table); - if (object.database != null) - message.database = String(object.database); - if (object.org_name != null) - message.org_name = String(object.org_name); - if (object.column_length != null) - message.column_length = object.column_length >>> 0; - if (object.charset != null) - message.charset = object.charset >>> 0; - if (object.decimals != null) - message.decimals = object.decimals >>> 0; - if (object.flags != null) - message.flags = object.flags >>> 0; - if (object.column_type != null) - message.column_type = String(object.column_type); return message; }; /** - * Creates a plain object from a Field message. Also converts values to other types if specified. + * Creates a plain object from a VTGateCallerID message. Also converts values to other types if specified. * @function toObject - * @memberof query.Field + * @memberof query.VTGateCallerID * @static - * @param {query.Field} message Field + * @param {query.VTGateCallerID} message VTGateCallerID * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Field.toObject = function toObject(message, options) { + VTGateCallerID.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.name = ""; - object.type = options.enums === String ? "NULL_TYPE" : 0; - object.table = ""; - object.org_table = ""; - object.database = ""; - object.org_name = ""; - object.column_length = 0; - object.charset = 0; - object.decimals = 0; - object.flags = 0; - object.column_type = ""; + if (options.arrays || options.defaults) + object.groups = []; + if (options.defaults) + object.username = ""; + if (message.username != null && message.hasOwnProperty("username")) + object.username = message.username; + if (message.groups && message.groups.length) { + object.groups = []; + for (let j = 0; j < message.groups.length; ++j) + object.groups[j] = message.groups[j]; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; - if (message.table != null && message.hasOwnProperty("table")) - object.table = message.table; - if (message.org_table != null && message.hasOwnProperty("org_table")) - object.org_table = message.org_table; - if (message.database != null && message.hasOwnProperty("database")) - object.database = message.database; - if (message.org_name != null && message.hasOwnProperty("org_name")) - object.org_name = message.org_name; - if (message.column_length != null && message.hasOwnProperty("column_length")) - object.column_length = message.column_length; - if (message.charset != null && message.hasOwnProperty("charset")) - object.charset = message.charset; - if (message.decimals != null && message.hasOwnProperty("decimals")) - object.decimals = message.decimals; - if (message.flags != null && message.hasOwnProperty("flags")) - object.flags = message.flags; - if (message.column_type != null && message.hasOwnProperty("column_type")) - object.column_type = message.column_type; return object; }; /** - * Converts this Field to JSON. + * Converts this VTGateCallerID to JSON. * @function toJSON - * @memberof query.Field + * @memberof query.VTGateCallerID * @instance * @returns {Object.} JSON object */ - Field.prototype.toJSON = function toJSON() { + VTGateCallerID.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Field + * Gets the default type url for VTGateCallerID * @function getTypeUrl - * @memberof query.Field + * @memberof query.VTGateCallerID * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Field.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VTGateCallerID.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.Field"; + return typeUrlPrefix + "/query.VTGateCallerID"; }; - return Field; + return VTGateCallerID; })(); - query.Row = (function() { + query.EventToken = (function() { /** - * Properties of a Row. + * Properties of an EventToken. * @memberof query - * @interface IRow - * @property {Array.|null} [lengths] Row lengths - * @property {Uint8Array|null} [values] Row values + * @interface IEventToken + * @property {number|Long|null} [timestamp] EventToken timestamp + * @property {string|null} [shard] EventToken shard + * @property {string|null} [position] EventToken position */ /** - * Constructs a new Row. + * Constructs a new EventToken. * @memberof query - * @classdesc Represents a Row. - * @implements IRow + * @classdesc Represents an EventToken. + * @implements IEventToken * @constructor - * @param {query.IRow=} [properties] Properties to set + * @param {query.IEventToken=} [properties] Properties to set */ - function Row(properties) { - this.lengths = []; + function EventToken(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -76649,100 +76532,103 @@ export const query = $root.query = (() => { } /** - * Row lengths. - * @member {Array.} lengths - * @memberof query.Row + * EventToken timestamp. + * @member {number|Long} timestamp + * @memberof query.EventToken * @instance */ - Row.prototype.lengths = $util.emptyArray; + EventToken.prototype.timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Row values. - * @member {Uint8Array} values - * @memberof query.Row + * EventToken shard. + * @member {string} shard + * @memberof query.EventToken * @instance */ - Row.prototype.values = $util.newBuffer([]); + EventToken.prototype.shard = ""; /** - * Creates a new Row instance using the specified properties. + * EventToken position. + * @member {string} position + * @memberof query.EventToken + * @instance + */ + EventToken.prototype.position = ""; + + /** + * Creates a new EventToken instance using the specified properties. * @function create - * @memberof query.Row + * @memberof query.EventToken * @static - * @param {query.IRow=} [properties] Properties to set - * @returns {query.Row} Row instance + * @param {query.IEventToken=} [properties] Properties to set + * @returns {query.EventToken} EventToken instance */ - Row.create = function create(properties) { - return new Row(properties); + EventToken.create = function create(properties) { + return new EventToken(properties); }; /** - * Encodes the specified Row message. Does not implicitly {@link query.Row.verify|verify} messages. + * Encodes the specified EventToken message. Does not implicitly {@link query.EventToken.verify|verify} messages. * @function encode - * @memberof query.Row + * @memberof query.EventToken * @static - * @param {query.IRow} message Row message or plain object to encode + * @param {query.IEventToken} message EventToken message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Row.encode = function encode(message, writer) { + EventToken.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.lengths != null && message.lengths.length) { - writer.uint32(/* id 1, wireType 2 =*/10).fork(); - for (let i = 0; i < message.lengths.length; ++i) - writer.sint64(message.lengths[i]); - writer.ldelim(); - } - if (message.values != null && Object.hasOwnProperty.call(message, "values")) - writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.values); + if (message.timestamp != null && Object.hasOwnProperty.call(message, "timestamp")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.timestamp); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.position); return writer; }; /** - * Encodes the specified Row message, length delimited. Does not implicitly {@link query.Row.verify|verify} messages. + * Encodes the specified EventToken message, length delimited. Does not implicitly {@link query.EventToken.verify|verify} messages. * @function encodeDelimited - * @memberof query.Row + * @memberof query.EventToken * @static - * @param {query.IRow} message Row message or plain object to encode + * @param {query.IEventToken} message EventToken message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Row.encodeDelimited = function encodeDelimited(message, writer) { + EventToken.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Row message from the specified reader or buffer. + * Decodes an EventToken message from the specified reader or buffer. * @function decode - * @memberof query.Row + * @memberof query.EventToken * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.Row} Row + * @returns {query.EventToken} EventToken * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Row.decode = function decode(reader, length) { + EventToken.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.query.Row(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.EventToken(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.lengths && message.lengths.length)) - message.lengths = []; - if ((tag & 7) === 2) { - let end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) - message.lengths.push(reader.sint64()); - } else - message.lengths.push(reader.sint64()); + message.timestamp = reader.int64(); break; } case 2: { - message.values = reader.bytes(); + message.shard = reader.string(); + break; + } + case 3: { + message.position = reader.string(); break; } default: @@ -76754,169 +76640,308 @@ export const query = $root.query = (() => { }; /** - * Decodes a Row message from the specified reader or buffer, length delimited. + * Decodes an EventToken message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.Row + * @memberof query.EventToken * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.Row} Row + * @returns {query.EventToken} EventToken * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Row.decodeDelimited = function decodeDelimited(reader) { + EventToken.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Row message. + * Verifies an EventToken message. * @function verify - * @memberof query.Row + * @memberof query.EventToken * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Row.verify = function verify(message) { + EventToken.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.lengths != null && message.hasOwnProperty("lengths")) { - if (!Array.isArray(message.lengths)) - return "lengths: array expected"; - for (let i = 0; i < message.lengths.length; ++i) - if (!$util.isInteger(message.lengths[i]) && !(message.lengths[i] && $util.isInteger(message.lengths[i].low) && $util.isInteger(message.lengths[i].high))) - return "lengths: integer|Long[] expected"; - } - if (message.values != null && message.hasOwnProperty("values")) - if (!(message.values && typeof message.values.length === "number" || $util.isString(message.values))) - return "values: buffer expected"; + if (message.timestamp != null && message.hasOwnProperty("timestamp")) + if (!$util.isInteger(message.timestamp) && !(message.timestamp && $util.isInteger(message.timestamp.low) && $util.isInteger(message.timestamp.high))) + return "timestamp: integer|Long expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; return null; }; /** - * Creates a Row message from a plain object. Also converts values to their respective internal types. + * Creates an EventToken message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.Row + * @memberof query.EventToken * @static * @param {Object.} object Plain object - * @returns {query.Row} Row + * @returns {query.EventToken} EventToken */ - Row.fromObject = function fromObject(object) { - if (object instanceof $root.query.Row) + EventToken.fromObject = function fromObject(object) { + if (object instanceof $root.query.EventToken) return object; - let message = new $root.query.Row(); - if (object.lengths) { - if (!Array.isArray(object.lengths)) - throw TypeError(".query.Row.lengths: array expected"); - message.lengths = []; - for (let i = 0; i < object.lengths.length; ++i) - if ($util.Long) - (message.lengths[i] = $util.Long.fromValue(object.lengths[i])).unsigned = false; - else if (typeof object.lengths[i] === "string") - message.lengths[i] = parseInt(object.lengths[i], 10); - else if (typeof object.lengths[i] === "number") - message.lengths[i] = object.lengths[i]; - else if (typeof object.lengths[i] === "object") - message.lengths[i] = new $util.LongBits(object.lengths[i].low >>> 0, object.lengths[i].high >>> 0).toNumber(); - } - if (object.values != null) - if (typeof object.values === "string") - $util.base64.decode(object.values, message.values = $util.newBuffer($util.base64.length(object.values)), 0); - else if (object.values.length >= 0) - message.values = object.values; + let message = new $root.query.EventToken(); + if (object.timestamp != null) + if ($util.Long) + (message.timestamp = $util.Long.fromValue(object.timestamp)).unsigned = false; + else if (typeof object.timestamp === "string") + message.timestamp = parseInt(object.timestamp, 10); + else if (typeof object.timestamp === "number") + message.timestamp = object.timestamp; + else if (typeof object.timestamp === "object") + message.timestamp = new $util.LongBits(object.timestamp.low >>> 0, object.timestamp.high >>> 0).toNumber(); + if (object.shard != null) + message.shard = String(object.shard); + if (object.position != null) + message.position = String(object.position); return message; }; /** - * Creates a plain object from a Row message. Also converts values to other types if specified. + * Creates a plain object from an EventToken message. Also converts values to other types if specified. * @function toObject - * @memberof query.Row + * @memberof query.EventToken * @static - * @param {query.Row} message Row + * @param {query.EventToken} message EventToken * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Row.toObject = function toObject(message, options) { + EventToken.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.lengths = []; - if (options.defaults) - if (options.bytes === String) - object.values = ""; - else { - object.values = []; - if (options.bytes !== Array) - object.values = $util.newBuffer(object.values); - } - if (message.lengths && message.lengths.length) { - object.lengths = []; - for (let j = 0; j < message.lengths.length; ++j) - if (typeof message.lengths[j] === "number") - object.lengths[j] = options.longs === String ? String(message.lengths[j]) : message.lengths[j]; - else - object.lengths[j] = options.longs === String ? $util.Long.prototype.toString.call(message.lengths[j]) : options.longs === Number ? new $util.LongBits(message.lengths[j].low >>> 0, message.lengths[j].high >>> 0).toNumber() : message.lengths[j]; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.timestamp = options.longs === String ? "0" : 0; + object.shard = ""; + object.position = ""; } - if (message.values != null && message.hasOwnProperty("values")) - object.values = options.bytes === String ? $util.base64.encode(message.values, 0, message.values.length) : options.bytes === Array ? Array.prototype.slice.call(message.values) : message.values; + if (message.timestamp != null && message.hasOwnProperty("timestamp")) + if (typeof message.timestamp === "number") + object.timestamp = options.longs === String ? String(message.timestamp) : message.timestamp; + else + object.timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.timestamp) : options.longs === Number ? new $util.LongBits(message.timestamp.low >>> 0, message.timestamp.high >>> 0).toNumber() : message.timestamp; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; return object; }; /** - * Converts this Row to JSON. + * Converts this EventToken to JSON. * @function toJSON - * @memberof query.Row + * @memberof query.EventToken * @instance * @returns {Object.} JSON object */ - Row.prototype.toJSON = function toJSON() { + EventToken.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Row + * Gets the default type url for EventToken * @function getTypeUrl - * @memberof query.Row + * @memberof query.EventToken * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Row.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + EventToken.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.Row"; + return typeUrlPrefix + "/query.EventToken"; }; - return Row; + return EventToken; })(); - query.QueryResult = (function() { + /** + * MySqlFlag enum. + * @name query.MySqlFlag + * @enum {number} + * @property {number} EMPTY=0 EMPTY value + * @property {number} NOT_NULL_FLAG=1 NOT_NULL_FLAG value + * @property {number} PRI_KEY_FLAG=2 PRI_KEY_FLAG value + * @property {number} UNIQUE_KEY_FLAG=4 UNIQUE_KEY_FLAG value + * @property {number} MULTIPLE_KEY_FLAG=8 MULTIPLE_KEY_FLAG value + * @property {number} BLOB_FLAG=16 BLOB_FLAG value + * @property {number} UNSIGNED_FLAG=32 UNSIGNED_FLAG value + * @property {number} ZEROFILL_FLAG=64 ZEROFILL_FLAG value + * @property {number} BINARY_FLAG=128 BINARY_FLAG value + * @property {number} ENUM_FLAG=256 ENUM_FLAG value + * @property {number} AUTO_INCREMENT_FLAG=512 AUTO_INCREMENT_FLAG value + * @property {number} TIMESTAMP_FLAG=1024 TIMESTAMP_FLAG value + * @property {number} SET_FLAG=2048 SET_FLAG value + * @property {number} NO_DEFAULT_VALUE_FLAG=4096 NO_DEFAULT_VALUE_FLAG value + * @property {number} ON_UPDATE_NOW_FLAG=8192 ON_UPDATE_NOW_FLAG value + * @property {number} NUM_FLAG=32768 NUM_FLAG value + * @property {number} PART_KEY_FLAG=16384 PART_KEY_FLAG value + * @property {number} GROUP_FLAG=32768 GROUP_FLAG value + * @property {number} UNIQUE_FLAG=65536 UNIQUE_FLAG value + * @property {number} BINCMP_FLAG=131072 BINCMP_FLAG value + */ + query.MySqlFlag = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "EMPTY"] = 0; + values[valuesById[1] = "NOT_NULL_FLAG"] = 1; + values[valuesById[2] = "PRI_KEY_FLAG"] = 2; + values[valuesById[4] = "UNIQUE_KEY_FLAG"] = 4; + values[valuesById[8] = "MULTIPLE_KEY_FLAG"] = 8; + values[valuesById[16] = "BLOB_FLAG"] = 16; + values[valuesById[32] = "UNSIGNED_FLAG"] = 32; + values[valuesById[64] = "ZEROFILL_FLAG"] = 64; + values[valuesById[128] = "BINARY_FLAG"] = 128; + values[valuesById[256] = "ENUM_FLAG"] = 256; + values[valuesById[512] = "AUTO_INCREMENT_FLAG"] = 512; + values[valuesById[1024] = "TIMESTAMP_FLAG"] = 1024; + values[valuesById[2048] = "SET_FLAG"] = 2048; + values[valuesById[4096] = "NO_DEFAULT_VALUE_FLAG"] = 4096; + values[valuesById[8192] = "ON_UPDATE_NOW_FLAG"] = 8192; + values[valuesById[32768] = "NUM_FLAG"] = 32768; + values[valuesById[16384] = "PART_KEY_FLAG"] = 16384; + values["GROUP_FLAG"] = 32768; + values[valuesById[65536] = "UNIQUE_FLAG"] = 65536; + values[valuesById[131072] = "BINCMP_FLAG"] = 131072; + return values; + })(); + + /** + * Flag enum. + * @name query.Flag + * @enum {number} + * @property {number} NONE=0 NONE value + * @property {number} ISINTEGRAL=256 ISINTEGRAL value + * @property {number} ISUNSIGNED=512 ISUNSIGNED value + * @property {number} ISFLOAT=1024 ISFLOAT value + * @property {number} ISQUOTED=2048 ISQUOTED value + * @property {number} ISTEXT=4096 ISTEXT value + * @property {number} ISBINARY=8192 ISBINARY value + */ + query.Flag = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "NONE"] = 0; + values[valuesById[256] = "ISINTEGRAL"] = 256; + values[valuesById[512] = "ISUNSIGNED"] = 512; + values[valuesById[1024] = "ISFLOAT"] = 1024; + values[valuesById[2048] = "ISQUOTED"] = 2048; + values[valuesById[4096] = "ISTEXT"] = 4096; + values[valuesById[8192] = "ISBINARY"] = 8192; + return values; + })(); + + /** + * Type enum. + * @name query.Type + * @enum {number} + * @property {number} NULL_TYPE=0 NULL_TYPE value + * @property {number} INT8=257 INT8 value + * @property {number} UINT8=770 UINT8 value + * @property {number} INT16=259 INT16 value + * @property {number} UINT16=772 UINT16 value + * @property {number} INT24=261 INT24 value + * @property {number} UINT24=774 UINT24 value + * @property {number} INT32=263 INT32 value + * @property {number} UINT32=776 UINT32 value + * @property {number} INT64=265 INT64 value + * @property {number} UINT64=778 UINT64 value + * @property {number} FLOAT32=1035 FLOAT32 value + * @property {number} FLOAT64=1036 FLOAT64 value + * @property {number} TIMESTAMP=2061 TIMESTAMP value + * @property {number} DATE=2062 DATE value + * @property {number} TIME=2063 TIME value + * @property {number} DATETIME=2064 DATETIME value + * @property {number} YEAR=785 YEAR value + * @property {number} DECIMAL=18 DECIMAL value + * @property {number} TEXT=6163 TEXT value + * @property {number} BLOB=10260 BLOB value + * @property {number} VARCHAR=6165 VARCHAR value + * @property {number} VARBINARY=10262 VARBINARY value + * @property {number} CHAR=6167 CHAR value + * @property {number} BINARY=10264 BINARY value + * @property {number} BIT=2073 BIT value + * @property {number} ENUM=2074 ENUM value + * @property {number} SET=2075 SET value + * @property {number} TUPLE=28 TUPLE value + * @property {number} GEOMETRY=2077 GEOMETRY value + * @property {number} JSON=2078 JSON value + * @property {number} EXPRESSION=31 EXPRESSION value + * @property {number} HEXNUM=4128 HEXNUM value + * @property {number} HEXVAL=4129 HEXVAL value + * @property {number} BITNUM=4130 BITNUM value + */ + query.Type = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "NULL_TYPE"] = 0; + values[valuesById[257] = "INT8"] = 257; + values[valuesById[770] = "UINT8"] = 770; + values[valuesById[259] = "INT16"] = 259; + values[valuesById[772] = "UINT16"] = 772; + values[valuesById[261] = "INT24"] = 261; + values[valuesById[774] = "UINT24"] = 774; + values[valuesById[263] = "INT32"] = 263; + values[valuesById[776] = "UINT32"] = 776; + values[valuesById[265] = "INT64"] = 265; + values[valuesById[778] = "UINT64"] = 778; + values[valuesById[1035] = "FLOAT32"] = 1035; + values[valuesById[1036] = "FLOAT64"] = 1036; + values[valuesById[2061] = "TIMESTAMP"] = 2061; + values[valuesById[2062] = "DATE"] = 2062; + values[valuesById[2063] = "TIME"] = 2063; + values[valuesById[2064] = "DATETIME"] = 2064; + values[valuesById[785] = "YEAR"] = 785; + values[valuesById[18] = "DECIMAL"] = 18; + values[valuesById[6163] = "TEXT"] = 6163; + values[valuesById[10260] = "BLOB"] = 10260; + values[valuesById[6165] = "VARCHAR"] = 6165; + values[valuesById[10262] = "VARBINARY"] = 10262; + values[valuesById[6167] = "CHAR"] = 6167; + values[valuesById[10264] = "BINARY"] = 10264; + values[valuesById[2073] = "BIT"] = 2073; + values[valuesById[2074] = "ENUM"] = 2074; + values[valuesById[2075] = "SET"] = 2075; + values[valuesById[28] = "TUPLE"] = 28; + values[valuesById[2077] = "GEOMETRY"] = 2077; + values[valuesById[2078] = "JSON"] = 2078; + values[valuesById[31] = "EXPRESSION"] = 31; + values[valuesById[4128] = "HEXNUM"] = 4128; + values[valuesById[4129] = "HEXVAL"] = 4129; + values[valuesById[4130] = "BITNUM"] = 4130; + return values; + })(); + + query.Value = (function() { /** - * Properties of a QueryResult. + * Properties of a Value. * @memberof query - * @interface IQueryResult - * @property {Array.|null} [fields] QueryResult fields - * @property {number|Long|null} [rows_affected] QueryResult rows_affected - * @property {number|Long|null} [insert_id] QueryResult insert_id - * @property {Array.|null} [rows] QueryResult rows - * @property {string|null} [info] QueryResult info - * @property {string|null} [session_state_changes] QueryResult session_state_changes + * @interface IValue + * @property {query.Type|null} [type] Value type + * @property {Uint8Array|null} [value] Value value */ /** - * Constructs a new QueryResult. + * Constructs a new Value. * @memberof query - * @classdesc Represents a QueryResult. - * @implements IQueryResult + * @classdesc Represents a Value. + * @implements IValue * @constructor - * @param {query.IQueryResult=} [properties] Properties to set + * @param {query.IValue=} [properties] Properties to set */ - function QueryResult(properties) { - this.fields = []; - this.rows = []; + function Value(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -76924,151 +76949,89 @@ export const query = $root.query = (() => { } /** - * QueryResult fields. - * @member {Array.} fields - * @memberof query.QueryResult - * @instance - */ - QueryResult.prototype.fields = $util.emptyArray; - - /** - * QueryResult rows_affected. - * @member {number|Long} rows_affected - * @memberof query.QueryResult - * @instance - */ - QueryResult.prototype.rows_affected = $util.Long ? $util.Long.fromBits(0,0,true) : 0; - - /** - * QueryResult insert_id. - * @member {number|Long} insert_id - * @memberof query.QueryResult - * @instance - */ - QueryResult.prototype.insert_id = $util.Long ? $util.Long.fromBits(0,0,true) : 0; - - /** - * QueryResult rows. - * @member {Array.} rows - * @memberof query.QueryResult - * @instance - */ - QueryResult.prototype.rows = $util.emptyArray; - - /** - * QueryResult info. - * @member {string} info - * @memberof query.QueryResult + * Value type. + * @member {query.Type} type + * @memberof query.Value * @instance */ - QueryResult.prototype.info = ""; + Value.prototype.type = 0; /** - * QueryResult session_state_changes. - * @member {string} session_state_changes - * @memberof query.QueryResult + * Value value. + * @member {Uint8Array} value + * @memberof query.Value * @instance */ - QueryResult.prototype.session_state_changes = ""; + Value.prototype.value = $util.newBuffer([]); /** - * Creates a new QueryResult instance using the specified properties. + * Creates a new Value instance using the specified properties. * @function create - * @memberof query.QueryResult + * @memberof query.Value * @static - * @param {query.IQueryResult=} [properties] Properties to set - * @returns {query.QueryResult} QueryResult instance + * @param {query.IValue=} [properties] Properties to set + * @returns {query.Value} Value instance */ - QueryResult.create = function create(properties) { - return new QueryResult(properties); + Value.create = function create(properties) { + return new Value(properties); }; /** - * Encodes the specified QueryResult message. Does not implicitly {@link query.QueryResult.verify|verify} messages. + * Encodes the specified Value message. Does not implicitly {@link query.Value.verify|verify} messages. * @function encode - * @memberof query.QueryResult + * @memberof query.Value * @static - * @param {query.IQueryResult} message QueryResult message or plain object to encode + * @param {query.IValue} message Value message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - QueryResult.encode = function encode(message, writer) { + Value.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.fields != null && message.fields.length) - for (let i = 0; i < message.fields.length; ++i) - $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.rows_affected != null && Object.hasOwnProperty.call(message, "rows_affected")) - writer.uint32(/* id 2, wireType 0 =*/16).uint64(message.rows_affected); - if (message.insert_id != null && Object.hasOwnProperty.call(message, "insert_id")) - writer.uint32(/* id 3, wireType 0 =*/24).uint64(message.insert_id); - if (message.rows != null && message.rows.length) - for (let i = 0; i < message.rows.length; ++i) - $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.info != null && Object.hasOwnProperty.call(message, "info")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.info); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.session_state_changes); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); + if (message.value != null && Object.hasOwnProperty.call(message, "value")) + writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.value); return writer; }; /** - * Encodes the specified QueryResult message, length delimited. Does not implicitly {@link query.QueryResult.verify|verify} messages. + * Encodes the specified Value message, length delimited. Does not implicitly {@link query.Value.verify|verify} messages. * @function encodeDelimited - * @memberof query.QueryResult + * @memberof query.Value * @static - * @param {query.IQueryResult} message QueryResult message or plain object to encode + * @param {query.IValue} message Value message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - QueryResult.encodeDelimited = function encodeDelimited(message, writer) { + Value.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a QueryResult message from the specified reader or buffer. + * Decodes a Value message from the specified reader or buffer. * @function decode - * @memberof query.QueryResult + * @memberof query.Value * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.QueryResult} QueryResult + * @returns {query.Value} Value * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - QueryResult.decode = function decode(reader, length) { + Value.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.query.QueryResult(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.Value(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.fields && message.fields.length)) - message.fields = []; - message.fields.push($root.query.Field.decode(reader, reader.uint32())); + message.type = reader.int32(); break; } case 2: { - message.rows_affected = reader.uint64(); - break; - } - case 3: { - message.insert_id = reader.uint64(); - break; - } - case 4: { - if (!(message.rows && message.rows.length)) - message.rows = []; - message.rows.push($root.query.Row.decode(reader, reader.uint32())); - break; - } - case 6: { - message.info = reader.string(); - break; - } - case 7: { - message.session_state_changes = reader.string(); + message.value = reader.bytes(); break; } default: @@ -77080,228 +77043,327 @@ export const query = $root.query = (() => { }; /** - * Decodes a QueryResult message from the specified reader or buffer, length delimited. + * Decodes a Value message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.QueryResult + * @memberof query.Value * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.QueryResult} QueryResult + * @returns {query.Value} Value * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - QueryResult.decodeDelimited = function decodeDelimited(reader) { + Value.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a QueryResult message. + * Verifies a Value message. * @function verify - * @memberof query.QueryResult + * @memberof query.Value * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - QueryResult.verify = function verify(message) { + Value.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.fields != null && message.hasOwnProperty("fields")) { - if (!Array.isArray(message.fields)) - return "fields: array expected"; - for (let i = 0; i < message.fields.length; ++i) { - let error = $root.query.Field.verify(message.fields[i]); - if (error) - return "fields." + error; - } - } - if (message.rows_affected != null && message.hasOwnProperty("rows_affected")) - if (!$util.isInteger(message.rows_affected) && !(message.rows_affected && $util.isInteger(message.rows_affected.low) && $util.isInteger(message.rows_affected.high))) - return "rows_affected: integer|Long expected"; - if (message.insert_id != null && message.hasOwnProperty("insert_id")) - if (!$util.isInteger(message.insert_id) && !(message.insert_id && $util.isInteger(message.insert_id.low) && $util.isInteger(message.insert_id.high))) - return "insert_id: integer|Long expected"; - if (message.rows != null && message.hasOwnProperty("rows")) { - if (!Array.isArray(message.rows)) - return "rows: array expected"; - for (let i = 0; i < message.rows.length; ++i) { - let error = $root.query.Row.verify(message.rows[i]); - if (error) - return "rows." + error; + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "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; } - } - if (message.info != null && message.hasOwnProperty("info")) - if (!$util.isString(message.info)) - return "info: string expected"; - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; + if (message.value != null && message.hasOwnProperty("value")) + if (!(message.value && typeof message.value.length === "number" || $util.isString(message.value))) + return "value: buffer expected"; return null; }; /** - * Creates a QueryResult message from a plain object. Also converts values to their respective internal types. + * Creates a Value message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.QueryResult + * @memberof query.Value * @static * @param {Object.} object Plain object - * @returns {query.QueryResult} QueryResult + * @returns {query.Value} Value */ - QueryResult.fromObject = function fromObject(object) { - if (object instanceof $root.query.QueryResult) + Value.fromObject = function fromObject(object) { + if (object instanceof $root.query.Value) return object; - let message = new $root.query.QueryResult(); - if (object.fields) { - if (!Array.isArray(object.fields)) - throw TypeError(".query.QueryResult.fields: array expected"); - message.fields = []; - for (let i = 0; i < object.fields.length; ++i) { - if (typeof object.fields[i] !== "object") - throw TypeError(".query.QueryResult.fields: object expected"); - message.fields[i] = $root.query.Field.fromObject(object.fields[i]); - } - } - if (object.rows_affected != null) - if ($util.Long) - (message.rows_affected = $util.Long.fromValue(object.rows_affected)).unsigned = true; - else if (typeof object.rows_affected === "string") - message.rows_affected = parseInt(object.rows_affected, 10); - else if (typeof object.rows_affected === "number") - message.rows_affected = object.rows_affected; - else if (typeof object.rows_affected === "object") - message.rows_affected = new $util.LongBits(object.rows_affected.low >>> 0, object.rows_affected.high >>> 0).toNumber(true); - if (object.insert_id != null) - if ($util.Long) - (message.insert_id = $util.Long.fromValue(object.insert_id)).unsigned = true; - else if (typeof object.insert_id === "string") - message.insert_id = parseInt(object.insert_id, 10); - else if (typeof object.insert_id === "number") - message.insert_id = object.insert_id; - else if (typeof object.insert_id === "object") - message.insert_id = new $util.LongBits(object.insert_id.low >>> 0, object.insert_id.high >>> 0).toNumber(true); - if (object.rows) { - if (!Array.isArray(object.rows)) - throw TypeError(".query.QueryResult.rows: array expected"); - message.rows = []; - for (let i = 0; i < object.rows.length; ++i) { - if (typeof object.rows[i] !== "object") - throw TypeError(".query.QueryResult.rows: object expected"); - message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + let message = new $root.query.Value(); + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; } + break; + case "NULL_TYPE": + case 0: + message.type = 0; + break; + case "INT8": + case 257: + message.type = 257; + break; + case "UINT8": + case 770: + message.type = 770; + break; + case "INT16": + case 259: + message.type = 259; + break; + case "UINT16": + case 772: + message.type = 772; + break; + case "INT24": + case 261: + message.type = 261; + break; + case "UINT24": + case 774: + message.type = 774; + break; + case "INT32": + case 263: + message.type = 263; + break; + case "UINT32": + case 776: + message.type = 776; + break; + case "INT64": + case 265: + message.type = 265; + break; + case "UINT64": + case 778: + message.type = 778; + break; + case "FLOAT32": + case 1035: + message.type = 1035; + break; + case "FLOAT64": + case 1036: + message.type = 1036; + break; + case "TIMESTAMP": + case 2061: + message.type = 2061; + break; + case "DATE": + case 2062: + message.type = 2062; + break; + case "TIME": + case 2063: + message.type = 2063; + break; + case "DATETIME": + case 2064: + message.type = 2064; + break; + case "YEAR": + case 785: + message.type = 785; + break; + case "DECIMAL": + case 18: + message.type = 18; + break; + case "TEXT": + case 6163: + message.type = 6163; + break; + case "BLOB": + case 10260: + message.type = 10260; + break; + case "VARCHAR": + case 6165: + message.type = 6165; + break; + case "VARBINARY": + case 10262: + message.type = 10262; + break; + case "CHAR": + case 6167: + message.type = 6167; + break; + case "BINARY": + case 10264: + message.type = 10264; + break; + case "BIT": + case 2073: + message.type = 2073; + break; + case "ENUM": + case 2074: + message.type = 2074; + break; + case "SET": + case 2075: + message.type = 2075; + break; + case "TUPLE": + case 28: + message.type = 28; + break; + case "GEOMETRY": + case 2077: + message.type = 2077; + break; + case "JSON": + case 2078: + message.type = 2078; + break; + case "EXPRESSION": + case 31: + message.type = 31; + break; + case "HEXNUM": + case 4128: + message.type = 4128; + break; + case "HEXVAL": + case 4129: + message.type = 4129; + break; + case "BITNUM": + case 4130: + message.type = 4130; + break; } - if (object.info != null) - message.info = String(object.info); - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); + if (object.value != null) + if (typeof object.value === "string") + $util.base64.decode(object.value, message.value = $util.newBuffer($util.base64.length(object.value)), 0); + else if (object.value.length >= 0) + message.value = object.value; return message; }; /** - * Creates a plain object from a QueryResult message. Also converts values to other types if specified. + * Creates a plain object from a Value message. Also converts values to other types if specified. * @function toObject - * @memberof query.QueryResult + * @memberof query.Value * @static - * @param {query.QueryResult} message QueryResult + * @param {query.Value} message Value * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - QueryResult.toObject = function toObject(message, options) { + Value.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.fields = []; - object.rows = []; - } if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, true); - object.rows_affected = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.rows_affected = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, true); - object.insert_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.insert_id = options.longs === String ? "0" : 0; - object.info = ""; - object.session_state_changes = ""; - } - if (message.fields && message.fields.length) { - object.fields = []; - for (let j = 0; j < message.fields.length; ++j) - object.fields[j] = $root.query.Field.toObject(message.fields[j], options); - } - if (message.rows_affected != null && message.hasOwnProperty("rows_affected")) - if (typeof message.rows_affected === "number") - object.rows_affected = options.longs === String ? String(message.rows_affected) : message.rows_affected; - else - object.rows_affected = options.longs === String ? $util.Long.prototype.toString.call(message.rows_affected) : options.longs === Number ? new $util.LongBits(message.rows_affected.low >>> 0, message.rows_affected.high >>> 0).toNumber(true) : message.rows_affected; - if (message.insert_id != null && message.hasOwnProperty("insert_id")) - if (typeof message.insert_id === "number") - object.insert_id = options.longs === String ? String(message.insert_id) : message.insert_id; - else - object.insert_id = options.longs === String ? $util.Long.prototype.toString.call(message.insert_id) : options.longs === Number ? new $util.LongBits(message.insert_id.low >>> 0, message.insert_id.high >>> 0).toNumber(true) : message.insert_id; - if (message.rows && message.rows.length) { - object.rows = []; - for (let j = 0; j < message.rows.length; ++j) - object.rows[j] = $root.query.Row.toObject(message.rows[j], options); + object.type = options.enums === String ? "NULL_TYPE" : 0; + if (options.bytes === String) + object.value = ""; + else { + object.value = []; + if (options.bytes !== Array) + object.value = $util.newBuffer(object.value); + } } - if (message.info != null && message.hasOwnProperty("info")) - object.info = message.info; - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; + if (message.value != null && message.hasOwnProperty("value")) + object.value = options.bytes === String ? $util.base64.encode(message.value, 0, message.value.length) : options.bytes === Array ? Array.prototype.slice.call(message.value) : message.value; return object; }; /** - * Converts this QueryResult to JSON. + * Converts this Value to JSON. * @function toJSON - * @memberof query.QueryResult + * @memberof query.Value * @instance * @returns {Object.} JSON object */ - QueryResult.prototype.toJSON = function toJSON() { + Value.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for QueryResult + * Gets the default type url for Value * @function getTypeUrl - * @memberof query.QueryResult + * @memberof query.Value * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - QueryResult.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Value.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.QueryResult"; + return typeUrlPrefix + "/query.Value"; }; - return QueryResult; + return Value; })(); - query.QueryWarning = (function() { + query.BindVariable = (function() { /** - * Properties of a QueryWarning. + * Properties of a BindVariable. * @memberof query - * @interface IQueryWarning - * @property {number|null} [code] QueryWarning code - * @property {string|null} [message] QueryWarning message + * @interface IBindVariable + * @property {query.Type|null} [type] BindVariable type + * @property {Uint8Array|null} [value] BindVariable value + * @property {Array.|null} [values] BindVariable values */ /** - * Constructs a new QueryWarning. + * Constructs a new BindVariable. * @memberof query - * @classdesc Represents a QueryWarning. - * @implements IQueryWarning + * @classdesc Represents a BindVariable. + * @implements IBindVariable * @constructor - * @param {query.IQueryWarning=} [properties] Properties to set + * @param {query.IBindVariable=} [properties] Properties to set */ - function QueryWarning(properties) { + function BindVariable(properties) { + this.values = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -77309,89 +77371,106 @@ export const query = $root.query = (() => { } /** - * QueryWarning code. - * @member {number} code - * @memberof query.QueryWarning + * BindVariable type. + * @member {query.Type} type + * @memberof query.BindVariable * @instance */ - QueryWarning.prototype.code = 0; + BindVariable.prototype.type = 0; /** - * QueryWarning message. - * @member {string} message - * @memberof query.QueryWarning + * BindVariable value. + * @member {Uint8Array} value + * @memberof query.BindVariable * @instance */ - QueryWarning.prototype.message = ""; + BindVariable.prototype.value = $util.newBuffer([]); /** - * Creates a new QueryWarning instance using the specified properties. + * BindVariable values. + * @member {Array.} values + * @memberof query.BindVariable + * @instance + */ + BindVariable.prototype.values = $util.emptyArray; + + /** + * Creates a new BindVariable instance using the specified properties. * @function create - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static - * @param {query.IQueryWarning=} [properties] Properties to set - * @returns {query.QueryWarning} QueryWarning instance + * @param {query.IBindVariable=} [properties] Properties to set + * @returns {query.BindVariable} BindVariable instance */ - QueryWarning.create = function create(properties) { - return new QueryWarning(properties); + BindVariable.create = function create(properties) { + return new BindVariable(properties); }; /** - * Encodes the specified QueryWarning message. Does not implicitly {@link query.QueryWarning.verify|verify} messages. + * Encodes the specified BindVariable message. Does not implicitly {@link query.BindVariable.verify|verify} messages. * @function encode - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static - * @param {query.IQueryWarning} message QueryWarning message or plain object to encode + * @param {query.IBindVariable} message BindVariable message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - QueryWarning.encode = function encode(message, writer) { + BindVariable.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.code != null && Object.hasOwnProperty.call(message, "code")) - writer.uint32(/* id 1, wireType 0 =*/8).uint32(message.code); - if (message.message != null && Object.hasOwnProperty.call(message, "message")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.message); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.type); + if (message.value != null && Object.hasOwnProperty.call(message, "value")) + writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.value); + if (message.values != null && message.values.length) + for (let i = 0; i < message.values.length; ++i) + $root.query.Value.encode(message.values[i], writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified QueryWarning message, length delimited. Does not implicitly {@link query.QueryWarning.verify|verify} messages. + * Encodes the specified BindVariable message, length delimited. Does not implicitly {@link query.BindVariable.verify|verify} messages. * @function encodeDelimited - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static - * @param {query.IQueryWarning} message QueryWarning message or plain object to encode + * @param {query.IBindVariable} message BindVariable message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - QueryWarning.encodeDelimited = function encodeDelimited(message, writer) { + BindVariable.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a QueryWarning message from the specified reader or buffer. + * Decodes a BindVariable message from the specified reader or buffer. * @function decode - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.QueryWarning} QueryWarning + * @returns {query.BindVariable} BindVariable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - QueryWarning.decode = function decode(reader, length) { + BindVariable.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.query.QueryWarning(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BindVariable(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.code = reader.uint32(); + message.type = reader.int32(); break; } case 2: { - message.message = reader.string(); + message.value = reader.bytes(); + break; + } + case 3: { + if (!(message.values && message.values.length)) + message.values = []; + message.values.push($root.query.Value.decode(reader, reader.uint32())); break; } default: @@ -77403,133 +77482,352 @@ export const query = $root.query = (() => { }; /** - * Decodes a QueryWarning message from the specified reader or buffer, length delimited. + * Decodes a BindVariable message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.QueryWarning} QueryWarning + * @returns {query.BindVariable} BindVariable * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - QueryWarning.decodeDelimited = function decodeDelimited(reader) { + BindVariable.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a QueryWarning message. + * Verifies a BindVariable message. * @function verify - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - QueryWarning.verify = function verify(message) { + BindVariable.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.code != null && message.hasOwnProperty("code")) - if (!$util.isInteger(message.code)) - return "code: integer expected"; - if (message.message != null && message.hasOwnProperty("message")) - if (!$util.isString(message.message)) - return "message: string expected"; + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "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; + } + if (message.value != null && message.hasOwnProperty("value")) + if (!(message.value && typeof message.value.length === "number" || $util.isString(message.value))) + return "value: buffer expected"; + if (message.values != null && message.hasOwnProperty("values")) { + if (!Array.isArray(message.values)) + return "values: array expected"; + for (let i = 0; i < message.values.length; ++i) { + let error = $root.query.Value.verify(message.values[i]); + if (error) + return "values." + error; + } + } return null; }; /** - * Creates a QueryWarning message from a plain object. Also converts values to their respective internal types. + * Creates a BindVariable message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.QueryWarning + * @memberof query.BindVariable * @static * @param {Object.} object Plain object - * @returns {query.QueryWarning} QueryWarning + * @returns {query.BindVariable} BindVariable */ - QueryWarning.fromObject = function fromObject(object) { - if (object instanceof $root.query.QueryWarning) + BindVariable.fromObject = function fromObject(object) { + if (object instanceof $root.query.BindVariable) return object; - let message = new $root.query.QueryWarning(); - if (object.code != null) - message.code = object.code >>> 0; - if (object.message != null) - message.message = String(object.message); - return message; - }; - - /** - * Creates a plain object from a QueryWarning message. Also converts values to other types if specified. - * @function toObject - * @memberof query.QueryWarning - * @static - * @param {query.QueryWarning} message QueryWarning - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - QueryWarning.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.code = 0; - object.message = ""; - } - if (message.code != null && message.hasOwnProperty("code")) - object.code = message.code; - if (message.message != null && message.hasOwnProperty("message")) - object.message = message.message; - return object; - }; - - /** - * Converts this QueryWarning to JSON. - * @function toJSON - * @memberof query.QueryWarning - * @instance - * @returns {Object.} JSON object - */ - QueryWarning.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for QueryWarning - * @function getTypeUrl - * @memberof query.QueryWarning - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - QueryWarning.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/query.QueryWarning"; - }; - - return QueryWarning; - })(); - - query.StreamEvent = (function() { - - /** - * Properties of a StreamEvent. + let message = new $root.query.BindVariable(); + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; + } + break; + case "NULL_TYPE": + case 0: + message.type = 0; + break; + case "INT8": + case 257: + message.type = 257; + break; + case "UINT8": + case 770: + message.type = 770; + break; + case "INT16": + case 259: + message.type = 259; + break; + case "UINT16": + case 772: + message.type = 772; + break; + case "INT24": + case 261: + message.type = 261; + break; + case "UINT24": + case 774: + message.type = 774; + break; + case "INT32": + case 263: + message.type = 263; + break; + case "UINT32": + case 776: + message.type = 776; + break; + case "INT64": + case 265: + message.type = 265; + break; + case "UINT64": + case 778: + message.type = 778; + break; + case "FLOAT32": + case 1035: + message.type = 1035; + break; + case "FLOAT64": + case 1036: + message.type = 1036; + break; + case "TIMESTAMP": + case 2061: + message.type = 2061; + break; + case "DATE": + case 2062: + message.type = 2062; + break; + case "TIME": + case 2063: + message.type = 2063; + break; + case "DATETIME": + case 2064: + message.type = 2064; + break; + case "YEAR": + case 785: + message.type = 785; + break; + case "DECIMAL": + case 18: + message.type = 18; + break; + case "TEXT": + case 6163: + message.type = 6163; + break; + case "BLOB": + case 10260: + message.type = 10260; + break; + case "VARCHAR": + case 6165: + message.type = 6165; + break; + case "VARBINARY": + case 10262: + message.type = 10262; + break; + case "CHAR": + case 6167: + message.type = 6167; + break; + case "BINARY": + case 10264: + message.type = 10264; + break; + case "BIT": + case 2073: + message.type = 2073; + break; + case "ENUM": + case 2074: + message.type = 2074; + break; + case "SET": + case 2075: + message.type = 2075; + break; + case "TUPLE": + case 28: + message.type = 28; + break; + case "GEOMETRY": + case 2077: + message.type = 2077; + break; + case "JSON": + case 2078: + message.type = 2078; + break; + case "EXPRESSION": + case 31: + message.type = 31; + break; + case "HEXNUM": + case 4128: + message.type = 4128; + break; + case "HEXVAL": + case 4129: + message.type = 4129; + break; + case "BITNUM": + case 4130: + message.type = 4130; + break; + } + if (object.value != null) + if (typeof object.value === "string") + $util.base64.decode(object.value, message.value = $util.newBuffer($util.base64.length(object.value)), 0); + else if (object.value.length >= 0) + message.value = object.value; + if (object.values) { + if (!Array.isArray(object.values)) + throw TypeError(".query.BindVariable.values: array expected"); + message.values = []; + for (let i = 0; i < object.values.length; ++i) { + if (typeof object.values[i] !== "object") + throw TypeError(".query.BindVariable.values: object expected"); + message.values[i] = $root.query.Value.fromObject(object.values[i]); + } + } + return message; + }; + + /** + * Creates a plain object from a BindVariable message. Also converts values to other types if specified. + * @function toObject + * @memberof query.BindVariable + * @static + * @param {query.BindVariable} message BindVariable + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + BindVariable.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.values = []; + if (options.defaults) { + object.type = options.enums === String ? "NULL_TYPE" : 0; + if (options.bytes === String) + object.value = ""; + else { + object.value = []; + if (options.bytes !== Array) + object.value = $util.newBuffer(object.value); + } + } + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; + if (message.value != null && message.hasOwnProperty("value")) + object.value = options.bytes === String ? $util.base64.encode(message.value, 0, message.value.length) : options.bytes === Array ? Array.prototype.slice.call(message.value) : message.value; + if (message.values && message.values.length) { + object.values = []; + for (let j = 0; j < message.values.length; ++j) + object.values[j] = $root.query.Value.toObject(message.values[j], options); + } + return object; + }; + + /** + * Converts this BindVariable to JSON. + * @function toJSON + * @memberof query.BindVariable + * @instance + * @returns {Object.} JSON object + */ + BindVariable.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for BindVariable + * @function getTypeUrl + * @memberof query.BindVariable + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + BindVariable.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/query.BindVariable"; + }; + + return BindVariable; + })(); + + query.BoundQuery = (function() { + + /** + * Properties of a BoundQuery. * @memberof query - * @interface IStreamEvent - * @property {Array.|null} [statements] StreamEvent statements - * @property {query.IEventToken|null} [event_token] StreamEvent event_token + * @interface IBoundQuery + * @property {string|null} [sql] BoundQuery sql + * @property {Object.|null} [bind_variables] BoundQuery bind_variables */ /** - * Constructs a new StreamEvent. + * Constructs a new BoundQuery. * @memberof query - * @classdesc Represents a StreamEvent. - * @implements IStreamEvent + * @classdesc Represents a BoundQuery. + * @implements IBoundQuery * @constructor - * @param {query.IStreamEvent=} [properties] Properties to set + * @param {query.IBoundQuery=} [properties] Properties to set */ - function StreamEvent(properties) { - this.statements = []; + function BoundQuery(properties) { + this.bind_variables = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -77537,92 +77835,111 @@ export const query = $root.query = (() => { } /** - * StreamEvent statements. - * @member {Array.} statements - * @memberof query.StreamEvent + * BoundQuery sql. + * @member {string} sql + * @memberof query.BoundQuery * @instance */ - StreamEvent.prototype.statements = $util.emptyArray; + BoundQuery.prototype.sql = ""; /** - * StreamEvent event_token. - * @member {query.IEventToken|null|undefined} event_token - * @memberof query.StreamEvent + * BoundQuery bind_variables. + * @member {Object.} bind_variables + * @memberof query.BoundQuery * @instance */ - StreamEvent.prototype.event_token = null; + BoundQuery.prototype.bind_variables = $util.emptyObject; /** - * Creates a new StreamEvent instance using the specified properties. + * Creates a new BoundQuery instance using the specified properties. * @function create - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static - * @param {query.IStreamEvent=} [properties] Properties to set - * @returns {query.StreamEvent} StreamEvent instance + * @param {query.IBoundQuery=} [properties] Properties to set + * @returns {query.BoundQuery} BoundQuery instance */ - StreamEvent.create = function create(properties) { - return new StreamEvent(properties); + BoundQuery.create = function create(properties) { + return new BoundQuery(properties); }; /** - * Encodes the specified StreamEvent message. Does not implicitly {@link query.StreamEvent.verify|verify} messages. + * Encodes the specified BoundQuery message. Does not implicitly {@link query.BoundQuery.verify|verify} messages. * @function encode - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static - * @param {query.IStreamEvent} message StreamEvent message or plain object to encode + * @param {query.IBoundQuery} message BoundQuery message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamEvent.encode = function encode(message, writer) { + BoundQuery.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.statements != null && message.statements.length) - for (let i = 0; i < message.statements.length; ++i) - $root.query.StreamEvent.Statement.encode(message.statements[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.event_token != null && Object.hasOwnProperty.call(message, "event_token")) - $root.query.EventToken.encode(message.event_token, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.sql); + if (message.bind_variables != null && Object.hasOwnProperty.call(message, "bind_variables")) + for (let keys = Object.keys(message.bind_variables), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.query.BindVariable.encode(message.bind_variables[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified StreamEvent message, length delimited. Does not implicitly {@link query.StreamEvent.verify|verify} messages. + * Encodes the specified BoundQuery message, length delimited. Does not implicitly {@link query.BoundQuery.verify|verify} messages. * @function encodeDelimited - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static - * @param {query.IStreamEvent} message StreamEvent message or plain object to encode + * @param {query.IBoundQuery} message BoundQuery message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamEvent.encodeDelimited = function encodeDelimited(message, writer) { + BoundQuery.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamEvent message from the specified reader or buffer. + * Decodes a BoundQuery message from the specified reader or buffer. * @function decode - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StreamEvent} StreamEvent + * @returns {query.BoundQuery} BoundQuery * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamEvent.decode = function decode(reader, length) { + BoundQuery.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.query.StreamEvent(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BoundQuery(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.statements && message.statements.length)) - message.statements = []; - message.statements.push($root.query.StreamEvent.Statement.decode(reader, reader.uint32())); + message.sql = reader.string(); break; } case 2: { - message.event_token = $root.query.EventToken.decode(reader, reader.uint32()); + if (message.bind_variables === $util.emptyObject) + message.bind_variables = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.query.BindVariable.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.bind_variables[key] = value; break; } default: @@ -77634,548 +77951,162 @@ export const query = $root.query = (() => { }; /** - * Decodes a StreamEvent message from the specified reader or buffer, length delimited. + * Decodes a BoundQuery message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamEvent} StreamEvent + * @returns {query.BoundQuery} BoundQuery * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamEvent.decodeDelimited = function decodeDelimited(reader) { + BoundQuery.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamEvent message. + * Verifies a BoundQuery message. * @function verify - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamEvent.verify = function verify(message) { + BoundQuery.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.statements != null && message.hasOwnProperty("statements")) { - if (!Array.isArray(message.statements)) - return "statements: array expected"; - for (let i = 0; i < message.statements.length; ++i) { - let error = $root.query.StreamEvent.Statement.verify(message.statements[i]); + if (message.sql != null && message.hasOwnProperty("sql")) + if (!$util.isString(message.sql)) + return "sql: string expected"; + if (message.bind_variables != null && message.hasOwnProperty("bind_variables")) { + if (!$util.isObject(message.bind_variables)) + return "bind_variables: object expected"; + let key = Object.keys(message.bind_variables); + for (let i = 0; i < key.length; ++i) { + let error = $root.query.BindVariable.verify(message.bind_variables[key[i]]); if (error) - return "statements." + error; + return "bind_variables." + error; } } - if (message.event_token != null && message.hasOwnProperty("event_token")) { - let error = $root.query.EventToken.verify(message.event_token); - if (error) - return "event_token." + error; - } return null; }; /** - * Creates a StreamEvent message from a plain object. Also converts values to their respective internal types. + * Creates a BoundQuery message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static * @param {Object.} object Plain object - * @returns {query.StreamEvent} StreamEvent + * @returns {query.BoundQuery} BoundQuery */ - StreamEvent.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamEvent) + BoundQuery.fromObject = function fromObject(object) { + if (object instanceof $root.query.BoundQuery) return object; - let message = new $root.query.StreamEvent(); - if (object.statements) { - if (!Array.isArray(object.statements)) - throw TypeError(".query.StreamEvent.statements: array expected"); - message.statements = []; - for (let i = 0; i < object.statements.length; ++i) { - if (typeof object.statements[i] !== "object") - throw TypeError(".query.StreamEvent.statements: object expected"); - message.statements[i] = $root.query.StreamEvent.Statement.fromObject(object.statements[i]); + let message = new $root.query.BoundQuery(); + if (object.sql != null) + message.sql = String(object.sql); + if (object.bind_variables) { + if (typeof object.bind_variables !== "object") + throw TypeError(".query.BoundQuery.bind_variables: object expected"); + message.bind_variables = {}; + for (let keys = Object.keys(object.bind_variables), i = 0; i < keys.length; ++i) { + if (typeof object.bind_variables[keys[i]] !== "object") + throw TypeError(".query.BoundQuery.bind_variables: object expected"); + message.bind_variables[keys[i]] = $root.query.BindVariable.fromObject(object.bind_variables[keys[i]]); } } - if (object.event_token != null) { - if (typeof object.event_token !== "object") - throw TypeError(".query.StreamEvent.event_token: object expected"); - message.event_token = $root.query.EventToken.fromObject(object.event_token); - } return message; }; /** - * Creates a plain object from a StreamEvent message. Also converts values to other types if specified. + * Creates a plain object from a BoundQuery message. Also converts values to other types if specified. * @function toObject - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static - * @param {query.StreamEvent} message StreamEvent + * @param {query.BoundQuery} message BoundQuery * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamEvent.toObject = function toObject(message, options) { + BoundQuery.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.statements = []; + if (options.objects || options.defaults) + object.bind_variables = {}; if (options.defaults) - object.event_token = null; - if (message.statements && message.statements.length) { - object.statements = []; - for (let j = 0; j < message.statements.length; ++j) - object.statements[j] = $root.query.StreamEvent.Statement.toObject(message.statements[j], options); + object.sql = ""; + if (message.sql != null && message.hasOwnProperty("sql")) + object.sql = message.sql; + let keys2; + if (message.bind_variables && (keys2 = Object.keys(message.bind_variables)).length) { + object.bind_variables = {}; + for (let j = 0; j < keys2.length; ++j) + object.bind_variables[keys2[j]] = $root.query.BindVariable.toObject(message.bind_variables[keys2[j]], options); } - if (message.event_token != null && message.hasOwnProperty("event_token")) - object.event_token = $root.query.EventToken.toObject(message.event_token, options); return object; }; /** - * Converts this StreamEvent to JSON. + * Converts this BoundQuery to JSON. * @function toJSON - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @instance * @returns {Object.} JSON object */ - StreamEvent.prototype.toJSON = function toJSON() { + BoundQuery.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamEvent + * Gets the default type url for BoundQuery * @function getTypeUrl - * @memberof query.StreamEvent + * @memberof query.BoundQuery * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BoundQuery.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StreamEvent"; + return typeUrlPrefix + "/query.BoundQuery"; }; - StreamEvent.Statement = (function() { + return BoundQuery; + })(); - /** - * Properties of a Statement. - * @memberof query.StreamEvent - * @interface IStatement - * @property {query.StreamEvent.Statement.Category|null} [category] Statement category - * @property {string|null} [table_name] Statement table_name - * @property {Array.|null} [primary_key_fields] Statement primary_key_fields - * @property {Array.|null} [primary_key_values] Statement primary_key_values - * @property {Uint8Array|null} [sql] Statement sql - */ + query.ExecuteOptions = (function() { - /** - * Constructs a new Statement. - * @memberof query.StreamEvent - * @classdesc Represents a Statement. - * @implements IStatement - * @constructor - * @param {query.StreamEvent.IStatement=} [properties] Properties to set - */ - function Statement(properties) { - this.primary_key_fields = []; - this.primary_key_values = []; - 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]]; - } - - /** - * Statement category. - * @member {query.StreamEvent.Statement.Category} category - * @memberof query.StreamEvent.Statement - * @instance - */ - Statement.prototype.category = 0; - - /** - * Statement table_name. - * @member {string} table_name - * @memberof query.StreamEvent.Statement - * @instance - */ - Statement.prototype.table_name = ""; - - /** - * Statement primary_key_fields. - * @member {Array.} primary_key_fields - * @memberof query.StreamEvent.Statement - * @instance - */ - Statement.prototype.primary_key_fields = $util.emptyArray; - - /** - * Statement primary_key_values. - * @member {Array.} primary_key_values - * @memberof query.StreamEvent.Statement - * @instance - */ - Statement.prototype.primary_key_values = $util.emptyArray; - - /** - * Statement sql. - * @member {Uint8Array} sql - * @memberof query.StreamEvent.Statement - * @instance - */ - Statement.prototype.sql = $util.newBuffer([]); - - /** - * Creates a new Statement instance using the specified properties. - * @function create - * @memberof query.StreamEvent.Statement - * @static - * @param {query.StreamEvent.IStatement=} [properties] Properties to set - * @returns {query.StreamEvent.Statement} Statement instance - */ - Statement.create = function create(properties) { - return new Statement(properties); - }; - - /** - * Encodes the specified Statement message. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. - * @function encode - * @memberof query.StreamEvent.Statement - * @static - * @param {query.StreamEvent.IStatement} message Statement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Statement.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.category != null && Object.hasOwnProperty.call(message, "category")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.category); - if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.table_name); - if (message.primary_key_fields != null && message.primary_key_fields.length) - for (let i = 0; i < message.primary_key_fields.length; ++i) - $root.query.Field.encode(message.primary_key_fields[i], writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.primary_key_values != null && message.primary_key_values.length) - for (let i = 0; i < message.primary_key_values.length; ++i) - $root.query.Row.encode(message.primary_key_values[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) - writer.uint32(/* id 5, wireType 2 =*/42).bytes(message.sql); - return writer; - }; - - /** - * Encodes the specified Statement message, length delimited. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. - * @function encodeDelimited - * @memberof query.StreamEvent.Statement - * @static - * @param {query.StreamEvent.IStatement} message Statement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Statement.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a Statement message from the specified reader or buffer. - * @function decode - * @memberof query.StreamEvent.Statement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {query.StreamEvent.Statement} Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Statement.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.query.StreamEvent.Statement(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.category = reader.int32(); - break; - } - case 2: { - message.table_name = reader.string(); - break; - } - case 3: { - if (!(message.primary_key_fields && message.primary_key_fields.length)) - message.primary_key_fields = []; - message.primary_key_fields.push($root.query.Field.decode(reader, reader.uint32())); - break; - } - case 4: { - if (!(message.primary_key_values && message.primary_key_values.length)) - message.primary_key_values = []; - message.primary_key_values.push($root.query.Row.decode(reader, reader.uint32())); - break; - } - case 5: { - message.sql = reader.bytes(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a Statement message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof query.StreamEvent.Statement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamEvent.Statement} Statement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Statement.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a Statement message. - * @function verify - * @memberof query.StreamEvent.Statement - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Statement.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.category != null && message.hasOwnProperty("category")) - switch (message.category) { - default: - return "category: enum value expected"; - case 0: - case 1: - case 2: - break; - } - if (message.table_name != null && message.hasOwnProperty("table_name")) - if (!$util.isString(message.table_name)) - return "table_name: string expected"; - if (message.primary_key_fields != null && message.hasOwnProperty("primary_key_fields")) { - if (!Array.isArray(message.primary_key_fields)) - return "primary_key_fields: array expected"; - for (let i = 0; i < message.primary_key_fields.length; ++i) { - let error = $root.query.Field.verify(message.primary_key_fields[i]); - if (error) - return "primary_key_fields." + error; - } - } - if (message.primary_key_values != null && message.hasOwnProperty("primary_key_values")) { - if (!Array.isArray(message.primary_key_values)) - return "primary_key_values: array expected"; - for (let i = 0; i < message.primary_key_values.length; ++i) { - let error = $root.query.Row.verify(message.primary_key_values[i]); - if (error) - return "primary_key_values." + error; - } - } - if (message.sql != null && message.hasOwnProperty("sql")) - if (!(message.sql && typeof message.sql.length === "number" || $util.isString(message.sql))) - return "sql: buffer expected"; - return null; - }; - - /** - * Creates a Statement message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof query.StreamEvent.Statement - * @static - * @param {Object.} object Plain object - * @returns {query.StreamEvent.Statement} Statement - */ - Statement.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamEvent.Statement) - return object; - let message = new $root.query.StreamEvent.Statement(); - switch (object.category) { - default: - if (typeof object.category === "number") { - message.category = object.category; - break; - } - break; - case "Error": - case 0: - message.category = 0; - break; - case "DML": - case 1: - message.category = 1; - break; - case "DDL": - case 2: - message.category = 2; - break; - } - if (object.table_name != null) - message.table_name = String(object.table_name); - if (object.primary_key_fields) { - if (!Array.isArray(object.primary_key_fields)) - throw TypeError(".query.StreamEvent.Statement.primary_key_fields: array expected"); - message.primary_key_fields = []; - for (let i = 0; i < object.primary_key_fields.length; ++i) { - if (typeof object.primary_key_fields[i] !== "object") - throw TypeError(".query.StreamEvent.Statement.primary_key_fields: object expected"); - message.primary_key_fields[i] = $root.query.Field.fromObject(object.primary_key_fields[i]); - } - } - if (object.primary_key_values) { - if (!Array.isArray(object.primary_key_values)) - throw TypeError(".query.StreamEvent.Statement.primary_key_values: array expected"); - message.primary_key_values = []; - for (let i = 0; i < object.primary_key_values.length; ++i) { - if (typeof object.primary_key_values[i] !== "object") - throw TypeError(".query.StreamEvent.Statement.primary_key_values: object expected"); - message.primary_key_values[i] = $root.query.Row.fromObject(object.primary_key_values[i]); - } - } - if (object.sql != null) - if (typeof object.sql === "string") - $util.base64.decode(object.sql, message.sql = $util.newBuffer($util.base64.length(object.sql)), 0); - else if (object.sql.length >= 0) - message.sql = object.sql; - return message; - }; - - /** - * Creates a plain object from a Statement message. Also converts values to other types if specified. - * @function toObject - * @memberof query.StreamEvent.Statement - * @static - * @param {query.StreamEvent.Statement} message Statement - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Statement.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) { - object.primary_key_fields = []; - object.primary_key_values = []; - } - if (options.defaults) { - object.category = options.enums === String ? "Error" : 0; - object.table_name = ""; - if (options.bytes === String) - object.sql = ""; - else { - object.sql = []; - if (options.bytes !== Array) - object.sql = $util.newBuffer(object.sql); - } - } - if (message.category != null && message.hasOwnProperty("category")) - object.category = options.enums === String ? $root.query.StreamEvent.Statement.Category[message.category] === undefined ? message.category : $root.query.StreamEvent.Statement.Category[message.category] : message.category; - if (message.table_name != null && message.hasOwnProperty("table_name")) - object.table_name = message.table_name; - if (message.primary_key_fields && message.primary_key_fields.length) { - object.primary_key_fields = []; - for (let j = 0; j < message.primary_key_fields.length; ++j) - object.primary_key_fields[j] = $root.query.Field.toObject(message.primary_key_fields[j], options); - } - if (message.primary_key_values && message.primary_key_values.length) { - object.primary_key_values = []; - for (let j = 0; j < message.primary_key_values.length; ++j) - object.primary_key_values[j] = $root.query.Row.toObject(message.primary_key_values[j], options); - } - if (message.sql != null && message.hasOwnProperty("sql")) - object.sql = options.bytes === String ? $util.base64.encode(message.sql, 0, message.sql.length) : options.bytes === Array ? Array.prototype.slice.call(message.sql) : message.sql; - return object; - }; - - /** - * Converts this Statement to JSON. - * @function toJSON - * @memberof query.StreamEvent.Statement - * @instance - * @returns {Object.} JSON object - */ - Statement.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for Statement - * @function getTypeUrl - * @memberof query.StreamEvent.Statement - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Statement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/query.StreamEvent.Statement"; - }; - - /** - * Category enum. - * @name query.StreamEvent.Statement.Category - * @enum {number} - * @property {number} Error=0 Error value - * @property {number} DML=1 DML value - * @property {number} DDL=2 DDL value - */ - Statement.Category = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "Error"] = 0; - values[valuesById[1] = "DML"] = 1; - values[valuesById[2] = "DDL"] = 2; - return values; - })(); - - return Statement; - })(); - - return StreamEvent; - })(); - - query.ExecuteRequest = (function() { - - /** - * Properties of an ExecuteRequest. - * @memberof query - * @interface IExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] ExecuteRequest target - * @property {query.IBoundQuery|null} [query] ExecuteRequest query - * @property {number|Long|null} [transaction_id] ExecuteRequest transaction_id - * @property {query.IExecuteOptions|null} [options] ExecuteRequest options - * @property {number|Long|null} [reserved_id] ExecuteRequest reserved_id - */ + /** + * Properties of an ExecuteOptions. + * @memberof query + * @interface IExecuteOptions + * @property {query.ExecuteOptions.IncludedFields|null} [included_fields] ExecuteOptions included_fields + * @property {boolean|null} [client_found_rows] ExecuteOptions client_found_rows + * @property {query.ExecuteOptions.Workload|null} [workload] ExecuteOptions workload + * @property {number|Long|null} [sql_select_limit] ExecuteOptions sql_select_limit + * @property {query.ExecuteOptions.TransactionIsolation|null} [transaction_isolation] ExecuteOptions transaction_isolation + * @property {boolean|null} [skip_query_plan_cache] ExecuteOptions skip_query_plan_cache + * @property {query.ExecuteOptions.PlannerVersion|null} [planner_version] ExecuteOptions planner_version + * @property {boolean|null} [has_created_temp_tables] ExecuteOptions has_created_temp_tables + * @property {query.ExecuteOptions.Consolidator|null} [consolidator] ExecuteOptions consolidator + * @property {Array.|null} [transaction_access_mode] ExecuteOptions transaction_access_mode + * @property {string|null} [WorkloadName] ExecuteOptions WorkloadName + * @property {string|null} [priority] ExecuteOptions priority + */ /** - * Constructs a new ExecuteRequest. + * Constructs a new ExecuteOptions. * @memberof query - * @classdesc Represents an ExecuteRequest. - * @implements IExecuteRequest + * @classdesc Represents an ExecuteOptions. + * @implements IExecuteOptions * @constructor - * @param {query.IExecuteRequest=} [properties] Properties to set + * @param {query.IExecuteOptions=} [properties] Properties to set */ - function ExecuteRequest(properties) { + function ExecuteOptions(properties) { + this.transaction_access_mode = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -78183,159 +78114,240 @@ export const query = $root.query = (() => { } /** - * ExecuteRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ExecuteRequest + * ExecuteOptions included_fields. + * @member {query.ExecuteOptions.IncludedFields} included_fields + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.effective_caller_id = null; + ExecuteOptions.prototype.included_fields = 0; /** - * ExecuteRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ExecuteRequest + * ExecuteOptions client_found_rows. + * @member {boolean} client_found_rows + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.immediate_caller_id = null; + ExecuteOptions.prototype.client_found_rows = false; /** - * ExecuteRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.ExecuteRequest + * ExecuteOptions workload. + * @member {query.ExecuteOptions.Workload} workload + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.target = null; + ExecuteOptions.prototype.workload = 0; /** - * ExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.ExecuteRequest + * ExecuteOptions sql_select_limit. + * @member {number|Long} sql_select_limit + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.query = null; + ExecuteOptions.prototype.sql_select_limit = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ExecuteRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.ExecuteRequest + * ExecuteOptions transaction_isolation. + * @member {query.ExecuteOptions.TransactionIsolation} transaction_isolation + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ExecuteOptions.prototype.transaction_isolation = 0; /** - * ExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.ExecuteRequest + * ExecuteOptions skip_query_plan_cache. + * @member {boolean} skip_query_plan_cache + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.options = null; + ExecuteOptions.prototype.skip_query_plan_cache = false; /** - * ExecuteRequest reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ExecuteRequest + * ExecuteOptions planner_version. + * @member {query.ExecuteOptions.PlannerVersion} planner_version + * @memberof query.ExecuteOptions * @instance */ - ExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ExecuteOptions.prototype.planner_version = 0; /** - * Creates a new ExecuteRequest instance using the specified properties. + * ExecuteOptions has_created_temp_tables. + * @member {boolean} has_created_temp_tables + * @memberof query.ExecuteOptions + * @instance + */ + ExecuteOptions.prototype.has_created_temp_tables = false; + + /** + * ExecuteOptions consolidator. + * @member {query.ExecuteOptions.Consolidator} consolidator + * @memberof query.ExecuteOptions + * @instance + */ + ExecuteOptions.prototype.consolidator = 0; + + /** + * ExecuteOptions transaction_access_mode. + * @member {Array.} transaction_access_mode + * @memberof query.ExecuteOptions + * @instance + */ + ExecuteOptions.prototype.transaction_access_mode = $util.emptyArray; + + /** + * ExecuteOptions WorkloadName. + * @member {string} WorkloadName + * @memberof query.ExecuteOptions + * @instance + */ + ExecuteOptions.prototype.WorkloadName = ""; + + /** + * ExecuteOptions priority. + * @member {string} priority + * @memberof query.ExecuteOptions + * @instance + */ + ExecuteOptions.prototype.priority = ""; + + /** + * Creates a new ExecuteOptions instance using the specified properties. * @function create - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static - * @param {query.IExecuteRequest=} [properties] Properties to set - * @returns {query.ExecuteRequest} ExecuteRequest instance + * @param {query.IExecuteOptions=} [properties] Properties to set + * @returns {query.ExecuteOptions} ExecuteOptions instance */ - ExecuteRequest.create = function create(properties) { - return new ExecuteRequest(properties); + ExecuteOptions.create = function create(properties) { + return new ExecuteOptions(properties); }; /** - * Encodes the specified ExecuteRequest message. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. + * Encodes the specified ExecuteOptions message. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. * @function encode - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static - * @param {query.IExecuteRequest} message ExecuteRequest message or plain object to encode + * @param {query.IExecuteOptions} message ExecuteOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteRequest.encode = function encode(message, writer) { + ExecuteOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.transaction_id); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); - return writer; + if (message.included_fields != null && Object.hasOwnProperty.call(message, "included_fields")) + writer.uint32(/* id 4, wireType 0 =*/32).int32(message.included_fields); + if (message.client_found_rows != null && Object.hasOwnProperty.call(message, "client_found_rows")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.client_found_rows); + if (message.workload != null && Object.hasOwnProperty.call(message, "workload")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.workload); + if (message.sql_select_limit != null && Object.hasOwnProperty.call(message, "sql_select_limit")) + writer.uint32(/* id 8, wireType 0 =*/64).int64(message.sql_select_limit); + if (message.transaction_isolation != null && Object.hasOwnProperty.call(message, "transaction_isolation")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.transaction_isolation); + if (message.skip_query_plan_cache != null && Object.hasOwnProperty.call(message, "skip_query_plan_cache")) + writer.uint32(/* id 10, wireType 0 =*/80).bool(message.skip_query_plan_cache); + if (message.planner_version != null && Object.hasOwnProperty.call(message, "planner_version")) + writer.uint32(/* id 11, wireType 0 =*/88).int32(message.planner_version); + if (message.has_created_temp_tables != null && Object.hasOwnProperty.call(message, "has_created_temp_tables")) + writer.uint32(/* id 12, wireType 0 =*/96).bool(message.has_created_temp_tables); + if (message.consolidator != null && Object.hasOwnProperty.call(message, "consolidator")) + writer.uint32(/* id 13, wireType 0 =*/104).int32(message.consolidator); + if (message.transaction_access_mode != null && message.transaction_access_mode.length) { + writer.uint32(/* id 14, wireType 2 =*/114).fork(); + for (let i = 0; i < message.transaction_access_mode.length; ++i) + writer.int32(message.transaction_access_mode[i]); + writer.ldelim(); + } + if (message.WorkloadName != null && Object.hasOwnProperty.call(message, "WorkloadName")) + writer.uint32(/* id 15, wireType 2 =*/122).string(message.WorkloadName); + if (message.priority != null && Object.hasOwnProperty.call(message, "priority")) + writer.uint32(/* id 16, wireType 2 =*/130).string(message.priority); + return writer; }; /** - * Encodes the specified ExecuteRequest message, length delimited. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. + * Encodes the specified ExecuteOptions message, length delimited. Does not implicitly {@link query.ExecuteOptions.verify|verify} messages. * @function encodeDelimited - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static - * @param {query.IExecuteRequest} message ExecuteRequest message or plain object to encode + * @param {query.IExecuteOptions} message ExecuteOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteRequest message from the specified reader or buffer. + * Decodes an ExecuteOptions message from the specified reader or buffer. * @function decode - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ExecuteRequest} ExecuteRequest + * @returns {query.ExecuteOptions} ExecuteOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteRequest.decode = function decode(reader, length) { + ExecuteOptions.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.query.ExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ExecuteOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + case 4: { + message.included_fields = reader.int32(); break; } - case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + case 5: { + message.client_found_rows = reader.bool(); break; } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); + case 6: { + message.workload = reader.int32(); break; } - case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + case 8: { + message.sql_select_limit = reader.int64(); break; } - case 5: { - message.transaction_id = reader.int64(); + case 9: { + message.transaction_isolation = reader.int32(); break; } - case 6: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + case 10: { + message.skip_query_plan_cache = reader.bool(); break; } - case 7: { - message.reserved_id = reader.int64(); + case 11: { + message.planner_version = reader.int32(); + break; + } + case 12: { + message.has_created_temp_tables = reader.bool(); + break; + } + case 13: { + message.consolidator = reader.int32(); + break; + } + case 14: { + if (!(message.transaction_access_mode && message.transaction_access_mode.length)) + message.transaction_access_mode = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.transaction_access_mode.push(reader.int32()); + } else + message.transaction_access_mode.push(reader.int32()); + break; + } + case 15: { + message.WorkloadName = reader.string(); + break; + } + case 16: { + message.priority = reader.string(); break; } default: @@ -78347,675 +78359,1175 @@ export const query = $root.query = (() => { }; /** - * Decodes an ExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ExecuteRequest} ExecuteRequest + * @returns {query.ExecuteOptions} ExecuteOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + ExecuteOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteRequest message. + * Verifies an ExecuteOptions message. * @function verify - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteRequest.verify = function verify(message) { + ExecuteOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; - } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; + if (message.included_fields != null && message.hasOwnProperty("included_fields")) + switch (message.included_fields) { + default: + return "included_fields: enum value expected"; + case 0: + case 1: + case 2: + break; + } + if (message.client_found_rows != null && message.hasOwnProperty("client_found_rows")) + if (typeof message.client_found_rows !== "boolean") + return "client_found_rows: boolean expected"; + if (message.workload != null && message.hasOwnProperty("workload")) + switch (message.workload) { + default: + return "workload: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.sql_select_limit != null && message.hasOwnProperty("sql_select_limit")) + if (!$util.isInteger(message.sql_select_limit) && !(message.sql_select_limit && $util.isInteger(message.sql_select_limit.low) && $util.isInteger(message.sql_select_limit.high))) + return "sql_select_limit: integer|Long expected"; + if (message.transaction_isolation != null && message.hasOwnProperty("transaction_isolation")) + switch (message.transaction_isolation) { + default: + return "transaction_isolation: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + if (message.skip_query_plan_cache != null && message.hasOwnProperty("skip_query_plan_cache")) + if (typeof message.skip_query_plan_cache !== "boolean") + return "skip_query_plan_cache: boolean expected"; + if (message.planner_version != null && message.hasOwnProperty("planner_version")) + switch (message.planner_version) { + default: + return "planner_version: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + break; + } + if (message.has_created_temp_tables != null && message.hasOwnProperty("has_created_temp_tables")) + if (typeof message.has_created_temp_tables !== "boolean") + return "has_created_temp_tables: boolean expected"; + if (message.consolidator != null && message.hasOwnProperty("consolidator")) + switch (message.consolidator) { + default: + return "consolidator: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.transaction_access_mode != null && message.hasOwnProperty("transaction_access_mode")) { + if (!Array.isArray(message.transaction_access_mode)) + return "transaction_access_mode: array expected"; + for (let i = 0; i < message.transaction_access_mode.length; ++i) + switch (message.transaction_access_mode[i]) { + default: + return "transaction_access_mode: enum value[] expected"; + case 0: + case 1: + case 2: + break; + } } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.WorkloadName != null && message.hasOwnProperty("WorkloadName")) + if (!$util.isString(message.WorkloadName)) + return "WorkloadName: string expected"; + if (message.priority != null && message.hasOwnProperty("priority")) + if (!$util.isString(message.priority)) + return "priority: string expected"; return null; }; /** - * Creates an ExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static * @param {Object.} object Plain object - * @returns {query.ExecuteRequest} ExecuteRequest + * @returns {query.ExecuteOptions} ExecuteOptions */ - ExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ExecuteRequest) + ExecuteOptions.fromObject = function fromObject(object) { + if (object instanceof $root.query.ExecuteOptions) return object; - let message = new $root.query.ExecuteRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ExecuteRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + let message = new $root.query.ExecuteOptions(); + switch (object.included_fields) { + default: + if (typeof object.included_fields === "number") { + message.included_fields = object.included_fields; + break; + } + break; + case "TYPE_AND_NAME": + case 0: + message.included_fields = 0; + break; + case "TYPE_ONLY": + case 1: + message.included_fields = 1; + break; + case "ALL": + case 2: + message.included_fields = 2; + break; } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ExecuteRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + if (object.client_found_rows != null) + message.client_found_rows = Boolean(object.client_found_rows); + switch (object.workload) { + default: + if (typeof object.workload === "number") { + message.workload = object.workload; + break; + } + break; + case "UNSPECIFIED": + case 0: + message.workload = 0; + break; + case "OLTP": + case 1: + message.workload = 1; + break; + case "OLAP": + case 2: + message.workload = 2; + break; + case "DBA": + case 3: + message.workload = 3; + break; } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.ExecuteRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + if (object.sql_select_limit != null) + if ($util.Long) + (message.sql_select_limit = $util.Long.fromValue(object.sql_select_limit)).unsigned = false; + else if (typeof object.sql_select_limit === "string") + message.sql_select_limit = parseInt(object.sql_select_limit, 10); + else if (typeof object.sql_select_limit === "number") + message.sql_select_limit = object.sql_select_limit; + else if (typeof object.sql_select_limit === "object") + message.sql_select_limit = new $util.LongBits(object.sql_select_limit.low >>> 0, object.sql_select_limit.high >>> 0).toNumber(); + switch (object.transaction_isolation) { + default: + if (typeof object.transaction_isolation === "number") { + message.transaction_isolation = object.transaction_isolation; + break; + } + break; + case "DEFAULT": + case 0: + message.transaction_isolation = 0; + break; + case "REPEATABLE_READ": + case 1: + message.transaction_isolation = 1; + break; + case "READ_COMMITTED": + case 2: + message.transaction_isolation = 2; + break; + case "READ_UNCOMMITTED": + case 3: + message.transaction_isolation = 3; + break; + case "SERIALIZABLE": + case 4: + message.transaction_isolation = 4; + break; + case "CONSISTENT_SNAPSHOT_READ_ONLY": + case 5: + message.transaction_isolation = 5; + break; + case "AUTOCOMMIT": + case 6: + message.transaction_isolation = 6; + break; } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.ExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); + if (object.skip_query_plan_cache != null) + message.skip_query_plan_cache = Boolean(object.skip_query_plan_cache); + switch (object.planner_version) { + default: + if (typeof object.planner_version === "number") { + message.planner_version = object.planner_version; + break; + } + break; + case "DEFAULT_PLANNER": + case 0: + message.planner_version = 0; + break; + case "V3": + case 1: + message.planner_version = 1; + break; + case "Gen4": + case 2: + message.planner_version = 2; + break; + case "Gen4Greedy": + case 3: + message.planner_version = 3; + break; + case "Gen4Left2Right": + case 4: + message.planner_version = 4; + break; + case "Gen4WithFallback": + case 5: + message.planner_version = 5; + break; + case "Gen4CompareV3": + case 6: + message.planner_version = 6; + break; + case "V3Insert": + case 7: + message.planner_version = 7; + break; } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.ExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); + if (object.has_created_temp_tables != null) + message.has_created_temp_tables = Boolean(object.has_created_temp_tables); + switch (object.consolidator) { + default: + if (typeof object.consolidator === "number") { + message.consolidator = object.consolidator; + break; + } + break; + case "CONSOLIDATOR_UNSPECIFIED": + case 0: + message.consolidator = 0; + break; + case "CONSOLIDATOR_DISABLED": + case 1: + message.consolidator = 1; + break; + case "CONSOLIDATOR_ENABLED": + case 2: + message.consolidator = 2; + break; + case "CONSOLIDATOR_ENABLED_REPLICAS": + case 3: + message.consolidator = 3; + break; } - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.transaction_access_mode) { + if (!Array.isArray(object.transaction_access_mode)) + throw TypeError(".query.ExecuteOptions.transaction_access_mode: array expected"); + message.transaction_access_mode = []; + for (let i = 0; i < object.transaction_access_mode.length; ++i) + switch (object.transaction_access_mode[i]) { + default: + if (typeof object.transaction_access_mode[i] === "number") { + message.transaction_access_mode[i] = object.transaction_access_mode[i]; + break; + } + case "CONSISTENT_SNAPSHOT": + case 0: + message.transaction_access_mode[i] = 0; + break; + case "READ_WRITE": + case 1: + message.transaction_access_mode[i] = 1; + break; + case "READ_ONLY": + case 2: + message.transaction_access_mode[i] = 2; + break; + } + } + if (object.WorkloadName != null) + message.WorkloadName = String(object.WorkloadName); + if (object.priority != null) + message.priority = String(object.priority); return message; }; /** - * Creates a plain object from an ExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteOptions message. Also converts values to other types if specified. * @function toObject - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static - * @param {query.ExecuteRequest} message ExecuteRequest + * @param {query.ExecuteOptions} message ExecuteOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteRequest.toObject = function toObject(message, options) { + ExecuteOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.transaction_access_mode = []; if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.query = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - object.options = null; + object.included_fields = options.enums === String ? "TYPE_AND_NAME" : 0; + object.client_found_rows = false; + object.workload = options.enums === String ? "UNSPECIFIED" : 0; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.sql_select_limit = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.reserved_id = options.longs === String ? "0" : 0; + object.sql_select_limit = options.longs === String ? "0" : 0; + object.transaction_isolation = options.enums === String ? "DEFAULT" : 0; + object.skip_query_plan_cache = false; + object.planner_version = options.enums === String ? "DEFAULT_PLANNER" : 0; + object.has_created_temp_tables = false; + object.consolidator = options.enums === String ? "CONSOLIDATOR_UNSPECIFIED" : 0; + object.WorkloadName = ""; + object.priority = ""; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + if (message.included_fields != null && message.hasOwnProperty("included_fields")) + object.included_fields = options.enums === String ? $root.query.ExecuteOptions.IncludedFields[message.included_fields] === undefined ? message.included_fields : $root.query.ExecuteOptions.IncludedFields[message.included_fields] : message.included_fields; + if (message.client_found_rows != null && message.hasOwnProperty("client_found_rows")) + object.client_found_rows = message.client_found_rows; + if (message.workload != null && message.hasOwnProperty("workload")) + object.workload = options.enums === String ? $root.query.ExecuteOptions.Workload[message.workload] === undefined ? message.workload : $root.query.ExecuteOptions.Workload[message.workload] : message.workload; + if (message.sql_select_limit != null && message.hasOwnProperty("sql_select_limit")) + if (typeof message.sql_select_limit === "number") + object.sql_select_limit = options.longs === String ? String(message.sql_select_limit) : message.sql_select_limit; else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + object.sql_select_limit = options.longs === String ? $util.Long.prototype.toString.call(message.sql_select_limit) : options.longs === Number ? new $util.LongBits(message.sql_select_limit.low >>> 0, message.sql_select_limit.high >>> 0).toNumber() : message.sql_select_limit; + if (message.transaction_isolation != null && message.hasOwnProperty("transaction_isolation")) + object.transaction_isolation = options.enums === String ? $root.query.ExecuteOptions.TransactionIsolation[message.transaction_isolation] === undefined ? message.transaction_isolation : $root.query.ExecuteOptions.TransactionIsolation[message.transaction_isolation] : message.transaction_isolation; + if (message.skip_query_plan_cache != null && message.hasOwnProperty("skip_query_plan_cache")) + object.skip_query_plan_cache = message.skip_query_plan_cache; + if (message.planner_version != null && message.hasOwnProperty("planner_version")) + object.planner_version = options.enums === String ? $root.query.ExecuteOptions.PlannerVersion[message.planner_version] === undefined ? message.planner_version : $root.query.ExecuteOptions.PlannerVersion[message.planner_version] : message.planner_version; + if (message.has_created_temp_tables != null && message.hasOwnProperty("has_created_temp_tables")) + object.has_created_temp_tables = message.has_created_temp_tables; + if (message.consolidator != null && message.hasOwnProperty("consolidator")) + object.consolidator = options.enums === String ? $root.query.ExecuteOptions.Consolidator[message.consolidator] === undefined ? message.consolidator : $root.query.ExecuteOptions.Consolidator[message.consolidator] : message.consolidator; + if (message.transaction_access_mode && message.transaction_access_mode.length) { + object.transaction_access_mode = []; + for (let j = 0; j < message.transaction_access_mode.length; ++j) + object.transaction_access_mode[j] = options.enums === String ? $root.query.ExecuteOptions.TransactionAccessMode[message.transaction_access_mode[j]] === undefined ? message.transaction_access_mode[j] : $root.query.ExecuteOptions.TransactionAccessMode[message.transaction_access_mode[j]] : message.transaction_access_mode[j]; + } + if (message.WorkloadName != null && message.hasOwnProperty("WorkloadName")) + object.WorkloadName = message.WorkloadName; + if (message.priority != null && message.hasOwnProperty("priority")) + object.priority = message.priority; return object; }; /** - * Converts this ExecuteRequest to JSON. + * Converts this ExecuteOptions to JSON. * @function toJSON - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @instance * @returns {Object.} JSON object */ - ExecuteRequest.prototype.toJSON = function toJSON() { + ExecuteOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteRequest + * Gets the default type url for ExecuteOptions * @function getTypeUrl - * @memberof query.ExecuteRequest + * @memberof query.ExecuteOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ExecuteRequest"; + return typeUrlPrefix + "/query.ExecuteOptions"; }; - return ExecuteRequest; - })(); - - query.ExecuteResponse = (function() { - /** - * Properties of an ExecuteResponse. - * @memberof query - * @interface IExecuteResponse - * @property {query.IQueryResult|null} [result] ExecuteResponse result + * IncludedFields enum. + * @name query.ExecuteOptions.IncludedFields + * @enum {number} + * @property {number} TYPE_AND_NAME=0 TYPE_AND_NAME value + * @property {number} TYPE_ONLY=1 TYPE_ONLY value + * @property {number} ALL=2 ALL value */ + ExecuteOptions.IncludedFields = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "TYPE_AND_NAME"] = 0; + values[valuesById[1] = "TYPE_ONLY"] = 1; + values[valuesById[2] = "ALL"] = 2; + return values; + })(); /** - * Constructs a new ExecuteResponse. - * @memberof query - * @classdesc Represents an ExecuteResponse. - * @implements IExecuteResponse - * @constructor - * @param {query.IExecuteResponse=} [properties] Properties to set + * Workload enum. + * @name query.ExecuteOptions.Workload + * @enum {number} + * @property {number} UNSPECIFIED=0 UNSPECIFIED value + * @property {number} OLTP=1 OLTP value + * @property {number} OLAP=2 OLAP value + * @property {number} DBA=3 DBA value */ - function ExecuteResponse(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]]; - } + ExecuteOptions.Workload = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "UNSPECIFIED"] = 0; + values[valuesById[1] = "OLTP"] = 1; + values[valuesById[2] = "OLAP"] = 2; + values[valuesById[3] = "DBA"] = 3; + return values; + })(); /** - * ExecuteResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.ExecuteResponse - * @instance + * TransactionIsolation enum. + * @name query.ExecuteOptions.TransactionIsolation + * @enum {number} + * @property {number} DEFAULT=0 DEFAULT value + * @property {number} REPEATABLE_READ=1 REPEATABLE_READ value + * @property {number} READ_COMMITTED=2 READ_COMMITTED value + * @property {number} READ_UNCOMMITTED=3 READ_UNCOMMITTED value + * @property {number} SERIALIZABLE=4 SERIALIZABLE value + * @property {number} CONSISTENT_SNAPSHOT_READ_ONLY=5 CONSISTENT_SNAPSHOT_READ_ONLY value + * @property {number} AUTOCOMMIT=6 AUTOCOMMIT value */ - ExecuteResponse.prototype.result = null; + ExecuteOptions.TransactionIsolation = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "DEFAULT"] = 0; + values[valuesById[1] = "REPEATABLE_READ"] = 1; + values[valuesById[2] = "READ_COMMITTED"] = 2; + values[valuesById[3] = "READ_UNCOMMITTED"] = 3; + values[valuesById[4] = "SERIALIZABLE"] = 4; + values[valuesById[5] = "CONSISTENT_SNAPSHOT_READ_ONLY"] = 5; + values[valuesById[6] = "AUTOCOMMIT"] = 6; + return values; + })(); /** - * Creates a new ExecuteResponse instance using the specified properties. - * @function create - * @memberof query.ExecuteResponse - * @static - * @param {query.IExecuteResponse=} [properties] Properties to set - * @returns {query.ExecuteResponse} ExecuteResponse instance + * PlannerVersion enum. + * @name query.ExecuteOptions.PlannerVersion + * @enum {number} + * @property {number} DEFAULT_PLANNER=0 DEFAULT_PLANNER value + * @property {number} V3=1 V3 value + * @property {number} Gen4=2 Gen4 value + * @property {number} Gen4Greedy=3 Gen4Greedy value + * @property {number} Gen4Left2Right=4 Gen4Left2Right value + * @property {number} Gen4WithFallback=5 Gen4WithFallback value + * @property {number} Gen4CompareV3=6 Gen4CompareV3 value + * @property {number} V3Insert=7 V3Insert value */ - ExecuteResponse.create = function create(properties) { - return new ExecuteResponse(properties); - }; + ExecuteOptions.PlannerVersion = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "DEFAULT_PLANNER"] = 0; + values[valuesById[1] = "V3"] = 1; + values[valuesById[2] = "Gen4"] = 2; + values[valuesById[3] = "Gen4Greedy"] = 3; + values[valuesById[4] = "Gen4Left2Right"] = 4; + values[valuesById[5] = "Gen4WithFallback"] = 5; + values[valuesById[6] = "Gen4CompareV3"] = 6; + values[valuesById[7] = "V3Insert"] = 7; + return values; + })(); /** - * Encodes the specified ExecuteResponse message. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. - * @function encode - * @memberof query.ExecuteResponse - * @static - * @param {query.IExecuteResponse} message ExecuteResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * Consolidator enum. + * @name query.ExecuteOptions.Consolidator + * @enum {number} + * @property {number} CONSOLIDATOR_UNSPECIFIED=0 CONSOLIDATOR_UNSPECIFIED value + * @property {number} CONSOLIDATOR_DISABLED=1 CONSOLIDATOR_DISABLED value + * @property {number} CONSOLIDATOR_ENABLED=2 CONSOLIDATOR_ENABLED value + * @property {number} CONSOLIDATOR_ENABLED_REPLICAS=3 CONSOLIDATOR_ENABLED_REPLICAS value */ - ExecuteResponse.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - return writer; - }; + ExecuteOptions.Consolidator = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "CONSOLIDATOR_UNSPECIFIED"] = 0; + values[valuesById[1] = "CONSOLIDATOR_DISABLED"] = 1; + values[valuesById[2] = "CONSOLIDATOR_ENABLED"] = 2; + values[valuesById[3] = "CONSOLIDATOR_ENABLED_REPLICAS"] = 3; + return values; + })(); /** - * Encodes the specified ExecuteResponse message, length delimited. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. - * @function encodeDelimited - * @memberof query.ExecuteResponse - * @static - * @param {query.IExecuteResponse} message ExecuteResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * TransactionAccessMode enum. + * @name query.ExecuteOptions.TransactionAccessMode + * @enum {number} + * @property {number} CONSISTENT_SNAPSHOT=0 CONSISTENT_SNAPSHOT value + * @property {number} READ_WRITE=1 READ_WRITE value + * @property {number} READ_ONLY=2 READ_ONLY value */ - ExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + ExecuteOptions.TransactionAccessMode = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "CONSISTENT_SNAPSHOT"] = 0; + values[valuesById[1] = "READ_WRITE"] = 1; + values[valuesById[2] = "READ_ONLY"] = 2; + return values; + })(); + + return ExecuteOptions; + })(); + + query.Field = (function() { /** - * Decodes an ExecuteResponse message from the specified reader or buffer. - * @function decode - * @memberof query.ExecuteResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {query.ExecuteResponse} ExecuteResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * Properties of a Field. + * @memberof query + * @interface IField + * @property {string|null} [name] Field name + * @property {query.Type|null} [type] Field type + * @property {string|null} [table] Field table + * @property {string|null} [org_table] Field org_table + * @property {string|null} [database] Field database + * @property {string|null} [org_name] Field org_name + * @property {number|null} [column_length] Field column_length + * @property {number|null} [charset] Field charset + * @property {number|null} [decimals] Field decimals + * @property {number|null} [flags] Field flags + * @property {string|null} [column_type] Field column_type */ - ExecuteResponse.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.query.ExecuteResponse(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; /** - * Decodes an ExecuteResponse message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof query.ExecuteResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ExecuteResponse} ExecuteResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * Constructs a new Field. + * @memberof query + * @classdesc Represents a Field. + * @implements IField + * @constructor + * @param {query.IField=} [properties] Properties to set */ - ExecuteResponse.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + function Field(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]]; + } /** - * Verifies an ExecuteResponse message. - * @function verify - * @memberof query.ExecuteResponse - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not + * Field name. + * @member {string} name + * @memberof query.Field + * @instance */ - ExecuteResponse.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; - } - return null; - }; + Field.prototype.name = ""; /** - * Creates an ExecuteResponse message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof query.ExecuteResponse - * @static - * @param {Object.} object Plain object - * @returns {query.ExecuteResponse} ExecuteResponse + * Field type. + * @member {query.Type} type + * @memberof query.Field + * @instance */ - ExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ExecuteResponse) - return object; - let message = new $root.query.ExecuteResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.ExecuteResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); - } - return message; - }; + Field.prototype.type = 0; /** - * Creates a plain object from an ExecuteResponse message. Also converts values to other types if specified. - * @function toObject - * @memberof query.ExecuteResponse - * @static - * @param {query.ExecuteResponse} message ExecuteResponse - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object + * Field table. + * @member {string} table + * @memberof query.Field + * @instance */ - ExecuteResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); - return object; - }; + Field.prototype.table = ""; /** - * Converts this ExecuteResponse to JSON. - * @function toJSON - * @memberof query.ExecuteResponse + * Field org_table. + * @member {string} org_table + * @memberof query.Field * @instance - * @returns {Object.} JSON object */ - ExecuteResponse.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + Field.prototype.org_table = ""; /** - * Gets the default type url for ExecuteResponse - * @function getTypeUrl - * @memberof query.ExecuteResponse - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url + * Field database. + * @member {string} database + * @memberof query.Field + * @instance */ - ExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/query.ExecuteResponse"; - }; + Field.prototype.database = ""; - return ExecuteResponse; - })(); + /** + * Field org_name. + * @member {string} org_name + * @memberof query.Field + * @instance + */ + Field.prototype.org_name = ""; - query.ResultWithError = (function() { + /** + * Field column_length. + * @member {number} column_length + * @memberof query.Field + * @instance + */ + Field.prototype.column_length = 0; /** - * Properties of a ResultWithError. - * @memberof query - * @interface IResultWithError - * @property {vtrpc.IRPCError|null} [error] ResultWithError error - * @property {query.IQueryResult|null} [result] ResultWithError result + * Field charset. + * @member {number} charset + * @memberof query.Field + * @instance */ + Field.prototype.charset = 0; /** - * Constructs a new ResultWithError. - * @memberof query - * @classdesc Represents a ResultWithError. - * @implements IResultWithError - * @constructor - * @param {query.IResultWithError=} [properties] Properties to set + * Field decimals. + * @member {number} decimals + * @memberof query.Field + * @instance */ - function ResultWithError(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]]; - } + Field.prototype.decimals = 0; /** - * ResultWithError error. - * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.ResultWithError + * Field flags. + * @member {number} flags + * @memberof query.Field * @instance */ - ResultWithError.prototype.error = null; + Field.prototype.flags = 0; /** - * ResultWithError result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.ResultWithError + * Field column_type. + * @member {string} column_type + * @memberof query.Field * @instance */ - ResultWithError.prototype.result = null; + Field.prototype.column_type = ""; /** - * Creates a new ResultWithError instance using the specified properties. + * Creates a new Field instance using the specified properties. * @function create - * @memberof query.ResultWithError + * @memberof query.Field * @static - * @param {query.IResultWithError=} [properties] Properties to set - * @returns {query.ResultWithError} ResultWithError instance + * @param {query.IField=} [properties] Properties to set + * @returns {query.Field} Field instance */ - ResultWithError.create = function create(properties) { - return new ResultWithError(properties); + Field.create = function create(properties) { + return new Field(properties); }; /** - * Encodes the specified ResultWithError message. Does not implicitly {@link query.ResultWithError.verify|verify} messages. + * Encodes the specified Field message. Does not implicitly {@link query.Field.verify|verify} messages. * @function encode - * @memberof query.ResultWithError + * @memberof query.Field * @static - * @param {query.IResultWithError} message ResultWithError message or plain object to encode + * @param {query.IField} message Field message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResultWithError.encode = function encode(message, writer) { + Field.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.type); + if (message.table != null && Object.hasOwnProperty.call(message, "table")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.table); + if (message.org_table != null && Object.hasOwnProperty.call(message, "org_table")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.org_table); + if (message.database != null && Object.hasOwnProperty.call(message, "database")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.database); + if (message.org_name != null && Object.hasOwnProperty.call(message, "org_name")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.org_name); + if (message.column_length != null && Object.hasOwnProperty.call(message, "column_length")) + writer.uint32(/* id 7, wireType 0 =*/56).uint32(message.column_length); + if (message.charset != null && Object.hasOwnProperty.call(message, "charset")) + writer.uint32(/* id 8, wireType 0 =*/64).uint32(message.charset); + if (message.decimals != null && Object.hasOwnProperty.call(message, "decimals")) + writer.uint32(/* id 9, wireType 0 =*/72).uint32(message.decimals); + if (message.flags != null && Object.hasOwnProperty.call(message, "flags")) + writer.uint32(/* id 10, wireType 0 =*/80).uint32(message.flags); + if (message.column_type != null && Object.hasOwnProperty.call(message, "column_type")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.column_type); return writer; }; /** - * Encodes the specified ResultWithError message, length delimited. Does not implicitly {@link query.ResultWithError.verify|verify} messages. + * Encodes the specified Field message, length delimited. Does not implicitly {@link query.Field.verify|verify} messages. * @function encodeDelimited - * @memberof query.ResultWithError + * @memberof query.Field * @static - * @param {query.IResultWithError} message ResultWithError message or plain object to encode + * @param {query.IField} message Field message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ResultWithError.encodeDelimited = function encodeDelimited(message, writer) { + Field.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ResultWithError message from the specified reader or buffer. + * Decodes a Field message from the specified reader or buffer. * @function decode - * @memberof query.ResultWithError + * @memberof query.Field * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ResultWithError} ResultWithError + * @returns {query.Field} Field * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResultWithError.decode = function decode(reader, length) { + Field.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.query.ResultWithError(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.Field(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); + message.name = reader.string(); break; } case 2: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + message.type = reader.int32(); break; } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a ResultWithError message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof query.ResultWithError + case 3: { + message.table = reader.string(); + break; + } + case 4: { + message.org_table = reader.string(); + break; + } + case 5: { + message.database = reader.string(); + break; + } + case 6: { + message.org_name = reader.string(); + break; + } + case 7: { + message.column_length = reader.uint32(); + break; + } + case 8: { + message.charset = reader.uint32(); + break; + } + case 9: { + message.decimals = reader.uint32(); + break; + } + case 10: { + message.flags = reader.uint32(); + break; + } + case 11: { + message.column_type = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Field message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof query.Field * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ResultWithError} ResultWithError + * @returns {query.Field} Field * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ResultWithError.decodeDelimited = function decodeDelimited(reader) { + Field.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ResultWithError message. + * Verifies a Field message. * @function verify - * @memberof query.ResultWithError + * @memberof query.Field * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ResultWithError.verify = function verify(message) { + Field.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.vtrpc.RPCError.verify(message.error); - if (error) - return "error." + error; - } - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; - } + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "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; + } + if (message.table != null && message.hasOwnProperty("table")) + if (!$util.isString(message.table)) + return "table: string expected"; + if (message.org_table != null && message.hasOwnProperty("org_table")) + if (!$util.isString(message.org_table)) + return "org_table: string expected"; + if (message.database != null && message.hasOwnProperty("database")) + if (!$util.isString(message.database)) + return "database: string expected"; + if (message.org_name != null && message.hasOwnProperty("org_name")) + if (!$util.isString(message.org_name)) + return "org_name: string expected"; + if (message.column_length != null && message.hasOwnProperty("column_length")) + if (!$util.isInteger(message.column_length)) + return "column_length: integer expected"; + if (message.charset != null && message.hasOwnProperty("charset")) + if (!$util.isInteger(message.charset)) + return "charset: integer expected"; + if (message.decimals != null && message.hasOwnProperty("decimals")) + if (!$util.isInteger(message.decimals)) + return "decimals: integer expected"; + if (message.flags != null && message.hasOwnProperty("flags")) + if (!$util.isInteger(message.flags)) + return "flags: integer expected"; + if (message.column_type != null && message.hasOwnProperty("column_type")) + if (!$util.isString(message.column_type)) + return "column_type: string expected"; return null; }; /** - * Creates a ResultWithError message from a plain object. Also converts values to their respective internal types. + * Creates a Field message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ResultWithError + * @memberof query.Field * @static * @param {Object.} object Plain object - * @returns {query.ResultWithError} ResultWithError + * @returns {query.Field} Field */ - ResultWithError.fromObject = function fromObject(object) { - if (object instanceof $root.query.ResultWithError) + Field.fromObject = function fromObject(object) { + if (object instanceof $root.query.Field) return object; - let message = new $root.query.ResultWithError(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".query.ResultWithError.error: object expected"); - message.error = $root.vtrpc.RPCError.fromObject(object.error); - } - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.ResultWithError.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.query.Field(); + if (object.name != null) + message.name = String(object.name); + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; + } + break; + case "NULL_TYPE": + case 0: + message.type = 0; + break; + case "INT8": + case 257: + message.type = 257; + break; + case "UINT8": + case 770: + message.type = 770; + break; + case "INT16": + case 259: + message.type = 259; + break; + case "UINT16": + case 772: + message.type = 772; + break; + case "INT24": + case 261: + message.type = 261; + break; + case "UINT24": + case 774: + message.type = 774; + break; + case "INT32": + case 263: + message.type = 263; + break; + case "UINT32": + case 776: + message.type = 776; + break; + case "INT64": + case 265: + message.type = 265; + break; + case "UINT64": + case 778: + message.type = 778; + break; + case "FLOAT32": + case 1035: + message.type = 1035; + break; + case "FLOAT64": + case 1036: + message.type = 1036; + break; + case "TIMESTAMP": + case 2061: + message.type = 2061; + break; + case "DATE": + case 2062: + message.type = 2062; + break; + case "TIME": + case 2063: + message.type = 2063; + break; + case "DATETIME": + case 2064: + message.type = 2064; + break; + case "YEAR": + case 785: + message.type = 785; + break; + case "DECIMAL": + case 18: + message.type = 18; + break; + case "TEXT": + case 6163: + message.type = 6163; + break; + case "BLOB": + case 10260: + message.type = 10260; + break; + case "VARCHAR": + case 6165: + message.type = 6165; + break; + case "VARBINARY": + case 10262: + message.type = 10262; + break; + case "CHAR": + case 6167: + message.type = 6167; + break; + case "BINARY": + case 10264: + message.type = 10264; + break; + case "BIT": + case 2073: + message.type = 2073; + break; + case "ENUM": + case 2074: + message.type = 2074; + break; + case "SET": + case 2075: + message.type = 2075; + break; + case "TUPLE": + case 28: + message.type = 28; + break; + case "GEOMETRY": + case 2077: + message.type = 2077; + break; + case "JSON": + case 2078: + message.type = 2078; + break; + case "EXPRESSION": + case 31: + message.type = 31; + break; + case "HEXNUM": + case 4128: + message.type = 4128; + break; + case "HEXVAL": + case 4129: + message.type = 4129; + break; + case "BITNUM": + case 4130: + message.type = 4130; + break; } + if (object.table != null) + message.table = String(object.table); + if (object.org_table != null) + message.org_table = String(object.org_table); + if (object.database != null) + message.database = String(object.database); + if (object.org_name != null) + message.org_name = String(object.org_name); + if (object.column_length != null) + message.column_length = object.column_length >>> 0; + if (object.charset != null) + message.charset = object.charset >>> 0; + if (object.decimals != null) + message.decimals = object.decimals >>> 0; + if (object.flags != null) + message.flags = object.flags >>> 0; + if (object.column_type != null) + message.column_type = String(object.column_type); return message; }; /** - * Creates a plain object from a ResultWithError message. Also converts values to other types if specified. + * Creates a plain object from a Field message. Also converts values to other types if specified. * @function toObject - * @memberof query.ResultWithError + * @memberof query.Field * @static - * @param {query.ResultWithError} message ResultWithError + * @param {query.Field} message Field * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ResultWithError.toObject = function toObject(message, options) { + Field.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.error = null; - object.result = null; + object.name = ""; + object.type = options.enums === String ? "NULL_TYPE" : 0; + object.table = ""; + object.org_table = ""; + object.database = ""; + object.org_name = ""; + object.column_length = 0; + object.charset = 0; + object.decimals = 0; + object.flags = 0; + object.column_type = ""; } - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.vtrpc.RPCError.toObject(message.error, options); - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; + if (message.table != null && message.hasOwnProperty("table")) + object.table = message.table; + if (message.org_table != null && message.hasOwnProperty("org_table")) + object.org_table = message.org_table; + if (message.database != null && message.hasOwnProperty("database")) + object.database = message.database; + if (message.org_name != null && message.hasOwnProperty("org_name")) + object.org_name = message.org_name; + if (message.column_length != null && message.hasOwnProperty("column_length")) + object.column_length = message.column_length; + if (message.charset != null && message.hasOwnProperty("charset")) + object.charset = message.charset; + if (message.decimals != null && message.hasOwnProperty("decimals")) + object.decimals = message.decimals; + if (message.flags != null && message.hasOwnProperty("flags")) + object.flags = message.flags; + if (message.column_type != null && message.hasOwnProperty("column_type")) + object.column_type = message.column_type; return object; }; /** - * Converts this ResultWithError to JSON. + * Converts this Field to JSON. * @function toJSON - * @memberof query.ResultWithError + * @memberof query.Field * @instance * @returns {Object.} JSON object */ - ResultWithError.prototype.toJSON = function toJSON() { + Field.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ResultWithError + * Gets the default type url for Field * @function getTypeUrl - * @memberof query.ResultWithError + * @memberof query.Field * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ResultWithError.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Field.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ResultWithError"; + return typeUrlPrefix + "/query.Field"; }; - return ResultWithError; + return Field; })(); - query.StreamExecuteRequest = (function() { + query.Row = (function() { /** - * Properties of a StreamExecuteRequest. + * Properties of a Row. * @memberof query - * @interface IStreamExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] StreamExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] StreamExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] StreamExecuteRequest target - * @property {query.IBoundQuery|null} [query] StreamExecuteRequest query - * @property {query.IExecuteOptions|null} [options] StreamExecuteRequest options - * @property {number|Long|null} [transaction_id] StreamExecuteRequest transaction_id - * @property {number|Long|null} [reserved_id] StreamExecuteRequest reserved_id + * @interface IRow + * @property {Array.|null} [lengths] Row lengths + * @property {Uint8Array|null} [values] Row values */ /** - * Constructs a new StreamExecuteRequest. + * Constructs a new Row. * @memberof query - * @classdesc Represents a StreamExecuteRequest. - * @implements IStreamExecuteRequest + * @classdesc Represents a Row. + * @implements IRow * @constructor - * @param {query.IStreamExecuteRequest=} [properties] Properties to set + * @param {query.IRow=} [properties] Properties to set */ - function StreamExecuteRequest(properties) { + function Row(properties) { + this.lengths = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -79023,159 +79535,100 @@ export const query = $root.query = (() => { } /** - * StreamExecuteRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.StreamExecuteRequest - * @instance - */ - StreamExecuteRequest.prototype.effective_caller_id = null; - - /** - * StreamExecuteRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.StreamExecuteRequest - * @instance - */ - StreamExecuteRequest.prototype.immediate_caller_id = null; - - /** - * StreamExecuteRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.StreamExecuteRequest - * @instance - */ - StreamExecuteRequest.prototype.target = null; - - /** - * StreamExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.StreamExecuteRequest - * @instance - */ - StreamExecuteRequest.prototype.query = null; - - /** - * StreamExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.StreamExecuteRequest - * @instance - */ - StreamExecuteRequest.prototype.options = null; - - /** - * StreamExecuteRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.StreamExecuteRequest + * Row lengths. + * @member {Array.} lengths + * @memberof query.Row * @instance */ - StreamExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + Row.prototype.lengths = $util.emptyArray; /** - * StreamExecuteRequest reserved_id. - * @member {number|Long} reserved_id - * @memberof query.StreamExecuteRequest + * Row values. + * @member {Uint8Array} values + * @memberof query.Row * @instance */ - StreamExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + Row.prototype.values = $util.newBuffer([]); /** - * Creates a new StreamExecuteRequest instance using the specified properties. + * Creates a new Row instance using the specified properties. * @function create - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static - * @param {query.IStreamExecuteRequest=} [properties] Properties to set - * @returns {query.StreamExecuteRequest} StreamExecuteRequest instance + * @param {query.IRow=} [properties] Properties to set + * @returns {query.Row} Row instance */ - StreamExecuteRequest.create = function create(properties) { - return new StreamExecuteRequest(properties); + Row.create = function create(properties) { + return new Row(properties); }; /** - * Encodes the specified StreamExecuteRequest message. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. + * Encodes the specified Row message. Does not implicitly {@link query.Row.verify|verify} messages. * @function encode - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static - * @param {query.IStreamExecuteRequest} message StreamExecuteRequest message or plain object to encode + * @param {query.IRow} message Row message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamExecuteRequest.encode = function encode(message, writer) { + Row.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.transaction_id); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); - return writer; - }; - - /** - * Encodes the specified StreamExecuteRequest message, length delimited. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. - * @function encodeDelimited - * @memberof query.StreamExecuteRequest + if (message.lengths != null && message.lengths.length) { + writer.uint32(/* id 1, wireType 2 =*/10).fork(); + for (let i = 0; i < message.lengths.length; ++i) + writer.sint64(message.lengths[i]); + writer.ldelim(); + } + if (message.values != null && Object.hasOwnProperty.call(message, "values")) + writer.uint32(/* id 2, wireType 2 =*/18).bytes(message.values); + return writer; + }; + + /** + * Encodes the specified Row message, length delimited. Does not implicitly {@link query.Row.verify|verify} messages. + * @function encodeDelimited + * @memberof query.Row * @static - * @param {query.IStreamExecuteRequest} message StreamExecuteRequest message or plain object to encode + * @param {query.IRow} message Row message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + Row.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamExecuteRequest message from the specified reader or buffer. + * Decodes a Row message from the specified reader or buffer. * @function decode - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StreamExecuteRequest} StreamExecuteRequest + * @returns {query.Row} Row * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamExecuteRequest.decode = function decode(reader, length) { + Row.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.query.StreamExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.Row(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + if (!(message.lengths && message.lengths.length)) + message.lengths = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.lengths.push(reader.sint64()); + } else + message.lengths.push(reader.sint64()); break; } case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); - break; - } - case 5: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); - break; - } - case 6: { - message.transaction_id = reader.int64(); - break; - } - case 7: { - message.reserved_id = reader.int64(); + message.values = reader.bytes(); break; } default: @@ -79187,224 +79640,169 @@ export const query = $root.query = (() => { }; /** - * Decodes a StreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a Row message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamExecuteRequest} StreamExecuteRequest + * @returns {query.Row} Row * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + Row.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamExecuteRequest message. + * Verifies a Row message. * @function verify - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamExecuteRequest.verify = function verify(message) { + Row.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; - } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; + if (message.lengths != null && message.hasOwnProperty("lengths")) { + if (!Array.isArray(message.lengths)) + return "lengths: array expected"; + for (let i = 0; i < message.lengths.length; ++i) + if (!$util.isInteger(message.lengths[i]) && !(message.lengths[i] && $util.isInteger(message.lengths[i].low) && $util.isInteger(message.lengths[i].high))) + return "lengths: integer|Long[] expected"; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.values != null && message.hasOwnProperty("values")) + if (!(message.values && typeof message.values.length === "number" || $util.isString(message.values))) + return "values: buffer expected"; return null; }; /** - * Creates a StreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a Row message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static * @param {Object.} object Plain object - * @returns {query.StreamExecuteRequest} StreamExecuteRequest + * @returns {query.Row} Row */ - StreamExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamExecuteRequest) + Row.fromObject = function fromObject(object) { + if (object instanceof $root.query.Row) return object; - let message = new $root.query.StreamExecuteRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.StreamExecuteRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); - } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.StreamExecuteRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.StreamExecuteRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); - } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.StreamExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.StreamExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); + let message = new $root.query.Row(); + if (object.lengths) { + if (!Array.isArray(object.lengths)) + throw TypeError(".query.Row.lengths: array expected"); + message.lengths = []; + for (let i = 0; i < object.lengths.length; ++i) + if ($util.Long) + (message.lengths[i] = $util.Long.fromValue(object.lengths[i])).unsigned = false; + else if (typeof object.lengths[i] === "string") + message.lengths[i] = parseInt(object.lengths[i], 10); + else if (typeof object.lengths[i] === "number") + message.lengths[i] = object.lengths[i]; + else if (typeof object.lengths[i] === "object") + message.lengths[i] = new $util.LongBits(object.lengths[i].low >>> 0, object.lengths[i].high >>> 0).toNumber(); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.values != null) + if (typeof object.values === "string") + $util.base64.decode(object.values, message.values = $util.newBuffer($util.base64.length(object.values)), 0); + else if (object.values.length >= 0) + message.values = object.values; return message; }; /** - * Creates a plain object from a StreamExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a Row message. Also converts values to other types if specified. * @function toObject - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static - * @param {query.StreamExecuteRequest} message StreamExecuteRequest + * @param {query.Row} message Row * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamExecuteRequest.toObject = function toObject(message, options) { + Row.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.query = null; - object.options = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.reserved_id = options.longs === String ? "0" : 0; + if (options.arrays || options.defaults) + object.lengths = []; + if (options.defaults) + if (options.bytes === String) + object.values = ""; + else { + object.values = []; + if (options.bytes !== Array) + object.values = $util.newBuffer(object.values); + } + if (message.lengths && message.lengths.length) { + object.lengths = []; + for (let j = 0; j < message.lengths.length; ++j) + if (typeof message.lengths[j] === "number") + object.lengths[j] = options.longs === String ? String(message.lengths[j]) : message.lengths[j]; + else + object.lengths[j] = options.longs === String ? $util.Long.prototype.toString.call(message.lengths[j]) : options.longs === Number ? new $util.LongBits(message.lengths[j].low >>> 0, message.lengths[j].high >>> 0).toNumber() : message.lengths[j]; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; - else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + if (message.values != null && message.hasOwnProperty("values")) + object.values = options.bytes === String ? $util.base64.encode(message.values, 0, message.values.length) : options.bytes === Array ? Array.prototype.slice.call(message.values) : message.values; return object; }; /** - * Converts this StreamExecuteRequest to JSON. + * Converts this Row to JSON. * @function toJSON - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @instance * @returns {Object.} JSON object */ - StreamExecuteRequest.prototype.toJSON = function toJSON() { + Row.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamExecuteRequest + * Gets the default type url for Row * @function getTypeUrl - * @memberof query.StreamExecuteRequest + * @memberof query.Row * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Row.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StreamExecuteRequest"; + return typeUrlPrefix + "/query.Row"; }; - return StreamExecuteRequest; + return Row; })(); - query.StreamExecuteResponse = (function() { + query.QueryResult = (function() { /** - * Properties of a StreamExecuteResponse. + * Properties of a QueryResult. * @memberof query - * @interface IStreamExecuteResponse - * @property {query.IQueryResult|null} [result] StreamExecuteResponse result + * @interface IQueryResult + * @property {Array.|null} [fields] QueryResult fields + * @property {number|Long|null} [rows_affected] QueryResult rows_affected + * @property {number|Long|null} [insert_id] QueryResult insert_id + * @property {Array.|null} [rows] QueryResult rows + * @property {string|null} [info] QueryResult info + * @property {string|null} [session_state_changes] QueryResult session_state_changes */ /** - * Constructs a new StreamExecuteResponse. + * Constructs a new QueryResult. * @memberof query - * @classdesc Represents a StreamExecuteResponse. - * @implements IStreamExecuteResponse + * @classdesc Represents a QueryResult. + * @implements IQueryResult * @constructor - * @param {query.IStreamExecuteResponse=} [properties] Properties to set + * @param {query.IQueryResult=} [properties] Properties to set */ - function StreamExecuteResponse(properties) { + function QueryResult(properties) { + this.fields = []; + this.rows = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -79412,75 +79810,151 @@ export const query = $root.query = (() => { } /** - * StreamExecuteResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.StreamExecuteResponse + * QueryResult fields. + * @member {Array.} fields + * @memberof query.QueryResult * @instance */ - StreamExecuteResponse.prototype.result = null; + QueryResult.prototype.fields = $util.emptyArray; /** - * Creates a new StreamExecuteResponse instance using the specified properties. + * QueryResult rows_affected. + * @member {number|Long} rows_affected + * @memberof query.QueryResult + * @instance + */ + QueryResult.prototype.rows_affected = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + + /** + * QueryResult insert_id. + * @member {number|Long} insert_id + * @memberof query.QueryResult + * @instance + */ + QueryResult.prototype.insert_id = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + + /** + * QueryResult rows. + * @member {Array.} rows + * @memberof query.QueryResult + * @instance + */ + QueryResult.prototype.rows = $util.emptyArray; + + /** + * QueryResult info. + * @member {string} info + * @memberof query.QueryResult + * @instance + */ + QueryResult.prototype.info = ""; + + /** + * QueryResult session_state_changes. + * @member {string} session_state_changes + * @memberof query.QueryResult + * @instance + */ + QueryResult.prototype.session_state_changes = ""; + + /** + * Creates a new QueryResult instance using the specified properties. * @function create - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static - * @param {query.IStreamExecuteResponse=} [properties] Properties to set - * @returns {query.StreamExecuteResponse} StreamExecuteResponse instance + * @param {query.IQueryResult=} [properties] Properties to set + * @returns {query.QueryResult} QueryResult instance */ - StreamExecuteResponse.create = function create(properties) { - return new StreamExecuteResponse(properties); + QueryResult.create = function create(properties) { + return new QueryResult(properties); }; /** - * Encodes the specified StreamExecuteResponse message. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. + * Encodes the specified QueryResult message. Does not implicitly {@link query.QueryResult.verify|verify} messages. * @function encode - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static - * @param {query.IStreamExecuteResponse} message StreamExecuteResponse message or plain object to encode + * @param {query.IQueryResult} message QueryResult message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamExecuteResponse.encode = function encode(message, writer) { + QueryResult.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.fields != null && message.fields.length) + for (let i = 0; i < message.fields.length; ++i) + $root.query.Field.encode(message.fields[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.rows_affected != null && Object.hasOwnProperty.call(message, "rows_affected")) + writer.uint32(/* id 2, wireType 0 =*/16).uint64(message.rows_affected); + if (message.insert_id != null && Object.hasOwnProperty.call(message, "insert_id")) + writer.uint32(/* id 3, wireType 0 =*/24).uint64(message.insert_id); + if (message.rows != null && message.rows.length) + for (let i = 0; i < message.rows.length; ++i) + $root.query.Row.encode(message.rows[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.info != null && Object.hasOwnProperty.call(message, "info")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.info); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.session_state_changes); return writer; }; /** - * Encodes the specified StreamExecuteResponse message, length delimited. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. + * Encodes the specified QueryResult message, length delimited. Does not implicitly {@link query.QueryResult.verify|verify} messages. * @function encodeDelimited - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static - * @param {query.IStreamExecuteResponse} message StreamExecuteResponse message or plain object to encode + * @param {query.IQueryResult} message QueryResult message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + QueryResult.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamExecuteResponse message from the specified reader or buffer. + * Decodes a QueryResult message from the specified reader or buffer. * @function decode - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StreamExecuteResponse} StreamExecuteResponse + * @returns {query.QueryResult} QueryResult * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamExecuteResponse.decode = function decode(reader, length) { + QueryResult.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.query.StreamExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.QueryResult(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + if (!(message.fields && message.fields.length)) + message.fields = []; + message.fields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } + case 2: { + message.rows_affected = reader.uint64(); + break; + } + case 3: { + message.insert_id = reader.uint64(); + break; + } + case 4: { + if (!(message.rows && message.rows.length)) + message.rows = []; + message.rows.push($root.query.Row.decode(reader, reader.uint32())); + break; + } + case 6: { + message.info = reader.string(); + break; + } + case 7: { + message.session_state_changes = reader.string(); break; } default: @@ -79492,130 +79966,228 @@ export const query = $root.query = (() => { }; /** - * Decodes a StreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a QueryResult message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamExecuteResponse} StreamExecuteResponse + * @returns {query.QueryResult} QueryResult * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + QueryResult.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamExecuteResponse message. + * Verifies a QueryResult message. * @function verify - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamExecuteResponse.verify = function verify(message) { + QueryResult.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; + if (message.fields != null && message.hasOwnProperty("fields")) { + if (!Array.isArray(message.fields)) + return "fields: array expected"; + for (let i = 0; i < message.fields.length; ++i) { + let error = $root.query.Field.verify(message.fields[i]); + if (error) + return "fields." + error; + } + } + if (message.rows_affected != null && message.hasOwnProperty("rows_affected")) + if (!$util.isInteger(message.rows_affected) && !(message.rows_affected && $util.isInteger(message.rows_affected.low) && $util.isInteger(message.rows_affected.high))) + return "rows_affected: integer|Long expected"; + if (message.insert_id != null && message.hasOwnProperty("insert_id")) + if (!$util.isInteger(message.insert_id) && !(message.insert_id && $util.isInteger(message.insert_id.low) && $util.isInteger(message.insert_id.high))) + return "insert_id: integer|Long expected"; + if (message.rows != null && message.hasOwnProperty("rows")) { + if (!Array.isArray(message.rows)) + return "rows: array expected"; + for (let i = 0; i < message.rows.length; ++i) { + let error = $root.query.Row.verify(message.rows[i]); + if (error) + return "rows." + error; + } } + if (message.info != null && message.hasOwnProperty("info")) + if (!$util.isString(message.info)) + return "info: string expected"; + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a StreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a QueryResult message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static * @param {Object.} object Plain object - * @returns {query.StreamExecuteResponse} StreamExecuteResponse + * @returns {query.QueryResult} QueryResult */ - StreamExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamExecuteResponse) + QueryResult.fromObject = function fromObject(object) { + if (object instanceof $root.query.QueryResult) return object; - let message = new $root.query.StreamExecuteResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.StreamExecuteResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.query.QueryResult(); + if (object.fields) { + if (!Array.isArray(object.fields)) + throw TypeError(".query.QueryResult.fields: array expected"); + message.fields = []; + for (let i = 0; i < object.fields.length; ++i) { + if (typeof object.fields[i] !== "object") + throw TypeError(".query.QueryResult.fields: object expected"); + message.fields[i] = $root.query.Field.fromObject(object.fields[i]); + } } - return message; - }; - - /** - * Creates a plain object from a StreamExecuteResponse message. Also converts values to other types if specified. - * @function toObject - * @memberof query.StreamExecuteResponse - * @static - * @param {query.StreamExecuteResponse} message StreamExecuteResponse - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - StreamExecuteResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); - return object; - }; + if (object.rows_affected != null) + if ($util.Long) + (message.rows_affected = $util.Long.fromValue(object.rows_affected)).unsigned = true; + else if (typeof object.rows_affected === "string") + message.rows_affected = parseInt(object.rows_affected, 10); + else if (typeof object.rows_affected === "number") + message.rows_affected = object.rows_affected; + else if (typeof object.rows_affected === "object") + message.rows_affected = new $util.LongBits(object.rows_affected.low >>> 0, object.rows_affected.high >>> 0).toNumber(true); + if (object.insert_id != null) + if ($util.Long) + (message.insert_id = $util.Long.fromValue(object.insert_id)).unsigned = true; + else if (typeof object.insert_id === "string") + message.insert_id = parseInt(object.insert_id, 10); + else if (typeof object.insert_id === "number") + message.insert_id = object.insert_id; + else if (typeof object.insert_id === "object") + message.insert_id = new $util.LongBits(object.insert_id.low >>> 0, object.insert_id.high >>> 0).toNumber(true); + if (object.rows) { + if (!Array.isArray(object.rows)) + throw TypeError(".query.QueryResult.rows: array expected"); + message.rows = []; + for (let i = 0; i < object.rows.length; ++i) { + if (typeof object.rows[i] !== "object") + throw TypeError(".query.QueryResult.rows: object expected"); + message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + } + } + if (object.info != null) + message.info = String(object.info); + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); + return message; + }; /** - * Converts this StreamExecuteResponse to JSON. + * Creates a plain object from a QueryResult message. Also converts values to other types if specified. + * @function toObject + * @memberof query.QueryResult + * @static + * @param {query.QueryResult} message QueryResult + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + QueryResult.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.fields = []; + object.rows = []; + } + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, true); + object.rows_affected = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.rows_affected = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, true); + object.insert_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.insert_id = options.longs === String ? "0" : 0; + object.info = ""; + object.session_state_changes = ""; + } + if (message.fields && message.fields.length) { + object.fields = []; + for (let j = 0; j < message.fields.length; ++j) + object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + } + if (message.rows_affected != null && message.hasOwnProperty("rows_affected")) + if (typeof message.rows_affected === "number") + object.rows_affected = options.longs === String ? String(message.rows_affected) : message.rows_affected; + else + object.rows_affected = options.longs === String ? $util.Long.prototype.toString.call(message.rows_affected) : options.longs === Number ? new $util.LongBits(message.rows_affected.low >>> 0, message.rows_affected.high >>> 0).toNumber(true) : message.rows_affected; + if (message.insert_id != null && message.hasOwnProperty("insert_id")) + if (typeof message.insert_id === "number") + object.insert_id = options.longs === String ? String(message.insert_id) : message.insert_id; + else + object.insert_id = options.longs === String ? $util.Long.prototype.toString.call(message.insert_id) : options.longs === Number ? new $util.LongBits(message.insert_id.low >>> 0, message.insert_id.high >>> 0).toNumber(true) : message.insert_id; + if (message.rows && message.rows.length) { + object.rows = []; + for (let j = 0; j < message.rows.length; ++j) + object.rows[j] = $root.query.Row.toObject(message.rows[j], options); + } + if (message.info != null && message.hasOwnProperty("info")) + object.info = message.info; + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; + return object; + }; + + /** + * Converts this QueryResult to JSON. * @function toJSON - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @instance * @returns {Object.} JSON object */ - StreamExecuteResponse.prototype.toJSON = function toJSON() { + QueryResult.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamExecuteResponse + * Gets the default type url for QueryResult * @function getTypeUrl - * @memberof query.StreamExecuteResponse + * @memberof query.QueryResult * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + QueryResult.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StreamExecuteResponse"; + return typeUrlPrefix + "/query.QueryResult"; }; - return StreamExecuteResponse; + return QueryResult; })(); - query.BeginRequest = (function() { + query.QueryWarning = (function() { /** - * Properties of a BeginRequest. + * Properties of a QueryWarning. * @memberof query - * @interface IBeginRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginRequest immediate_caller_id - * @property {query.ITarget|null} [target] BeginRequest target - * @property {query.IExecuteOptions|null} [options] BeginRequest options + * @interface IQueryWarning + * @property {number|null} [code] QueryWarning code + * @property {string|null} [message] QueryWarning message */ /** - * Constructs a new BeginRequest. + * Constructs a new QueryWarning. * @memberof query - * @classdesc Represents a BeginRequest. - * @implements IBeginRequest + * @classdesc Represents a QueryWarning. + * @implements IQueryWarning * @constructor - * @param {query.IBeginRequest=} [properties] Properties to set + * @param {query.IQueryWarning=} [properties] Properties to set */ - function BeginRequest(properties) { + function QueryWarning(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -79623,117 +80195,89 @@ export const query = $root.query = (() => { } /** - * BeginRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.BeginRequest - * @instance - */ - BeginRequest.prototype.effective_caller_id = null; - - /** - * BeginRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.BeginRequest - * @instance - */ - BeginRequest.prototype.immediate_caller_id = null; - - /** - * BeginRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.BeginRequest + * QueryWarning code. + * @member {number} code + * @memberof query.QueryWarning * @instance */ - BeginRequest.prototype.target = null; + QueryWarning.prototype.code = 0; /** - * BeginRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.BeginRequest + * QueryWarning message. + * @member {string} message + * @memberof query.QueryWarning * @instance */ - BeginRequest.prototype.options = null; + QueryWarning.prototype.message = ""; /** - * Creates a new BeginRequest instance using the specified properties. + * Creates a new QueryWarning instance using the specified properties. * @function create - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static - * @param {query.IBeginRequest=} [properties] Properties to set - * @returns {query.BeginRequest} BeginRequest instance + * @param {query.IQueryWarning=} [properties] Properties to set + * @returns {query.QueryWarning} QueryWarning instance */ - BeginRequest.create = function create(properties) { - return new BeginRequest(properties); + QueryWarning.create = function create(properties) { + return new QueryWarning(properties); }; /** - * Encodes the specified BeginRequest message. Does not implicitly {@link query.BeginRequest.verify|verify} messages. + * Encodes the specified QueryWarning message. Does not implicitly {@link query.QueryWarning.verify|verify} messages. * @function encode - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static - * @param {query.IBeginRequest} message BeginRequest message or plain object to encode + * @param {query.IQueryWarning} message QueryWarning message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginRequest.encode = function encode(message, writer) { + QueryWarning.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.code != null && Object.hasOwnProperty.call(message, "code")) + writer.uint32(/* id 1, wireType 0 =*/8).uint32(message.code); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.message); return writer; }; /** - * Encodes the specified BeginRequest message, length delimited. Does not implicitly {@link query.BeginRequest.verify|verify} messages. + * Encodes the specified QueryWarning message, length delimited. Does not implicitly {@link query.QueryWarning.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static - * @param {query.IBeginRequest} message BeginRequest message or plain object to encode + * @param {query.IQueryWarning} message QueryWarning message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginRequest.encodeDelimited = function encodeDelimited(message, writer) { + QueryWarning.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginRequest message from the specified reader or buffer. + * Decodes a QueryWarning message from the specified reader or buffer. * @function decode - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginRequest} BeginRequest + * @returns {query.QueryWarning} QueryWarning * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginRequest.decode = function decode(reader, length) { + QueryWarning.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.query.BeginRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.QueryWarning(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + message.code = reader.uint32(); break; } case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + message.message = reader.string(); break; } default: @@ -79745,169 +80289,133 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginRequest message from the specified reader or buffer, length delimited. + * Decodes a QueryWarning message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginRequest} BeginRequest + * @returns {query.QueryWarning} QueryWarning * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginRequest.decodeDelimited = function decodeDelimited(reader) { + QueryWarning.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginRequest message. + * Verifies a QueryWarning message. * @function verify - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginRequest.verify = function verify(message) { + QueryWarning.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; - } + if (message.code != null && message.hasOwnProperty("code")) + if (!$util.isInteger(message.code)) + return "code: integer expected"; + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; return null; }; /** - * Creates a BeginRequest message from a plain object. Also converts values to their respective internal types. + * Creates a QueryWarning message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static * @param {Object.} object Plain object - * @returns {query.BeginRequest} BeginRequest + * @returns {query.QueryWarning} QueryWarning */ - BeginRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginRequest) + QueryWarning.fromObject = function fromObject(object) { + if (object instanceof $root.query.QueryWarning) return object; - let message = new $root.query.BeginRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.BeginRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); - } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.BeginRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.BeginRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.BeginRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); - } + let message = new $root.query.QueryWarning(); + if (object.code != null) + message.code = object.code >>> 0; + if (object.message != null) + message.message = String(object.message); return message; }; /** - * Creates a plain object from a BeginRequest message. Also converts values to other types if specified. + * Creates a plain object from a QueryWarning message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static - * @param {query.BeginRequest} message BeginRequest + * @param {query.QueryWarning} message QueryWarning * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginRequest.toObject = function toObject(message, options) { + QueryWarning.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.options = null; + object.code = 0; + object.message = ""; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.code != null && message.hasOwnProperty("code")) + object.code = message.code; + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; return object; }; /** - * Converts this BeginRequest to JSON. + * Converts this QueryWarning to JSON. * @function toJSON - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @instance * @returns {Object.} JSON object */ - BeginRequest.prototype.toJSON = function toJSON() { + QueryWarning.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginRequest + * Gets the default type url for QueryWarning * @function getTypeUrl - * @memberof query.BeginRequest + * @memberof query.QueryWarning * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + QueryWarning.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginRequest"; + return typeUrlPrefix + "/query.QueryWarning"; }; - return BeginRequest; + return QueryWarning; })(); - query.BeginResponse = (function() { + query.StreamEvent = (function() { /** - * Properties of a BeginResponse. + * Properties of a StreamEvent. * @memberof query - * @interface IBeginResponse - * @property {number|Long|null} [transaction_id] BeginResponse transaction_id - * @property {topodata.ITabletAlias|null} [tablet_alias] BeginResponse tablet_alias - * @property {string|null} [session_state_changes] BeginResponse session_state_changes + * @interface IStreamEvent + * @property {Array.|null} [statements] StreamEvent statements + * @property {query.IEventToken|null} [event_token] StreamEvent event_token */ /** - * Constructs a new BeginResponse. + * Constructs a new StreamEvent. * @memberof query - * @classdesc Represents a BeginResponse. - * @implements IBeginResponse + * @classdesc Represents a StreamEvent. + * @implements IStreamEvent * @constructor - * @param {query.IBeginResponse=} [properties] Properties to set + * @param {query.IStreamEvent=} [properties] Properties to set */ - function BeginResponse(properties) { + function StreamEvent(properties) { + this.statements = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -79915,103 +80423,92 @@ export const query = $root.query = (() => { } /** - * BeginResponse transaction_id. - * @member {number|Long} transaction_id - * @memberof query.BeginResponse - * @instance - */ - BeginResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * BeginResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.BeginResponse + * StreamEvent statements. + * @member {Array.} statements + * @memberof query.StreamEvent * @instance */ - BeginResponse.prototype.tablet_alias = null; + StreamEvent.prototype.statements = $util.emptyArray; /** - * BeginResponse session_state_changes. - * @member {string} session_state_changes - * @memberof query.BeginResponse + * StreamEvent event_token. + * @member {query.IEventToken|null|undefined} event_token + * @memberof query.StreamEvent * @instance */ - BeginResponse.prototype.session_state_changes = ""; + StreamEvent.prototype.event_token = null; /** - * Creates a new BeginResponse instance using the specified properties. + * Creates a new StreamEvent instance using the specified properties. * @function create - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static - * @param {query.IBeginResponse=} [properties] Properties to set - * @returns {query.BeginResponse} BeginResponse instance + * @param {query.IStreamEvent=} [properties] Properties to set + * @returns {query.StreamEvent} StreamEvent instance */ - BeginResponse.create = function create(properties) { - return new BeginResponse(properties); + StreamEvent.create = function create(properties) { + return new StreamEvent(properties); }; /** - * Encodes the specified BeginResponse message. Does not implicitly {@link query.BeginResponse.verify|verify} messages. + * Encodes the specified StreamEvent message. Does not implicitly {@link query.StreamEvent.verify|verify} messages. * @function encode - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static - * @param {query.IBeginResponse} message BeginResponse message or plain object to encode + * @param {query.IStreamEvent} message StreamEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginResponse.encode = function encode(message, writer) { + StreamEvent.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.transaction_id); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.session_state_changes); + if (message.statements != null && message.statements.length) + for (let i = 0; i < message.statements.length; ++i) + $root.query.StreamEvent.Statement.encode(message.statements[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.event_token != null && Object.hasOwnProperty.call(message, "event_token")) + $root.query.EventToken.encode(message.event_token, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified BeginResponse message, length delimited. Does not implicitly {@link query.BeginResponse.verify|verify} messages. + * Encodes the specified StreamEvent message, length delimited. Does not implicitly {@link query.StreamEvent.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static - * @param {query.IBeginResponse} message BeginResponse message or plain object to encode + * @param {query.IStreamEvent} message StreamEvent message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginResponse.encodeDelimited = function encodeDelimited(message, writer) { + StreamEvent.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginResponse message from the specified reader or buffer. + * Decodes a StreamEvent message from the specified reader or buffer. * @function decode - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginResponse} BeginResponse + * @returns {query.StreamEvent} StreamEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginResponse.decode = function decode(reader, length) { + StreamEvent.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.query.BeginResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StreamEvent(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.transaction_id = reader.int64(); + if (!(message.statements && message.statements.length)) + message.statements = []; + message.statements.push($root.query.StreamEvent.Statement.decode(reader, reader.uint32())); break; } case 2: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 3: { - message.session_state_changes = reader.string(); + message.event_token = $root.query.EventToken.decode(reader, reader.uint32()); break; } default: @@ -80023,161 +80520,548 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginResponse message from the specified reader or buffer, length delimited. + * Decodes a StreamEvent message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginResponse} BeginResponse + * @returns {query.StreamEvent} StreamEvent * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginResponse.decodeDelimited = function decodeDelimited(reader) { + StreamEvent.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginResponse message. + * Verifies a StreamEvent message. * @function verify - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginResponse.verify = function verify(message) { + StreamEvent.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (message.statements != null && message.hasOwnProperty("statements")) { + if (!Array.isArray(message.statements)) + return "statements: array expected"; + for (let i = 0; i < message.statements.length; ++i) { + let error = $root.query.StreamEvent.Statement.verify(message.statements[i]); + if (error) + return "statements." + error; + } + } + if (message.event_token != null && message.hasOwnProperty("event_token")) { + let error = $root.query.EventToken.verify(message.event_token); if (error) - return "tablet_alias." + error; + return "event_token." + error; } - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; return null; }; /** - * Creates a BeginResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StreamEvent message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static * @param {Object.} object Plain object - * @returns {query.BeginResponse} BeginResponse + * @returns {query.StreamEvent} StreamEvent */ - BeginResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginResponse) + StreamEvent.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamEvent) return object; - let message = new $root.query.BeginResponse(); - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".query.BeginResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); - return message; - }; - + let message = new $root.query.StreamEvent(); + if (object.statements) { + if (!Array.isArray(object.statements)) + throw TypeError(".query.StreamEvent.statements: array expected"); + message.statements = []; + for (let i = 0; i < object.statements.length; ++i) { + if (typeof object.statements[i] !== "object") + throw TypeError(".query.StreamEvent.statements: object expected"); + message.statements[i] = $root.query.StreamEvent.Statement.fromObject(object.statements[i]); + } + } + if (object.event_token != null) { + if (typeof object.event_token !== "object") + throw TypeError(".query.StreamEvent.event_token: object expected"); + message.event_token = $root.query.EventToken.fromObject(object.event_token); + } + return message; + }; + /** - * Creates a plain object from a BeginResponse message. Also converts values to other types if specified. + * Creates a plain object from a StreamEvent message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static - * @param {query.BeginResponse} message BeginResponse + * @param {query.StreamEvent} message StreamEvent * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginResponse.toObject = function toObject(message, options) { + StreamEvent.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - object.tablet_alias = null; - object.session_state_changes = ""; + if (options.arrays || options.defaults) + object.statements = []; + if (options.defaults) + object.event_token = null; + if (message.statements && message.statements.length) { + object.statements = []; + for (let j = 0; j < message.statements.length; ++j) + object.statements[j] = $root.query.StreamEvent.Statement.toObject(message.statements[j], options); } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; + if (message.event_token != null && message.hasOwnProperty("event_token")) + object.event_token = $root.query.EventToken.toObject(message.event_token, options); return object; }; /** - * Converts this BeginResponse to JSON. + * Converts this StreamEvent to JSON. * @function toJSON - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @instance * @returns {Object.} JSON object */ - BeginResponse.prototype.toJSON = function toJSON() { + StreamEvent.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginResponse + * Gets the default type url for StreamEvent * @function getTypeUrl - * @memberof query.BeginResponse + * @memberof query.StreamEvent * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamEvent.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginResponse"; + return typeUrlPrefix + "/query.StreamEvent"; }; - return BeginResponse; + StreamEvent.Statement = (function() { + + /** + * Properties of a Statement. + * @memberof query.StreamEvent + * @interface IStatement + * @property {query.StreamEvent.Statement.Category|null} [category] Statement category + * @property {string|null} [table_name] Statement table_name + * @property {Array.|null} [primary_key_fields] Statement primary_key_fields + * @property {Array.|null} [primary_key_values] Statement primary_key_values + * @property {Uint8Array|null} [sql] Statement sql + */ + + /** + * Constructs a new Statement. + * @memberof query.StreamEvent + * @classdesc Represents a Statement. + * @implements IStatement + * @constructor + * @param {query.StreamEvent.IStatement=} [properties] Properties to set + */ + function Statement(properties) { + this.primary_key_fields = []; + this.primary_key_values = []; + 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]]; + } + + /** + * Statement category. + * @member {query.StreamEvent.Statement.Category} category + * @memberof query.StreamEvent.Statement + * @instance + */ + Statement.prototype.category = 0; + + /** + * Statement table_name. + * @member {string} table_name + * @memberof query.StreamEvent.Statement + * @instance + */ + Statement.prototype.table_name = ""; + + /** + * Statement primary_key_fields. + * @member {Array.} primary_key_fields + * @memberof query.StreamEvent.Statement + * @instance + */ + Statement.prototype.primary_key_fields = $util.emptyArray; + + /** + * Statement primary_key_values. + * @member {Array.} primary_key_values + * @memberof query.StreamEvent.Statement + * @instance + */ + Statement.prototype.primary_key_values = $util.emptyArray; + + /** + * Statement sql. + * @member {Uint8Array} sql + * @memberof query.StreamEvent.Statement + * @instance + */ + Statement.prototype.sql = $util.newBuffer([]); + + /** + * Creates a new Statement instance using the specified properties. + * @function create + * @memberof query.StreamEvent.Statement + * @static + * @param {query.StreamEvent.IStatement=} [properties] Properties to set + * @returns {query.StreamEvent.Statement} Statement instance + */ + Statement.create = function create(properties) { + return new Statement(properties); + }; + + /** + * Encodes the specified Statement message. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. + * @function encode + * @memberof query.StreamEvent.Statement + * @static + * @param {query.StreamEvent.IStatement} message Statement message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Statement.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.category != null && Object.hasOwnProperty.call(message, "category")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.category); + if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.table_name); + if (message.primary_key_fields != null && message.primary_key_fields.length) + for (let i = 0; i < message.primary_key_fields.length; ++i) + $root.query.Field.encode(message.primary_key_fields[i], writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.primary_key_values != null && message.primary_key_values.length) + for (let i = 0; i < message.primary_key_values.length; ++i) + $root.query.Row.encode(message.primary_key_values[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) + writer.uint32(/* id 5, wireType 2 =*/42).bytes(message.sql); + return writer; + }; + + /** + * Encodes the specified Statement message, length delimited. Does not implicitly {@link query.StreamEvent.Statement.verify|verify} messages. + * @function encodeDelimited + * @memberof query.StreamEvent.Statement + * @static + * @param {query.StreamEvent.IStatement} message Statement message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Statement.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Statement message from the specified reader or buffer. + * @function decode + * @memberof query.StreamEvent.Statement + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {query.StreamEvent.Statement} Statement + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Statement.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.query.StreamEvent.Statement(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.category = reader.int32(); + break; + } + case 2: { + message.table_name = reader.string(); + break; + } + case 3: { + if (!(message.primary_key_fields && message.primary_key_fields.length)) + message.primary_key_fields = []; + message.primary_key_fields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } + case 4: { + if (!(message.primary_key_values && message.primary_key_values.length)) + message.primary_key_values = []; + message.primary_key_values.push($root.query.Row.decode(reader, reader.uint32())); + break; + } + case 5: { + message.sql = reader.bytes(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Statement message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof query.StreamEvent.Statement + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {query.StreamEvent.Statement} Statement + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Statement.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Statement message. + * @function verify + * @memberof query.StreamEvent.Statement + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Statement.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.category != null && message.hasOwnProperty("category")) + switch (message.category) { + default: + return "category: enum value expected"; + case 0: + case 1: + case 2: + break; + } + if (message.table_name != null && message.hasOwnProperty("table_name")) + if (!$util.isString(message.table_name)) + return "table_name: string expected"; + if (message.primary_key_fields != null && message.hasOwnProperty("primary_key_fields")) { + if (!Array.isArray(message.primary_key_fields)) + return "primary_key_fields: array expected"; + for (let i = 0; i < message.primary_key_fields.length; ++i) { + let error = $root.query.Field.verify(message.primary_key_fields[i]); + if (error) + return "primary_key_fields." + error; + } + } + if (message.primary_key_values != null && message.hasOwnProperty("primary_key_values")) { + if (!Array.isArray(message.primary_key_values)) + return "primary_key_values: array expected"; + for (let i = 0; i < message.primary_key_values.length; ++i) { + let error = $root.query.Row.verify(message.primary_key_values[i]); + if (error) + return "primary_key_values." + error; + } + } + if (message.sql != null && message.hasOwnProperty("sql")) + if (!(message.sql && typeof message.sql.length === "number" || $util.isString(message.sql))) + return "sql: buffer expected"; + return null; + }; + + /** + * Creates a Statement message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof query.StreamEvent.Statement + * @static + * @param {Object.} object Plain object + * @returns {query.StreamEvent.Statement} Statement + */ + Statement.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamEvent.Statement) + return object; + let message = new $root.query.StreamEvent.Statement(); + switch (object.category) { + default: + if (typeof object.category === "number") { + message.category = object.category; + break; + } + break; + case "Error": + case 0: + message.category = 0; + break; + case "DML": + case 1: + message.category = 1; + break; + case "DDL": + case 2: + message.category = 2; + break; + } + if (object.table_name != null) + message.table_name = String(object.table_name); + if (object.primary_key_fields) { + if (!Array.isArray(object.primary_key_fields)) + throw TypeError(".query.StreamEvent.Statement.primary_key_fields: array expected"); + message.primary_key_fields = []; + for (let i = 0; i < object.primary_key_fields.length; ++i) { + if (typeof object.primary_key_fields[i] !== "object") + throw TypeError(".query.StreamEvent.Statement.primary_key_fields: object expected"); + message.primary_key_fields[i] = $root.query.Field.fromObject(object.primary_key_fields[i]); + } + } + if (object.primary_key_values) { + if (!Array.isArray(object.primary_key_values)) + throw TypeError(".query.StreamEvent.Statement.primary_key_values: array expected"); + message.primary_key_values = []; + for (let i = 0; i < object.primary_key_values.length; ++i) { + if (typeof object.primary_key_values[i] !== "object") + throw TypeError(".query.StreamEvent.Statement.primary_key_values: object expected"); + message.primary_key_values[i] = $root.query.Row.fromObject(object.primary_key_values[i]); + } + } + if (object.sql != null) + if (typeof object.sql === "string") + $util.base64.decode(object.sql, message.sql = $util.newBuffer($util.base64.length(object.sql)), 0); + else if (object.sql.length >= 0) + message.sql = object.sql; + return message; + }; + + /** + * Creates a plain object from a Statement message. Also converts values to other types if specified. + * @function toObject + * @memberof query.StreamEvent.Statement + * @static + * @param {query.StreamEvent.Statement} message Statement + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Statement.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.primary_key_fields = []; + object.primary_key_values = []; + } + if (options.defaults) { + object.category = options.enums === String ? "Error" : 0; + object.table_name = ""; + if (options.bytes === String) + object.sql = ""; + else { + object.sql = []; + if (options.bytes !== Array) + object.sql = $util.newBuffer(object.sql); + } + } + if (message.category != null && message.hasOwnProperty("category")) + object.category = options.enums === String ? $root.query.StreamEvent.Statement.Category[message.category] === undefined ? message.category : $root.query.StreamEvent.Statement.Category[message.category] : message.category; + if (message.table_name != null && message.hasOwnProperty("table_name")) + object.table_name = message.table_name; + if (message.primary_key_fields && message.primary_key_fields.length) { + object.primary_key_fields = []; + for (let j = 0; j < message.primary_key_fields.length; ++j) + object.primary_key_fields[j] = $root.query.Field.toObject(message.primary_key_fields[j], options); + } + if (message.primary_key_values && message.primary_key_values.length) { + object.primary_key_values = []; + for (let j = 0; j < message.primary_key_values.length; ++j) + object.primary_key_values[j] = $root.query.Row.toObject(message.primary_key_values[j], options); + } + if (message.sql != null && message.hasOwnProperty("sql")) + object.sql = options.bytes === String ? $util.base64.encode(message.sql, 0, message.sql.length) : options.bytes === Array ? Array.prototype.slice.call(message.sql) : message.sql; + return object; + }; + + /** + * Converts this Statement to JSON. + * @function toJSON + * @memberof query.StreamEvent.Statement + * @instance + * @returns {Object.} JSON object + */ + Statement.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Statement + * @function getTypeUrl + * @memberof query.StreamEvent.Statement + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Statement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/query.StreamEvent.Statement"; + }; + + /** + * Category enum. + * @name query.StreamEvent.Statement.Category + * @enum {number} + * @property {number} Error=0 Error value + * @property {number} DML=1 DML value + * @property {number} DDL=2 DDL value + */ + Statement.Category = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "Error"] = 0; + values[valuesById[1] = "DML"] = 1; + values[valuesById[2] = "DDL"] = 2; + return values; + })(); + + return Statement; + })(); + + return StreamEvent; })(); - query.CommitRequest = (function() { + query.ExecuteRequest = (function() { /** - * Properties of a CommitRequest. + * Properties of an ExecuteRequest. * @memberof query - * @interface ICommitRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] CommitRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] CommitRequest immediate_caller_id - * @property {query.ITarget|null} [target] CommitRequest target - * @property {number|Long|null} [transaction_id] CommitRequest transaction_id + * @interface IExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] ExecuteRequest target + * @property {query.IBoundQuery|null} [query] ExecuteRequest query + * @property {number|Long|null} [transaction_id] ExecuteRequest transaction_id + * @property {query.IExecuteOptions|null} [options] ExecuteRequest options + * @property {number|Long|null} [reserved_id] ExecuteRequest reserved_id */ /** - * Constructs a new CommitRequest. + * Constructs a new ExecuteRequest. * @memberof query - * @classdesc Represents a CommitRequest. - * @implements ICommitRequest + * @classdesc Represents an ExecuteRequest. + * @implements IExecuteRequest * @constructor - * @param {query.ICommitRequest=} [properties] Properties to set + * @param {query.IExecuteRequest=} [properties] Properties to set */ - function CommitRequest(properties) { + function ExecuteRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -80185,59 +81069,83 @@ export const query = $root.query = (() => { } /** - * CommitRequest effective_caller_id. + * ExecuteRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @instance */ - CommitRequest.prototype.effective_caller_id = null; + ExecuteRequest.prototype.effective_caller_id = null; /** - * CommitRequest immediate_caller_id. + * ExecuteRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @instance */ - CommitRequest.prototype.immediate_caller_id = null; + ExecuteRequest.prototype.immediate_caller_id = null; /** - * CommitRequest target. + * ExecuteRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @instance */ - CommitRequest.prototype.target = null; + ExecuteRequest.prototype.target = null; /** - * CommitRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.CommitRequest + * ExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.ExecuteRequest * @instance */ - CommitRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ExecuteRequest.prototype.query = null; /** - * Creates a new CommitRequest instance using the specified properties. + * ExecuteRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.ExecuteRequest + * @instance + */ + ExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.ExecuteRequest + * @instance + */ + ExecuteRequest.prototype.options = null; + + /** + * ExecuteRequest reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ExecuteRequest + * @instance + */ + ExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Creates a new ExecuteRequest instance using the specified properties. * @function create - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static - * @param {query.ICommitRequest=} [properties] Properties to set - * @returns {query.CommitRequest} CommitRequest instance + * @param {query.IExecuteRequest=} [properties] Properties to set + * @returns {query.ExecuteRequest} ExecuteRequest instance */ - CommitRequest.create = function create(properties) { - return new CommitRequest(properties); + ExecuteRequest.create = function create(properties) { + return new ExecuteRequest(properties); }; /** - * Encodes the specified CommitRequest message. Does not implicitly {@link query.CommitRequest.verify|verify} messages. + * Encodes the specified ExecuteRequest message. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static - * @param {query.ICommitRequest} message CommitRequest message or plain object to encode + * @param {query.IExecuteRequest} message ExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitRequest.encode = function encode(message, writer) { + ExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -80246,39 +81154,45 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.transaction_id); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); return writer; }; /** - * Encodes the specified CommitRequest message, length delimited. Does not implicitly {@link query.CommitRequest.verify|verify} messages. + * Encodes the specified ExecuteRequest message, length delimited. Does not implicitly {@link query.ExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static - * @param {query.ICommitRequest} message CommitRequest message or plain object to encode + * @param {query.IExecuteRequest} message ExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitRequest.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CommitRequest message from the specified reader or buffer. + * Decodes an ExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.CommitRequest} CommitRequest + * @returns {query.ExecuteRequest} ExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitRequest.decode = function decode(reader, length) { + ExecuteRequest.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.query.CommitRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -80295,9 +81209,21 @@ export const query = $root.query = (() => { break; } case 4: { + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + break; + } + case 5: { message.transaction_id = reader.int64(); break; } + case 6: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 7: { + message.reserved_id = reader.int64(); + break; + } default: reader.skipType(tag & 7); break; @@ -80307,30 +81233,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a CommitRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CommitRequest} CommitRequest + * @returns {query.ExecuteRequest} ExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitRequest.decodeDelimited = function decodeDelimited(reader) { + ExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CommitRequest message. + * Verifies an ExecuteRequest message. * @function verify - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CommitRequest.verify = function verify(message) { + ExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -80348,39 +81274,57 @@ export const query = $root.query = (() => { if (error) return "target." + error; } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) return "transaction_id: integer|Long expected"; + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; return null; }; /** - * Creates a CommitRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.CommitRequest} CommitRequest + * @returns {query.ExecuteRequest} ExecuteRequest */ - CommitRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.CommitRequest) + ExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ExecuteRequest) return object; - let message = new $root.query.CommitRequest(); + let message = new $root.query.ExecuteRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.CommitRequest.effective_caller_id: object expected"); + throw TypeError(".query.ExecuteRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.CommitRequest.immediate_caller_id: object expected"); + throw TypeError(".query.ExecuteRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.CommitRequest.target: object expected"); + throw TypeError(".query.ExecuteRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.ExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } if (object.transaction_id != null) if ($util.Long) (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; @@ -80390,19 +81334,33 @@ export const query = $root.query = (() => { message.transaction_id = object.transaction_id; else if (typeof object.transaction_id === "object") message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.ExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a CommitRequest message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static - * @param {query.CommitRequest} message CommitRequest + * @param {query.ExecuteRequest} message ExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CommitRequest.toObject = function toObject(message, options) { + ExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -80410,11 +81368,18 @@ export const query = $root.query = (() => { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; + object.query = null; if ($util.Long) { let long = new $util.Long(0, 0, false); object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else object.transaction_id = options.longs === String ? "0" : 0; + object.options = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -80422,61 +81387,70 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) if (typeof message.transaction_id === "number") object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; return object; }; /** - * Converts this CommitRequest to JSON. + * Converts this ExecuteRequest to JSON. * @function toJSON - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @instance * @returns {Object.} JSON object */ - CommitRequest.prototype.toJSON = function toJSON() { + ExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CommitRequest + * Gets the default type url for ExecuteRequest * @function getTypeUrl - * @memberof query.CommitRequest + * @memberof query.ExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CommitRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.CommitRequest"; + return typeUrlPrefix + "/query.ExecuteRequest"; }; - return CommitRequest; + return ExecuteRequest; })(); - query.CommitResponse = (function() { + query.ExecuteResponse = (function() { /** - * Properties of a CommitResponse. + * Properties of an ExecuteResponse. * @memberof query - * @interface ICommitResponse - * @property {number|Long|null} [reserved_id] CommitResponse reserved_id + * @interface IExecuteResponse + * @property {query.IQueryResult|null} [result] ExecuteResponse result */ /** - * Constructs a new CommitResponse. + * Constructs a new ExecuteResponse. * @memberof query - * @classdesc Represents a CommitResponse. - * @implements ICommitResponse + * @classdesc Represents an ExecuteResponse. + * @implements IExecuteResponse * @constructor - * @param {query.ICommitResponse=} [properties] Properties to set + * @param {query.IExecuteResponse=} [properties] Properties to set */ - function CommitResponse(properties) { + function ExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -80484,75 +81458,75 @@ export const query = $root.query = (() => { } /** - * CommitResponse reserved_id. - * @member {number|Long} reserved_id - * @memberof query.CommitResponse + * ExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ExecuteResponse * @instance */ - CommitResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ExecuteResponse.prototype.result = null; /** - * Creates a new CommitResponse instance using the specified properties. + * Creates a new ExecuteResponse instance using the specified properties. * @function create - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static - * @param {query.ICommitResponse=} [properties] Properties to set - * @returns {query.CommitResponse} CommitResponse instance + * @param {query.IExecuteResponse=} [properties] Properties to set + * @returns {query.ExecuteResponse} ExecuteResponse instance */ - CommitResponse.create = function create(properties) { - return new CommitResponse(properties); + ExecuteResponse.create = function create(properties) { + return new ExecuteResponse(properties); }; /** - * Encodes the specified CommitResponse message. Does not implicitly {@link query.CommitResponse.verify|verify} messages. + * Encodes the specified ExecuteResponse message. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static - * @param {query.ICommitResponse} message CommitResponse message or plain object to encode + * @param {query.IExecuteResponse} message ExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitResponse.encode = function encode(message, writer) { + ExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.reserved_id); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified CommitResponse message, length delimited. Does not implicitly {@link query.CommitResponse.verify|verify} messages. + * Encodes the specified ExecuteResponse message, length delimited. Does not implicitly {@link query.ExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static - * @param {query.ICommitResponse} message CommitResponse message or plain object to encode + * @param {query.IExecuteResponse} message ExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitResponse.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CommitResponse message from the specified reader or buffer. + * Decodes an ExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.CommitResponse} CommitResponse + * @returns {query.ExecuteResponse} ExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitResponse.decode = function decode(reader, length) { + ExecuteResponse.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.query.CommitResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.reserved_id = reader.int64(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -80564,139 +81538,128 @@ export const query = $root.query = (() => { }; /** - * Decodes a CommitResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CommitResponse} CommitResponse + * @returns {query.ExecuteResponse} ExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitResponse.decodeDelimited = function decodeDelimited(reader) { + ExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CommitResponse message. + * Verifies an ExecuteResponse message. * @function verify - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CommitResponse.verify = function verify(message) { + ExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } return null; }; /** - * Creates a CommitResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.CommitResponse} CommitResponse + * @returns {query.ExecuteResponse} ExecuteResponse */ - CommitResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.CommitResponse) + ExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ExecuteResponse) return object; - let message = new $root.query.CommitResponse(); - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + let message = new $root.query.ExecuteResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } return message; }; /** - * Creates a plain object from a CommitResponse message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static - * @param {query.CommitResponse} message CommitResponse + * @param {query.ExecuteResponse} message ExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CommitResponse.toObject = function toObject(message, options) { + ExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.reserved_id = options.longs === String ? "0" : 0; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; - else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this CommitResponse to JSON. + * Converts this ExecuteResponse to JSON. * @function toJSON - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @instance * @returns {Object.} JSON object */ - CommitResponse.prototype.toJSON = function toJSON() { + ExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CommitResponse + * Gets the default type url for ExecuteResponse * @function getTypeUrl - * @memberof query.CommitResponse + * @memberof query.ExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CommitResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.CommitResponse"; + return typeUrlPrefix + "/query.ExecuteResponse"; }; - return CommitResponse; + return ExecuteResponse; })(); - query.RollbackRequest = (function() { + query.ResultWithError = (function() { /** - * Properties of a RollbackRequest. + * Properties of a ResultWithError. * @memberof query - * @interface IRollbackRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] RollbackRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] RollbackRequest immediate_caller_id - * @property {query.ITarget|null} [target] RollbackRequest target - * @property {number|Long|null} [transaction_id] RollbackRequest transaction_id + * @interface IResultWithError + * @property {vtrpc.IRPCError|null} [error] ResultWithError error + * @property {query.IQueryResult|null} [result] ResultWithError result */ /** - * Constructs a new RollbackRequest. + * Constructs a new ResultWithError. * @memberof query - * @classdesc Represents a RollbackRequest. - * @implements IRollbackRequest + * @classdesc Represents a ResultWithError. + * @implements IResultWithError * @constructor - * @param {query.IRollbackRequest=} [properties] Properties to set + * @param {query.IResultWithError=} [properties] Properties to set */ - function RollbackRequest(properties) { + function ResultWithError(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -80704,117 +81667,89 @@ export const query = $root.query = (() => { } /** - * RollbackRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.RollbackRequest - * @instance - */ - RollbackRequest.prototype.effective_caller_id = null; - - /** - * RollbackRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.RollbackRequest - * @instance - */ - RollbackRequest.prototype.immediate_caller_id = null; - - /** - * RollbackRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.RollbackRequest + * ResultWithError error. + * @member {vtrpc.IRPCError|null|undefined} error + * @memberof query.ResultWithError * @instance */ - RollbackRequest.prototype.target = null; + ResultWithError.prototype.error = null; /** - * RollbackRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.RollbackRequest + * ResultWithError result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ResultWithError * @instance */ - RollbackRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ResultWithError.prototype.result = null; /** - * Creates a new RollbackRequest instance using the specified properties. + * Creates a new ResultWithError instance using the specified properties. * @function create - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static - * @param {query.IRollbackRequest=} [properties] Properties to set - * @returns {query.RollbackRequest} RollbackRequest instance + * @param {query.IResultWithError=} [properties] Properties to set + * @returns {query.ResultWithError} ResultWithError instance */ - RollbackRequest.create = function create(properties) { - return new RollbackRequest(properties); + ResultWithError.create = function create(properties) { + return new ResultWithError(properties); }; /** - * Encodes the specified RollbackRequest message. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. + * Encodes the specified ResultWithError message. Does not implicitly {@link query.ResultWithError.verify|verify} messages. * @function encode - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static - * @param {query.IRollbackRequest} message RollbackRequest message or plain object to encode + * @param {query.IResultWithError} message ResultWithError message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackRequest.encode = function encode(message, writer) { + ResultWithError.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified RollbackRequest message, length delimited. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. + * Encodes the specified ResultWithError message, length delimited. Does not implicitly {@link query.ResultWithError.verify|verify} messages. * @function encodeDelimited - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static - * @param {query.IRollbackRequest} message RollbackRequest message or plain object to encode + * @param {query.IResultWithError} message ResultWithError message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackRequest.encodeDelimited = function encodeDelimited(message, writer) { + ResultWithError.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RollbackRequest message from the specified reader or buffer. + * Decodes a ResultWithError message from the specified reader or buffer. * @function decode - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.RollbackRequest} RollbackRequest + * @returns {query.ResultWithError} ResultWithError * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackRequest.decode = function decode(reader, length) { + ResultWithError.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.query.RollbackRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ResultWithError(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); break; } case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.transaction_id = reader.int64(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -80826,176 +81761,147 @@ export const query = $root.query = (() => { }; /** - * Decodes a RollbackRequest message from the specified reader or buffer, length delimited. + * Decodes a ResultWithError message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.RollbackRequest} RollbackRequest + * @returns {query.ResultWithError} ResultWithError * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackRequest.decodeDelimited = function decodeDelimited(reader) { + ResultWithError.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RollbackRequest message. + * Verifies a ResultWithError message. * @function verify - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RollbackRequest.verify = function verify(message) { + ResultWithError.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.vtrpc.RPCError.verify(message.error); if (error) - return "immediate_caller_id." + error; + return "error." + error; } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "target." + error; + return "result." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; return null; }; /** - * Creates a RollbackRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ResultWithError message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static * @param {Object.} object Plain object - * @returns {query.RollbackRequest} RollbackRequest + * @returns {query.ResultWithError} ResultWithError */ - RollbackRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.RollbackRequest) + ResultWithError.fromObject = function fromObject(object) { + if (object instanceof $root.query.ResultWithError) return object; - let message = new $root.query.RollbackRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.RollbackRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + let message = new $root.query.ResultWithError(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".query.ResultWithError.error: object expected"); + message.error = $root.vtrpc.RPCError.fromObject(object.error); } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.RollbackRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.RollbackRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ResultWithError.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a RollbackRequest message. Also converts values to other types if specified. + * Creates a plain object from a ResultWithError message. Also converts values to other types if specified. * @function toObject - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static - * @param {query.RollbackRequest} message RollbackRequest + * @param {query.ResultWithError} message ResultWithError * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RollbackRequest.toObject = function toObject(message, options) { + ResultWithError.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; + object.error = null; + object.result = null; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.vtrpc.RPCError.toObject(message.error, options); + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this RollbackRequest to JSON. + * Converts this ResultWithError to JSON. * @function toJSON - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @instance * @returns {Object.} JSON object */ - RollbackRequest.prototype.toJSON = function toJSON() { + ResultWithError.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RollbackRequest + * Gets the default type url for ResultWithError * @function getTypeUrl - * @memberof query.RollbackRequest + * @memberof query.ResultWithError * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RollbackRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ResultWithError.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.RollbackRequest"; + return typeUrlPrefix + "/query.ResultWithError"; }; - return RollbackRequest; + return ResultWithError; })(); - query.RollbackResponse = (function() { + query.StreamExecuteRequest = (function() { /** - * Properties of a RollbackResponse. + * Properties of a StreamExecuteRequest. * @memberof query - * @interface IRollbackResponse - * @property {number|Long|null} [reserved_id] RollbackResponse reserved_id + * @interface IStreamExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] StreamExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] StreamExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] StreamExecuteRequest target + * @property {query.IBoundQuery|null} [query] StreamExecuteRequest query + * @property {query.IExecuteOptions|null} [options] StreamExecuteRequest options + * @property {number|Long|null} [transaction_id] StreamExecuteRequest transaction_id + * @property {number|Long|null} [reserved_id] StreamExecuteRequest reserved_id */ /** - * Constructs a new RollbackResponse. + * Constructs a new StreamExecuteRequest. * @memberof query - * @classdesc Represents a RollbackResponse. - * @implements IRollbackResponse + * @classdesc Represents a StreamExecuteRequest. + * @implements IStreamExecuteRequest * @constructor - * @param {query.IRollbackResponse=} [properties] Properties to set + * @param {query.IStreamExecuteRequest=} [properties] Properties to set */ - function RollbackResponse(properties) { + function StreamExecuteRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -81003,74 +81909,158 @@ export const query = $root.query = (() => { } /** - * RollbackResponse reserved_id. + * StreamExecuteRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.effective_caller_id = null; + + /** + * StreamExecuteRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.immediate_caller_id = null; + + /** + * StreamExecuteRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.target = null; + + /** + * StreamExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.query = null; + + /** + * StreamExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.options = null; + + /** + * StreamExecuteRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.StreamExecuteRequest + * @instance + */ + StreamExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * StreamExecuteRequest reserved_id. * @member {number|Long} reserved_id - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @instance */ - RollbackResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + StreamExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new RollbackResponse instance using the specified properties. + * Creates a new StreamExecuteRequest instance using the specified properties. * @function create - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static - * @param {query.IRollbackResponse=} [properties] Properties to set - * @returns {query.RollbackResponse} RollbackResponse instance + * @param {query.IStreamExecuteRequest=} [properties] Properties to set + * @returns {query.StreamExecuteRequest} StreamExecuteRequest instance */ - RollbackResponse.create = function create(properties) { - return new RollbackResponse(properties); + StreamExecuteRequest.create = function create(properties) { + return new StreamExecuteRequest(properties); }; /** - * Encodes the specified RollbackResponse message. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. + * Encodes the specified StreamExecuteRequest message. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static - * @param {query.IRollbackResponse} message RollbackResponse message or plain object to encode + * @param {query.IStreamExecuteRequest} message StreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackResponse.encode = function encode(message, writer) { + StreamExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.transaction_id); if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.reserved_id); + writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); return writer; }; /** - * Encodes the specified RollbackResponse message, length delimited. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. + * Encodes the specified StreamExecuteRequest message, length delimited. Does not implicitly {@link query.StreamExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static - * @param {query.IRollbackResponse} message RollbackResponse message or plain object to encode + * @param {query.IStreamExecuteRequest} message StreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackResponse.encodeDelimited = function encodeDelimited(message, writer) { + StreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RollbackResponse message from the specified reader or buffer. + * Decodes a StreamExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.RollbackResponse} RollbackResponse + * @returns {query.StreamExecuteRequest} StreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackResponse.decode = function decode(reader, length) { + StreamExecuteRequest.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.query.RollbackResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StreamExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } + case 2: { + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + case 4: { + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + break; + } + case 5: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 6: { + message.transaction_id = reader.int64(); + break; + } + case 7: { message.reserved_id = reader.int64(); break; } @@ -81083,32 +82073,60 @@ export const query = $root.query = (() => { }; /** - * Decodes a RollbackResponse message from the specified reader or buffer, length delimited. + * Decodes a StreamExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.RollbackResponse} RollbackResponse + * @returns {query.StreamExecuteRequest} StreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackResponse.decodeDelimited = function decodeDelimited(reader) { + StreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RollbackResponse message. + * Verifies a StreamExecuteRequest message. * @function verify - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RollbackResponse.verify = function verify(message) { + StreamExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) return "reserved_id: integer|Long expected"; @@ -81116,17 +82134,51 @@ export const query = $root.query = (() => { }; /** - * Creates a RollbackResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.RollbackResponse} RollbackResponse + * @returns {query.StreamExecuteRequest} StreamExecuteRequest */ - RollbackResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.RollbackResponse) + StreamExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamExecuteRequest) return object; - let message = new $root.query.RollbackResponse(); + let message = new $root.query.StreamExecuteRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.StreamExecuteRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.StreamExecuteRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.StreamExecuteRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.StreamExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.StreamExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.reserved_id != null) if ($util.Long) (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; @@ -81140,24 +82192,50 @@ export const query = $root.query = (() => { }; /** - * Creates a plain object from a RollbackResponse message. Also converts values to other types if specified. + * Creates a plain object from a StreamExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static - * @param {query.RollbackResponse} message RollbackResponse + * @param {query.StreamExecuteRequest} message StreamExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RollbackResponse.toObject = function toObject(message, options) { + StreamExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) + if (options.defaults) { + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = null; + object.options = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; if ($util.Long) { let long = new $util.Long(0, 0, false); object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else object.reserved_id = options.longs === String ? "0" : 0; + } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) if (typeof message.reserved_id === "number") object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; @@ -81167,56 +82245,52 @@ export const query = $root.query = (() => { }; /** - * Converts this RollbackResponse to JSON. + * Converts this StreamExecuteRequest to JSON. * @function toJSON - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @instance * @returns {Object.} JSON object */ - RollbackResponse.prototype.toJSON = function toJSON() { + StreamExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RollbackResponse + * Gets the default type url for StreamExecuteRequest * @function getTypeUrl - * @memberof query.RollbackResponse + * @memberof query.StreamExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RollbackResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.RollbackResponse"; + return typeUrlPrefix + "/query.StreamExecuteRequest"; }; - return RollbackResponse; + return StreamExecuteRequest; })(); - query.PrepareRequest = (function() { + query.StreamExecuteResponse = (function() { /** - * Properties of a PrepareRequest. + * Properties of a StreamExecuteResponse. * @memberof query - * @interface IPrepareRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] PrepareRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] PrepareRequest immediate_caller_id - * @property {query.ITarget|null} [target] PrepareRequest target - * @property {number|Long|null} [transaction_id] PrepareRequest transaction_id - * @property {string|null} [dtid] PrepareRequest dtid + * @interface IStreamExecuteResponse + * @property {query.IQueryResult|null} [result] StreamExecuteResponse result */ /** - * Constructs a new PrepareRequest. + * Constructs a new StreamExecuteResponse. * @memberof query - * @classdesc Represents a PrepareRequest. - * @implements IPrepareRequest + * @classdesc Represents a StreamExecuteResponse. + * @implements IStreamExecuteResponse * @constructor - * @param {query.IPrepareRequest=} [properties] Properties to set + * @param {query.IStreamExecuteResponse=} [properties] Properties to set */ - function PrepareRequest(properties) { + function StreamExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -81224,131 +82298,75 @@ export const query = $root.query = (() => { } /** - * PrepareRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.PrepareRequest - * @instance - */ - PrepareRequest.prototype.effective_caller_id = null; - - /** - * PrepareRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.PrepareRequest - * @instance - */ - PrepareRequest.prototype.immediate_caller_id = null; - - /** - * PrepareRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.PrepareRequest - * @instance - */ - PrepareRequest.prototype.target = null; - - /** - * PrepareRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.PrepareRequest - * @instance - */ - PrepareRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * PrepareRequest dtid. - * @member {string} dtid - * @memberof query.PrepareRequest + * StreamExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.StreamExecuteResponse * @instance */ - PrepareRequest.prototype.dtid = ""; + StreamExecuteResponse.prototype.result = null; /** - * Creates a new PrepareRequest instance using the specified properties. + * Creates a new StreamExecuteResponse instance using the specified properties. * @function create - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static - * @param {query.IPrepareRequest=} [properties] Properties to set - * @returns {query.PrepareRequest} PrepareRequest instance + * @param {query.IStreamExecuteResponse=} [properties] Properties to set + * @returns {query.StreamExecuteResponse} StreamExecuteResponse instance */ - PrepareRequest.create = function create(properties) { - return new PrepareRequest(properties); + StreamExecuteResponse.create = function create(properties) { + return new StreamExecuteResponse(properties); }; /** - * Encodes the specified PrepareRequest message. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. + * Encodes the specified StreamExecuteResponse message. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static - * @param {query.IPrepareRequest} message PrepareRequest message or plain object to encode + * @param {query.IStreamExecuteResponse} message StreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrepareRequest.encode = function encode(message, writer) { + StreamExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); - if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified PrepareRequest message, length delimited. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. + * Encodes the specified StreamExecuteResponse message, length delimited. Does not implicitly {@link query.StreamExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static - * @param {query.IPrepareRequest} message PrepareRequest message or plain object to encode + * @param {query.IStreamExecuteResponse} message StreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrepareRequest.encodeDelimited = function encodeDelimited(message, writer) { + StreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PrepareRequest message from the specified reader or buffer. + * Decodes a StreamExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.PrepareRequest} PrepareRequest + * @returns {query.StreamExecuteResponse} StreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrepareRequest.decode = function decode(reader, length) { + StreamExecuteResponse.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.query.PrepareRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StreamExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); - break; - } - case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.transaction_id = reader.int64(); - break; - } - case 5: { - message.dtid = reader.string(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -81360,183 +82378,130 @@ export const query = $root.query = (() => { }; /** - * Decodes a PrepareRequest message from the specified reader or buffer, length delimited. + * Decodes a StreamExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.PrepareRequest} PrepareRequest + * @returns {query.StreamExecuteResponse} StreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrepareRequest.decodeDelimited = function decodeDelimited(reader) { + StreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PrepareRequest message. + * Verifies a StreamExecuteResponse message. * @function verify - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PrepareRequest.verify = function verify(message) { + StreamExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); - if (error) - return "target." + error; + return "result." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.dtid != null && message.hasOwnProperty("dtid")) - if (!$util.isString(message.dtid)) - return "dtid: string expected"; return null; }; /** - * Creates a PrepareRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.PrepareRequest} PrepareRequest + * @returns {query.StreamExecuteResponse} StreamExecuteResponse */ - PrepareRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.PrepareRequest) + StreamExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamExecuteResponse) return object; - let message = new $root.query.PrepareRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.PrepareRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); - } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.PrepareRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.PrepareRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + let message = new $root.query.StreamExecuteResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.StreamExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.dtid != null) - message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a PrepareRequest message. Also converts values to other types if specified. + * Creates a plain object from a StreamExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static - * @param {query.PrepareRequest} message PrepareRequest + * @param {query.StreamExecuteResponse} message StreamExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PrepareRequest.toObject = function toObject(message, options) { + StreamExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - object.dtid = ""; - } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.dtid != null && message.hasOwnProperty("dtid")) - object.dtid = message.dtid; + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this PrepareRequest to JSON. + * Converts this StreamExecuteResponse to JSON. * @function toJSON - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @instance * @returns {Object.} JSON object */ - PrepareRequest.prototype.toJSON = function toJSON() { + StreamExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PrepareRequest + * Gets the default type url for StreamExecuteResponse * @function getTypeUrl - * @memberof query.PrepareRequest + * @memberof query.StreamExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PrepareRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.PrepareRequest"; + return typeUrlPrefix + "/query.StreamExecuteResponse"; }; - return PrepareRequest; + return StreamExecuteResponse; })(); - query.PrepareResponse = (function() { + query.BeginRequest = (function() { /** - * Properties of a PrepareResponse. + * Properties of a BeginRequest. * @memberof query - * @interface IPrepareResponse + * @interface IBeginRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginRequest immediate_caller_id + * @property {query.ITarget|null} [target] BeginRequest target + * @property {query.IExecuteOptions|null} [options] BeginRequest options */ /** - * Constructs a new PrepareResponse. + * Constructs a new BeginRequest. * @memberof query - * @classdesc Represents a PrepareResponse. - * @implements IPrepareResponse + * @classdesc Represents a BeginRequest. + * @implements IBeginRequest * @constructor - * @param {query.IPrepareResponse=} [properties] Properties to set + * @param {query.IBeginRequest=} [properties] Properties to set */ - function PrepareResponse(properties) { + function BeginRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -81544,63 +82509,119 @@ export const query = $root.query = (() => { } /** - * Creates a new PrepareResponse instance using the specified properties. + * BeginRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.BeginRequest + * @instance + */ + BeginRequest.prototype.effective_caller_id = null; + + /** + * BeginRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.BeginRequest + * @instance + */ + BeginRequest.prototype.immediate_caller_id = null; + + /** + * BeginRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.BeginRequest + * @instance + */ + BeginRequest.prototype.target = null; + + /** + * BeginRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.BeginRequest + * @instance + */ + BeginRequest.prototype.options = null; + + /** + * Creates a new BeginRequest instance using the specified properties. * @function create - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static - * @param {query.IPrepareResponse=} [properties] Properties to set - * @returns {query.PrepareResponse} PrepareResponse instance + * @param {query.IBeginRequest=} [properties] Properties to set + * @returns {query.BeginRequest} BeginRequest instance */ - PrepareResponse.create = function create(properties) { - return new PrepareResponse(properties); + BeginRequest.create = function create(properties) { + return new BeginRequest(properties); }; /** - * Encodes the specified PrepareResponse message. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. + * Encodes the specified BeginRequest message. Does not implicitly {@link query.BeginRequest.verify|verify} messages. * @function encode - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static - * @param {query.IPrepareResponse} message PrepareResponse message or plain object to encode + * @param {query.IBeginRequest} message BeginRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrepareResponse.encode = function encode(message, writer) { + BeginRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified PrepareResponse message, length delimited. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. + * Encodes the specified BeginRequest message, length delimited. Does not implicitly {@link query.BeginRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static - * @param {query.IPrepareResponse} message PrepareResponse message or plain object to encode + * @param {query.IBeginRequest} message BeginRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrepareResponse.encodeDelimited = function encodeDelimited(message, writer) { + BeginRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PrepareResponse message from the specified reader or buffer. + * Decodes a BeginRequest message from the specified reader or buffer. * @function decode - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.PrepareResponse} PrepareResponse + * @returns {query.BeginRequest} BeginRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrepareResponse.decode = function decode(reader, length) { + BeginRequest.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.query.PrepareResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } + case 2: { + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + case 4: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -81610,112 +82631,169 @@ export const query = $root.query = (() => { }; /** - * Decodes a PrepareResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.PrepareResponse} PrepareResponse + * @returns {query.BeginRequest} BeginRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrepareResponse.decodeDelimited = function decodeDelimited(reader) { + BeginRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PrepareResponse message. + * Verifies a BeginRequest message. * @function verify - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PrepareResponse.verify = function verify(message) { + BeginRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } return null; }; /** - * Creates a PrepareResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static * @param {Object.} object Plain object - * @returns {query.PrepareResponse} PrepareResponse + * @returns {query.BeginRequest} BeginRequest */ - PrepareResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.PrepareResponse) + BeginRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginRequest) return object; - return new $root.query.PrepareResponse(); + let message = new $root.query.BeginRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.BeginRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.BeginRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.BeginRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.BeginRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + return message; }; /** - * Creates a plain object from a PrepareResponse message. Also converts values to other types if specified. + * Creates a plain object from a BeginRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static - * @param {query.PrepareResponse} message PrepareResponse + * @param {query.BeginRequest} message BeginRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PrepareResponse.toObject = function toObject() { - return {}; + BeginRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.options = null; + } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + return object; }; /** - * Converts this PrepareResponse to JSON. + * Converts this BeginRequest to JSON. * @function toJSON - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @instance * @returns {Object.} JSON object */ - PrepareResponse.prototype.toJSON = function toJSON() { + BeginRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PrepareResponse + * Gets the default type url for BeginRequest * @function getTypeUrl - * @memberof query.PrepareResponse + * @memberof query.BeginRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PrepareResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.PrepareResponse"; + return typeUrlPrefix + "/query.BeginRequest"; }; - return PrepareResponse; + return BeginRequest; })(); - query.CommitPreparedRequest = (function() { + query.BeginResponse = (function() { /** - * Properties of a CommitPreparedRequest. + * Properties of a BeginResponse. * @memberof query - * @interface ICommitPreparedRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] CommitPreparedRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] CommitPreparedRequest immediate_caller_id - * @property {query.ITarget|null} [target] CommitPreparedRequest target - * @property {string|null} [dtid] CommitPreparedRequest dtid + * @interface IBeginResponse + * @property {number|Long|null} [transaction_id] BeginResponse transaction_id + * @property {topodata.ITabletAlias|null} [tablet_alias] BeginResponse tablet_alias + * @property {string|null} [session_state_changes] BeginResponse session_state_changes */ /** - * Constructs a new CommitPreparedRequest. + * Constructs a new BeginResponse. * @memberof query - * @classdesc Represents a CommitPreparedRequest. - * @implements ICommitPreparedRequest + * @classdesc Represents a BeginResponse. + * @implements IBeginResponse * @constructor - * @param {query.ICommitPreparedRequest=} [properties] Properties to set + * @param {query.IBeginResponse=} [properties] Properties to set */ - function CommitPreparedRequest(properties) { + function BeginResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -81723,117 +82801,103 @@ export const query = $root.query = (() => { } /** - * CommitPreparedRequest effective_caller_id. - * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.CommitPreparedRequest - * @instance - */ - CommitPreparedRequest.prototype.effective_caller_id = null; - - /** - * CommitPreparedRequest immediate_caller_id. - * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.CommitPreparedRequest + * BeginResponse transaction_id. + * @member {number|Long} transaction_id + * @memberof query.BeginResponse * @instance */ - CommitPreparedRequest.prototype.immediate_caller_id = null; + BeginResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * CommitPreparedRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.CommitPreparedRequest + * BeginResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof query.BeginResponse * @instance */ - CommitPreparedRequest.prototype.target = null; + BeginResponse.prototype.tablet_alias = null; /** - * CommitPreparedRequest dtid. - * @member {string} dtid - * @memberof query.CommitPreparedRequest + * BeginResponse session_state_changes. + * @member {string} session_state_changes + * @memberof query.BeginResponse * @instance */ - CommitPreparedRequest.prototype.dtid = ""; + BeginResponse.prototype.session_state_changes = ""; /** - * Creates a new CommitPreparedRequest instance using the specified properties. + * Creates a new BeginResponse instance using the specified properties. * @function create - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static - * @param {query.ICommitPreparedRequest=} [properties] Properties to set - * @returns {query.CommitPreparedRequest} CommitPreparedRequest instance + * @param {query.IBeginResponse=} [properties] Properties to set + * @returns {query.BeginResponse} BeginResponse instance */ - CommitPreparedRequest.create = function create(properties) { - return new CommitPreparedRequest(properties); + BeginResponse.create = function create(properties) { + return new BeginResponse(properties); }; /** - * Encodes the specified CommitPreparedRequest message. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. + * Encodes the specified BeginResponse message. Does not implicitly {@link query.BeginResponse.verify|verify} messages. * @function encode - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static - * @param {query.ICommitPreparedRequest} message CommitPreparedRequest message or plain object to encode + * @param {query.IBeginResponse} message BeginResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitPreparedRequest.encode = function encode(message, writer) { + BeginResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) - $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) - $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.transaction_id); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.session_state_changes); return writer; }; /** - * Encodes the specified CommitPreparedRequest message, length delimited. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. + * Encodes the specified BeginResponse message, length delimited. Does not implicitly {@link query.BeginResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static - * @param {query.ICommitPreparedRequest} message CommitPreparedRequest message or plain object to encode + * @param {query.IBeginResponse} message BeginResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CommitPreparedRequest.encodeDelimited = function encodeDelimited(message, writer) { + BeginResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CommitPreparedRequest message from the specified reader or buffer. + * Decodes a BeginResponse message from the specified reader or buffer. * @function decode - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.CommitPreparedRequest} CommitPreparedRequest + * @returns {query.BeginResponse} BeginResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitPreparedRequest.decode = function decode(reader, length) { + BeginResponse.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.query.CommitPreparedRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + message.transaction_id = reader.int64(); break; } case 2: { - message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 3: { - message.target = $root.query.Target.decode(reader, reader.uint32()); - break; - } - case 4: { - message.dtid = reader.string(); + message.session_state_changes = reader.string(); break; } default: @@ -81845,341 +82909,161 @@ export const query = $root.query = (() => { }; /** - * Decodes a CommitPreparedRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CommitPreparedRequest} CommitPreparedRequest + * @returns {query.BeginResponse} BeginResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CommitPreparedRequest.decodeDelimited = function decodeDelimited(reader) { + BeginResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CommitPreparedRequest message. + * Verifies a BeginResponse message. * @function verify - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CommitPreparedRequest.verify = function verify(message) { + BeginResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); - if (error) - return "effective_caller_id." + error; - } - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { - let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); - if (error) - return "immediate_caller_id." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) - return "target." + error; + return "tablet_alias." + error; } - if (message.dtid != null && message.hasOwnProperty("dtid")) - if (!$util.isString(message.dtid)) - return "dtid: string expected"; + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a CommitPreparedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static * @param {Object.} object Plain object - * @returns {query.CommitPreparedRequest} CommitPreparedRequest + * @returns {query.BeginResponse} BeginResponse */ - CommitPreparedRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.CommitPreparedRequest) + BeginResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginResponse) return object; - let message = new $root.query.CommitPreparedRequest(); - if (object.effective_caller_id != null) { - if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.CommitPreparedRequest.effective_caller_id: object expected"); - message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); - } - if (object.immediate_caller_id != null) { - if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.CommitPreparedRequest.immediate_caller_id: object expected"); - message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.CommitPreparedRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + let message = new $root.query.BeginResponse(); + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".query.BeginResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.dtid != null) - message.dtid = String(object.dtid); + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a CommitPreparedRequest message. Also converts values to other types if specified. + * Creates a plain object from a BeginResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.CommitPreparedRequest + * @memberof query.BeginResponse * @static - * @param {query.CommitPreparedRequest} message CommitPreparedRequest + * @param {query.BeginResponse} message BeginResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CommitPreparedRequest.toObject = function toObject(message, options) { + BeginResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.effective_caller_id = null; - object.immediate_caller_id = null; - object.target = null; - object.dtid = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; + object.tablet_alias = null; + object.session_state_changes = ""; } - if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) - object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); - if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) - object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.dtid != null && message.hasOwnProperty("dtid")) - object.dtid = message.dtid; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this CommitPreparedRequest to JSON. - * @function toJSON - * @memberof query.CommitPreparedRequest - * @instance - * @returns {Object.} JSON object - */ - CommitPreparedRequest.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for CommitPreparedRequest - * @function getTypeUrl - * @memberof query.CommitPreparedRequest - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - CommitPreparedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/query.CommitPreparedRequest"; - }; - - return CommitPreparedRequest; - })(); - - query.CommitPreparedResponse = (function() { - - /** - * Properties of a CommitPreparedResponse. - * @memberof query - * @interface ICommitPreparedResponse - */ - - /** - * Constructs a new CommitPreparedResponse. - * @memberof query - * @classdesc Represents a CommitPreparedResponse. - * @implements ICommitPreparedResponse - * @constructor - * @param {query.ICommitPreparedResponse=} [properties] Properties to set - */ - function CommitPreparedResponse(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 CommitPreparedResponse instance using the specified properties. - * @function create - * @memberof query.CommitPreparedResponse - * @static - * @param {query.ICommitPreparedResponse=} [properties] Properties to set - * @returns {query.CommitPreparedResponse} CommitPreparedResponse instance - */ - CommitPreparedResponse.create = function create(properties) { - return new CommitPreparedResponse(properties); - }; - - /** - * Encodes the specified CommitPreparedResponse message. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. - * @function encode - * @memberof query.CommitPreparedResponse - * @static - * @param {query.ICommitPreparedResponse} message CommitPreparedResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - CommitPreparedResponse.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - return writer; - }; - - /** - * Encodes the specified CommitPreparedResponse message, length delimited. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. - * @function encodeDelimited - * @memberof query.CommitPreparedResponse - * @static - * @param {query.ICommitPreparedResponse} message CommitPreparedResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - CommitPreparedResponse.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a CommitPreparedResponse message from the specified reader or buffer. - * @function decode - * @memberof query.CommitPreparedResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {query.CommitPreparedResponse} CommitPreparedResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - CommitPreparedResponse.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.query.CommitPreparedResponse(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a CommitPreparedResponse message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof query.CommitPreparedResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CommitPreparedResponse} CommitPreparedResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - CommitPreparedResponse.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a CommitPreparedResponse message. - * @function verify - * @memberof query.CommitPreparedResponse - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - CommitPreparedResponse.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - return null; - }; - - /** - * Creates a CommitPreparedResponse message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof query.CommitPreparedResponse - * @static - * @param {Object.} object Plain object - * @returns {query.CommitPreparedResponse} CommitPreparedResponse - */ - CommitPreparedResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.CommitPreparedResponse) - return object; - return new $root.query.CommitPreparedResponse(); - }; - - /** - * Creates a plain object from a CommitPreparedResponse message. Also converts values to other types if specified. - * @function toObject - * @memberof query.CommitPreparedResponse - * @static - * @param {query.CommitPreparedResponse} message CommitPreparedResponse - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - CommitPreparedResponse.toObject = function toObject() { - return {}; - }; - - /** - * Converts this CommitPreparedResponse to JSON. + * Converts this BeginResponse to JSON. * @function toJSON - * @memberof query.CommitPreparedResponse + * @memberof query.BeginResponse * @instance * @returns {Object.} JSON object */ - CommitPreparedResponse.prototype.toJSON = function toJSON() { + BeginResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CommitPreparedResponse + * Gets the default type url for BeginResponse * @function getTypeUrl - * @memberof query.CommitPreparedResponse + * @memberof query.BeginResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CommitPreparedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.CommitPreparedResponse"; + return typeUrlPrefix + "/query.BeginResponse"; }; - return CommitPreparedResponse; + return BeginResponse; })(); - query.RollbackPreparedRequest = (function() { + query.CommitRequest = (function() { /** - * Properties of a RollbackPreparedRequest. + * Properties of a CommitRequest. * @memberof query - * @interface IRollbackPreparedRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] RollbackPreparedRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] RollbackPreparedRequest immediate_caller_id - * @property {query.ITarget|null} [target] RollbackPreparedRequest target - * @property {number|Long|null} [transaction_id] RollbackPreparedRequest transaction_id - * @property {string|null} [dtid] RollbackPreparedRequest dtid + * @interface ICommitRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] CommitRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] CommitRequest immediate_caller_id + * @property {query.ITarget|null} [target] CommitRequest target + * @property {number|Long|null} [transaction_id] CommitRequest transaction_id */ /** - * Constructs a new RollbackPreparedRequest. + * Constructs a new CommitRequest. * @memberof query - * @classdesc Represents a RollbackPreparedRequest. - * @implements IRollbackPreparedRequest + * @classdesc Represents a CommitRequest. + * @implements ICommitRequest * @constructor - * @param {query.IRollbackPreparedRequest=} [properties] Properties to set + * @param {query.ICommitRequest=} [properties] Properties to set */ - function RollbackPreparedRequest(properties) { + function CommitRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -82187,67 +83071,59 @@ export const query = $root.query = (() => { } /** - * RollbackPreparedRequest effective_caller_id. + * CommitRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @instance */ - RollbackPreparedRequest.prototype.effective_caller_id = null; + CommitRequest.prototype.effective_caller_id = null; /** - * RollbackPreparedRequest immediate_caller_id. + * CommitRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @instance */ - RollbackPreparedRequest.prototype.immediate_caller_id = null; + CommitRequest.prototype.immediate_caller_id = null; /** - * RollbackPreparedRequest target. + * CommitRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @instance */ - RollbackPreparedRequest.prototype.target = null; + CommitRequest.prototype.target = null; /** - * RollbackPreparedRequest transaction_id. + * CommitRequest transaction_id. * @member {number|Long} transaction_id - * @memberof query.RollbackPreparedRequest - * @instance - */ - RollbackPreparedRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * RollbackPreparedRequest dtid. - * @member {string} dtid - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @instance */ - RollbackPreparedRequest.prototype.dtid = ""; + CommitRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new RollbackPreparedRequest instance using the specified properties. + * Creates a new CommitRequest instance using the specified properties. * @function create - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static - * @param {query.IRollbackPreparedRequest=} [properties] Properties to set - * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest instance + * @param {query.ICommitRequest=} [properties] Properties to set + * @returns {query.CommitRequest} CommitRequest instance */ - RollbackPreparedRequest.create = function create(properties) { - return new RollbackPreparedRequest(properties); + CommitRequest.create = function create(properties) { + return new CommitRequest(properties); }; /** - * Encodes the specified RollbackPreparedRequest message. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. + * Encodes the specified CommitRequest message. Does not implicitly {@link query.CommitRequest.verify|verify} messages. * @function encode - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static - * @param {query.IRollbackPreparedRequest} message RollbackPreparedRequest message or plain object to encode + * @param {query.ICommitRequest} message CommitRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackPreparedRequest.encode = function encode(message, writer) { + CommitRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -82258,39 +83134,37 @@ export const query = $root.query = (() => { $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); - if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); return writer; }; /** - * Encodes the specified RollbackPreparedRequest message, length delimited. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. + * Encodes the specified CommitRequest message, length delimited. Does not implicitly {@link query.CommitRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static - * @param {query.IRollbackPreparedRequest} message RollbackPreparedRequest message or plain object to encode + * @param {query.ICommitRequest} message CommitRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackPreparedRequest.encodeDelimited = function encodeDelimited(message, writer) { + CommitRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RollbackPreparedRequest message from the specified reader or buffer. + * Decodes a CommitRequest message from the specified reader or buffer. * @function decode - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest + * @returns {query.CommitRequest} CommitRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackPreparedRequest.decode = function decode(reader, length) { + CommitRequest.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.query.RollbackPreparedRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CommitRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -82310,10 +83184,6 @@ export const query = $root.query = (() => { message.transaction_id = reader.int64(); break; } - case 5: { - message.dtid = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -82323,30 +83193,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a RollbackPreparedRequest message from the specified reader or buffer, length delimited. + * Decodes a CommitRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest + * @returns {query.CommitRequest} CommitRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackPreparedRequest.decodeDelimited = function decodeDelimited(reader) { + CommitRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RollbackPreparedRequest message. + * Verifies a CommitRequest message. * @function verify - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RollbackPreparedRequest.verify = function verify(message) { + CommitRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -82367,37 +83237,34 @@ export const query = $root.query = (() => { if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) return "transaction_id: integer|Long expected"; - if (message.dtid != null && message.hasOwnProperty("dtid")) - if (!$util.isString(message.dtid)) - return "dtid: string expected"; return null; }; /** - * Creates a RollbackPreparedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CommitRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static * @param {Object.} object Plain object - * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest + * @returns {query.CommitRequest} CommitRequest */ - RollbackPreparedRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.RollbackPreparedRequest) + CommitRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.CommitRequest) return object; - let message = new $root.query.RollbackPreparedRequest(); + let message = new $root.query.CommitRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.RollbackPreparedRequest.effective_caller_id: object expected"); + throw TypeError(".query.CommitRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.RollbackPreparedRequest.immediate_caller_id: object expected"); + throw TypeError(".query.CommitRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.RollbackPreparedRequest.target: object expected"); + throw TypeError(".query.CommitRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } if (object.transaction_id != null) @@ -82409,21 +83276,19 @@ export const query = $root.query = (() => { message.transaction_id = object.transaction_id; else if (typeof object.transaction_id === "object") message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.dtid != null) - message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a RollbackPreparedRequest message. Also converts values to other types if specified. + * Creates a plain object from a CommitRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static - * @param {query.RollbackPreparedRequest} message RollbackPreparedRequest + * @param {query.CommitRequest} message CommitRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RollbackPreparedRequest.toObject = function toObject(message, options) { + CommitRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -82436,7 +83301,6 @@ export const query = $root.query = (() => { object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else object.transaction_id = options.longs === String ? "0" : 0; - object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -82449,57 +83313,56 @@ export const query = $root.query = (() => { object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.dtid != null && message.hasOwnProperty("dtid")) - object.dtid = message.dtid; return object; }; /** - * Converts this RollbackPreparedRequest to JSON. + * Converts this CommitRequest to JSON. * @function toJSON - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @instance * @returns {Object.} JSON object */ - RollbackPreparedRequest.prototype.toJSON = function toJSON() { + CommitRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RollbackPreparedRequest + * Gets the default type url for CommitRequest * @function getTypeUrl - * @memberof query.RollbackPreparedRequest + * @memberof query.CommitRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RollbackPreparedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CommitRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.RollbackPreparedRequest"; + return typeUrlPrefix + "/query.CommitRequest"; }; - return RollbackPreparedRequest; + return CommitRequest; })(); - query.RollbackPreparedResponse = (function() { + query.CommitResponse = (function() { /** - * Properties of a RollbackPreparedResponse. + * Properties of a CommitResponse. * @memberof query - * @interface IRollbackPreparedResponse + * @interface ICommitResponse + * @property {number|Long|null} [reserved_id] CommitResponse reserved_id */ /** - * Constructs a new RollbackPreparedResponse. + * Constructs a new CommitResponse. * @memberof query - * @classdesc Represents a RollbackPreparedResponse. - * @implements IRollbackPreparedResponse + * @classdesc Represents a CommitResponse. + * @implements ICommitResponse * @constructor - * @param {query.IRollbackPreparedResponse=} [properties] Properties to set + * @param {query.ICommitResponse=} [properties] Properties to set */ - function RollbackPreparedResponse(properties) { + function CommitResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -82507,63 +83370,77 @@ export const query = $root.query = (() => { } /** - * Creates a new RollbackPreparedResponse instance using the specified properties. + * CommitResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.CommitResponse + * @instance + */ + CommitResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Creates a new CommitResponse instance using the specified properties. * @function create - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static - * @param {query.IRollbackPreparedResponse=} [properties] Properties to set - * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse instance + * @param {query.ICommitResponse=} [properties] Properties to set + * @returns {query.CommitResponse} CommitResponse instance */ - RollbackPreparedResponse.create = function create(properties) { - return new RollbackPreparedResponse(properties); + CommitResponse.create = function create(properties) { + return new CommitResponse(properties); }; /** - * Encodes the specified RollbackPreparedResponse message. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. + * Encodes the specified CommitResponse message. Does not implicitly {@link query.CommitResponse.verify|verify} messages. * @function encode - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static - * @param {query.IRollbackPreparedResponse} message RollbackPreparedResponse message or plain object to encode + * @param {query.ICommitResponse} message CommitResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackPreparedResponse.encode = function encode(message, writer) { + CommitResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.reserved_id); return writer; }; /** - * Encodes the specified RollbackPreparedResponse message, length delimited. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. + * Encodes the specified CommitResponse message, length delimited. Does not implicitly {@link query.CommitResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static - * @param {query.IRollbackPreparedResponse} message RollbackPreparedResponse message or plain object to encode + * @param {query.ICommitResponse} message CommitResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RollbackPreparedResponse.encodeDelimited = function encodeDelimited(message, writer) { + CommitResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RollbackPreparedResponse message from the specified reader or buffer. + * Decodes a CommitResponse message from the specified reader or buffer. * @function decode - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse + * @returns {query.CommitResponse} CommitResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackPreparedResponse.decode = function decode(reader, length) { + CommitResponse.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.query.RollbackPreparedResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CommitResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.reserved_id = reader.int64(); + break; + } default: reader.skipType(tag & 7); break; @@ -82573,114 +83450,139 @@ export const query = $root.query = (() => { }; /** - * Decodes a RollbackPreparedResponse message from the specified reader or buffer, length delimited. + * Decodes a CommitResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse + * @returns {query.CommitResponse} CommitResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RollbackPreparedResponse.decodeDelimited = function decodeDelimited(reader) { + CommitResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RollbackPreparedResponse message. + * Verifies a CommitResponse message. * @function verify - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RollbackPreparedResponse.verify = function verify(message) { + CommitResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; return null; }; /** - * Creates a RollbackPreparedResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CommitResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static * @param {Object.} object Plain object - * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse + * @returns {query.CommitResponse} CommitResponse */ - RollbackPreparedResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.RollbackPreparedResponse) + CommitResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.CommitResponse) return object; - return new $root.query.RollbackPreparedResponse(); - }; + let message = new $root.query.CommitResponse(); + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + return message; + }; /** - * Creates a plain object from a RollbackPreparedResponse message. Also converts values to other types if specified. + * Creates a plain object from a CommitResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static - * @param {query.RollbackPreparedResponse} message RollbackPreparedResponse + * @param {query.CommitResponse} message CommitResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RollbackPreparedResponse.toObject = function toObject() { - return {}; + CommitResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + return object; }; /** - * Converts this RollbackPreparedResponse to JSON. + * Converts this CommitResponse to JSON. * @function toJSON - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @instance * @returns {Object.} JSON object */ - RollbackPreparedResponse.prototype.toJSON = function toJSON() { + CommitResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RollbackPreparedResponse + * Gets the default type url for CommitResponse * @function getTypeUrl - * @memberof query.RollbackPreparedResponse + * @memberof query.CommitResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RollbackPreparedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CommitResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.RollbackPreparedResponse"; + return typeUrlPrefix + "/query.CommitResponse"; }; - return RollbackPreparedResponse; + return CommitResponse; })(); - query.CreateTransactionRequest = (function() { + query.RollbackRequest = (function() { /** - * Properties of a CreateTransactionRequest. + * Properties of a RollbackRequest. * @memberof query - * @interface ICreateTransactionRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] CreateTransactionRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] CreateTransactionRequest immediate_caller_id - * @property {query.ITarget|null} [target] CreateTransactionRequest target - * @property {string|null} [dtid] CreateTransactionRequest dtid - * @property {Array.|null} [participants] CreateTransactionRequest participants + * @interface IRollbackRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] RollbackRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] RollbackRequest immediate_caller_id + * @property {query.ITarget|null} [target] RollbackRequest target + * @property {number|Long|null} [transaction_id] RollbackRequest transaction_id */ /** - * Constructs a new CreateTransactionRequest. + * Constructs a new RollbackRequest. * @memberof query - * @classdesc Represents a CreateTransactionRequest. - * @implements ICreateTransactionRequest + * @classdesc Represents a RollbackRequest. + * @implements IRollbackRequest * @constructor - * @param {query.ICreateTransactionRequest=} [properties] Properties to set + * @param {query.IRollbackRequest=} [properties] Properties to set */ - function CreateTransactionRequest(properties) { - this.participants = []; + function RollbackRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -82688,67 +83590,59 @@ export const query = $root.query = (() => { } /** - * CreateTransactionRequest effective_caller_id. + * RollbackRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @instance */ - CreateTransactionRequest.prototype.effective_caller_id = null; + RollbackRequest.prototype.effective_caller_id = null; /** - * CreateTransactionRequest immediate_caller_id. + * RollbackRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @instance */ - CreateTransactionRequest.prototype.immediate_caller_id = null; + RollbackRequest.prototype.immediate_caller_id = null; /** - * CreateTransactionRequest target. + * RollbackRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.CreateTransactionRequest - * @instance - */ - CreateTransactionRequest.prototype.target = null; - - /** - * CreateTransactionRequest dtid. - * @member {string} dtid - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @instance */ - CreateTransactionRequest.prototype.dtid = ""; + RollbackRequest.prototype.target = null; /** - * CreateTransactionRequest participants. - * @member {Array.} participants - * @memberof query.CreateTransactionRequest + * RollbackRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.RollbackRequest * @instance */ - CreateTransactionRequest.prototype.participants = $util.emptyArray; + RollbackRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new CreateTransactionRequest instance using the specified properties. + * Creates a new RollbackRequest instance using the specified properties. * @function create - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static - * @param {query.ICreateTransactionRequest=} [properties] Properties to set - * @returns {query.CreateTransactionRequest} CreateTransactionRequest instance + * @param {query.IRollbackRequest=} [properties] Properties to set + * @returns {query.RollbackRequest} RollbackRequest instance */ - CreateTransactionRequest.create = function create(properties) { - return new CreateTransactionRequest(properties); + RollbackRequest.create = function create(properties) { + return new RollbackRequest(properties); }; /** - * Encodes the specified CreateTransactionRequest message. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. + * Encodes the specified RollbackRequest message. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. * @function encode - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static - * @param {query.ICreateTransactionRequest} message CreateTransactionRequest message or plain object to encode + * @param {query.IRollbackRequest} message RollbackRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateTransactionRequest.encode = function encode(message, writer) { + RollbackRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -82757,42 +83651,39 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); - if (message.participants != null && message.participants.length) - for (let i = 0; i < message.participants.length; ++i) - $root.query.Target.encode(message.participants[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); return writer; }; /** - * Encodes the specified CreateTransactionRequest message, length delimited. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. + * Encodes the specified RollbackRequest message, length delimited. Does not implicitly {@link query.RollbackRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static - * @param {query.ICreateTransactionRequest} message CreateTransactionRequest message or plain object to encode + * @param {query.IRollbackRequest} message RollbackRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { + RollbackRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateTransactionRequest message from the specified reader or buffer. + * Decodes a RollbackRequest message from the specified reader or buffer. * @function decode - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.CreateTransactionRequest} CreateTransactionRequest + * @returns {query.RollbackRequest} RollbackRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateTransactionRequest.decode = function decode(reader, length) { + RollbackRequest.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.query.CreateTransactionRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.RollbackRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -82809,13 +83700,7 @@ export const query = $root.query = (() => { break; } case 4: { - message.dtid = reader.string(); - break; - } - case 5: { - if (!(message.participants && message.participants.length)) - message.participants = []; - message.participants.push($root.query.Target.decode(reader, reader.uint32())); + message.transaction_id = reader.int64(); break; } default: @@ -82827,30 +83712,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a CreateTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a RollbackRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CreateTransactionRequest} CreateTransactionRequest + * @returns {query.RollbackRequest} RollbackRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateTransactionRequest.decodeDelimited = function decodeDelimited(reader) { + RollbackRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateTransactionRequest message. + * Verifies a RollbackRequest message. * @function verify - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateTransactionRequest.verify = function verify(message) { + RollbackRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -82868,83 +83753,73 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.dtid != null && message.hasOwnProperty("dtid")) - if (!$util.isString(message.dtid)) - return "dtid: string expected"; - if (message.participants != null && message.hasOwnProperty("participants")) { - if (!Array.isArray(message.participants)) - return "participants: array expected"; - for (let i = 0; i < message.participants.length; ++i) { - let error = $root.query.Target.verify(message.participants[i]); - if (error) - return "participants." + error; - } - } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; return null; }; /** - * Creates a CreateTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static * @param {Object.} object Plain object - * @returns {query.CreateTransactionRequest} CreateTransactionRequest + * @returns {query.RollbackRequest} RollbackRequest */ - CreateTransactionRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.CreateTransactionRequest) + RollbackRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.RollbackRequest) return object; - let message = new $root.query.CreateTransactionRequest(); + let message = new $root.query.RollbackRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.CreateTransactionRequest.effective_caller_id: object expected"); + throw TypeError(".query.RollbackRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.CreateTransactionRequest.immediate_caller_id: object expected"); + throw TypeError(".query.RollbackRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.CreateTransactionRequest.target: object expected"); + throw TypeError(".query.RollbackRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.dtid != null) - message.dtid = String(object.dtid); - if (object.participants) { - if (!Array.isArray(object.participants)) - throw TypeError(".query.CreateTransactionRequest.participants: array expected"); - message.participants = []; - for (let i = 0; i < object.participants.length; ++i) { - if (typeof object.participants[i] !== "object") - throw TypeError(".query.CreateTransactionRequest.participants: object expected"); - message.participants[i] = $root.query.Target.fromObject(object.participants[i]); - } - } + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a CreateTransactionRequest message. Also converts values to other types if specified. + * Creates a plain object from a RollbackRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static - * @param {query.CreateTransactionRequest} message CreateTransactionRequest + * @param {query.RollbackRequest} message RollbackRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CreateTransactionRequest.toObject = function toObject(message, options) { + RollbackRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.participants = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.dtid = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -82952,62 +83827,61 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.dtid != null && message.hasOwnProperty("dtid")) - object.dtid = message.dtid; - if (message.participants && message.participants.length) { - object.participants = []; - for (let j = 0; j < message.participants.length; ++j) - object.participants[j] = $root.query.Target.toObject(message.participants[j], options); - } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; return object; }; /** - * Converts this CreateTransactionRequest to JSON. + * Converts this RollbackRequest to JSON. * @function toJSON - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @instance * @returns {Object.} JSON object */ - CreateTransactionRequest.prototype.toJSON = function toJSON() { + RollbackRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateTransactionRequest + * Gets the default type url for RollbackRequest * @function getTypeUrl - * @memberof query.CreateTransactionRequest + * @memberof query.RollbackRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RollbackRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.CreateTransactionRequest"; + return typeUrlPrefix + "/query.RollbackRequest"; }; - return CreateTransactionRequest; + return RollbackRequest; })(); - query.CreateTransactionResponse = (function() { + query.RollbackResponse = (function() { /** - * Properties of a CreateTransactionResponse. + * Properties of a RollbackResponse. * @memberof query - * @interface ICreateTransactionResponse + * @interface IRollbackResponse + * @property {number|Long|null} [reserved_id] RollbackResponse reserved_id */ /** - * Constructs a new CreateTransactionResponse. + * Constructs a new RollbackResponse. * @memberof query - * @classdesc Represents a CreateTransactionResponse. - * @implements ICreateTransactionResponse + * @classdesc Represents a RollbackResponse. + * @implements IRollbackResponse * @constructor - * @param {query.ICreateTransactionResponse=} [properties] Properties to set + * @param {query.IRollbackResponse=} [properties] Properties to set */ - function CreateTransactionResponse(properties) { + function RollbackResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -83015,63 +83889,77 @@ export const query = $root.query = (() => { } /** - * Creates a new CreateTransactionResponse instance using the specified properties. + * RollbackResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.RollbackResponse + * @instance + */ + RollbackResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Creates a new RollbackResponse instance using the specified properties. * @function create - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static - * @param {query.ICreateTransactionResponse=} [properties] Properties to set - * @returns {query.CreateTransactionResponse} CreateTransactionResponse instance + * @param {query.IRollbackResponse=} [properties] Properties to set + * @returns {query.RollbackResponse} RollbackResponse instance */ - CreateTransactionResponse.create = function create(properties) { - return new CreateTransactionResponse(properties); + RollbackResponse.create = function create(properties) { + return new RollbackResponse(properties); }; /** - * Encodes the specified CreateTransactionResponse message. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. + * Encodes the specified RollbackResponse message. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. * @function encode - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static - * @param {query.ICreateTransactionResponse} message CreateTransactionResponse message or plain object to encode + * @param {query.IRollbackResponse} message RollbackResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateTransactionResponse.encode = function encode(message, writer) { + RollbackResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.reserved_id); return writer; }; /** - * Encodes the specified CreateTransactionResponse message, length delimited. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. + * Encodes the specified RollbackResponse message, length delimited. Does not implicitly {@link query.RollbackResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static - * @param {query.ICreateTransactionResponse} message CreateTransactionResponse message or plain object to encode + * @param {query.IRollbackResponse} message RollbackResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { + RollbackResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateTransactionResponse message from the specified reader or buffer. + * Decodes a RollbackResponse message from the specified reader or buffer. * @function decode - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.CreateTransactionResponse} CreateTransactionResponse + * @returns {query.RollbackResponse} RollbackResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateTransactionResponse.decode = function decode(reader, length) { + RollbackResponse.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.query.CreateTransactionResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.RollbackResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.reserved_id = reader.int64(); + break; + } default: reader.skipType(tag & 7); break; @@ -83081,113 +83969,140 @@ export const query = $root.query = (() => { }; /** - * Decodes a CreateTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a RollbackResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.CreateTransactionResponse} CreateTransactionResponse + * @returns {query.RollbackResponse} RollbackResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateTransactionResponse.decodeDelimited = function decodeDelimited(reader) { + RollbackResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateTransactionResponse message. + * Verifies a RollbackResponse message. * @function verify - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateTransactionResponse.verify = function verify(message) { + RollbackResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; return null; }; /** - * Creates a CreateTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static * @param {Object.} object Plain object - * @returns {query.CreateTransactionResponse} CreateTransactionResponse + * @returns {query.RollbackResponse} RollbackResponse */ - CreateTransactionResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.CreateTransactionResponse) + RollbackResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.RollbackResponse) return object; - return new $root.query.CreateTransactionResponse(); + let message = new $root.query.RollbackResponse(); + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + return message; }; /** - * Creates a plain object from a CreateTransactionResponse message. Also converts values to other types if specified. + * Creates a plain object from a RollbackResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static - * @param {query.CreateTransactionResponse} message CreateTransactionResponse + * @param {query.RollbackResponse} message RollbackResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CreateTransactionResponse.toObject = function toObject() { - return {}; + RollbackResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + return object; }; /** - * Converts this CreateTransactionResponse to JSON. + * Converts this RollbackResponse to JSON. * @function toJSON - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @instance * @returns {Object.} JSON object */ - CreateTransactionResponse.prototype.toJSON = function toJSON() { + RollbackResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateTransactionResponse + * Gets the default type url for RollbackResponse * @function getTypeUrl - * @memberof query.CreateTransactionResponse + * @memberof query.RollbackResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RollbackResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.CreateTransactionResponse"; + return typeUrlPrefix + "/query.RollbackResponse"; }; - return CreateTransactionResponse; + return RollbackResponse; })(); - query.StartCommitRequest = (function() { + query.PrepareRequest = (function() { /** - * Properties of a StartCommitRequest. + * Properties of a PrepareRequest. * @memberof query - * @interface IStartCommitRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] StartCommitRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] StartCommitRequest immediate_caller_id - * @property {query.ITarget|null} [target] StartCommitRequest target - * @property {number|Long|null} [transaction_id] StartCommitRequest transaction_id - * @property {string|null} [dtid] StartCommitRequest dtid + * @interface IPrepareRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] PrepareRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] PrepareRequest immediate_caller_id + * @property {query.ITarget|null} [target] PrepareRequest target + * @property {number|Long|null} [transaction_id] PrepareRequest transaction_id + * @property {string|null} [dtid] PrepareRequest dtid */ /** - * Constructs a new StartCommitRequest. + * Constructs a new PrepareRequest. * @memberof query - * @classdesc Represents a StartCommitRequest. - * @implements IStartCommitRequest + * @classdesc Represents a PrepareRequest. + * @implements IPrepareRequest * @constructor - * @param {query.IStartCommitRequest=} [properties] Properties to set + * @param {query.IPrepareRequest=} [properties] Properties to set */ - function StartCommitRequest(properties) { + function PrepareRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -83195,67 +84110,67 @@ export const query = $root.query = (() => { } /** - * StartCommitRequest effective_caller_id. + * PrepareRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance */ - StartCommitRequest.prototype.effective_caller_id = null; + PrepareRequest.prototype.effective_caller_id = null; /** - * StartCommitRequest immediate_caller_id. + * PrepareRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance */ - StartCommitRequest.prototype.immediate_caller_id = null; + PrepareRequest.prototype.immediate_caller_id = null; /** - * StartCommitRequest target. + * PrepareRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance */ - StartCommitRequest.prototype.target = null; + PrepareRequest.prototype.target = null; /** - * StartCommitRequest transaction_id. + * PrepareRequest transaction_id. * @member {number|Long} transaction_id - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance */ - StartCommitRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + PrepareRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * StartCommitRequest dtid. + * PrepareRequest dtid. * @member {string} dtid - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance */ - StartCommitRequest.prototype.dtid = ""; + PrepareRequest.prototype.dtid = ""; /** - * Creates a new StartCommitRequest instance using the specified properties. + * Creates a new PrepareRequest instance using the specified properties. * @function create - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static - * @param {query.IStartCommitRequest=} [properties] Properties to set - * @returns {query.StartCommitRequest} StartCommitRequest instance + * @param {query.IPrepareRequest=} [properties] Properties to set + * @returns {query.PrepareRequest} PrepareRequest instance */ - StartCommitRequest.create = function create(properties) { - return new StartCommitRequest(properties); + PrepareRequest.create = function create(properties) { + return new PrepareRequest(properties); }; /** - * Encodes the specified StartCommitRequest message. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. + * Encodes the specified PrepareRequest message. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. * @function encode - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static - * @param {query.IStartCommitRequest} message StartCommitRequest message or plain object to encode + * @param {query.IPrepareRequest} message PrepareRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartCommitRequest.encode = function encode(message, writer) { + PrepareRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -83272,33 +84187,33 @@ export const query = $root.query = (() => { }; /** - * Encodes the specified StartCommitRequest message, length delimited. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. + * Encodes the specified PrepareRequest message, length delimited. Does not implicitly {@link query.PrepareRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static - * @param {query.IStartCommitRequest} message StartCommitRequest message or plain object to encode + * @param {query.IPrepareRequest} message PrepareRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartCommitRequest.encodeDelimited = function encodeDelimited(message, writer) { + PrepareRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StartCommitRequest message from the specified reader or buffer. + * Decodes a PrepareRequest message from the specified reader or buffer. * @function decode - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StartCommitRequest} StartCommitRequest + * @returns {query.PrepareRequest} PrepareRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartCommitRequest.decode = function decode(reader, length) { + PrepareRequest.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.query.StartCommitRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.PrepareRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -83331,30 +84246,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a StartCommitRequest message from the specified reader or buffer, length delimited. + * Decodes a PrepareRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StartCommitRequest} StartCommitRequest + * @returns {query.PrepareRequest} PrepareRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartCommitRequest.decodeDelimited = function decodeDelimited(reader) { + PrepareRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StartCommitRequest message. + * Verifies a PrepareRequest message. * @function verify - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StartCommitRequest.verify = function verify(message) { + PrepareRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -83382,30 +84297,30 @@ export const query = $root.query = (() => { }; /** - * Creates a StartCommitRequest message from a plain object. Also converts values to their respective internal types. + * Creates a PrepareRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static * @param {Object.} object Plain object - * @returns {query.StartCommitRequest} StartCommitRequest + * @returns {query.PrepareRequest} PrepareRequest */ - StartCommitRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.StartCommitRequest) + PrepareRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.PrepareRequest) return object; - let message = new $root.query.StartCommitRequest(); + let message = new $root.query.PrepareRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.StartCommitRequest.effective_caller_id: object expected"); + throw TypeError(".query.PrepareRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.StartCommitRequest.immediate_caller_id: object expected"); + throw TypeError(".query.PrepareRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.StartCommitRequest.target: object expected"); + throw TypeError(".query.PrepareRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } if (object.transaction_id != null) @@ -83423,15 +84338,15 @@ export const query = $root.query = (() => { }; /** - * Creates a plain object from a StartCommitRequest message. Also converts values to other types if specified. + * Creates a plain object from a PrepareRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static - * @param {query.StartCommitRequest} message StartCommitRequest + * @param {query.PrepareRequest} message PrepareRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StartCommitRequest.toObject = function toObject(message, options) { + PrepareRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -83463,51 +84378,51 @@ export const query = $root.query = (() => { }; /** - * Converts this StartCommitRequest to JSON. + * Converts this PrepareRequest to JSON. * @function toJSON - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @instance * @returns {Object.} JSON object */ - StartCommitRequest.prototype.toJSON = function toJSON() { + PrepareRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StartCommitRequest + * Gets the default type url for PrepareRequest * @function getTypeUrl - * @memberof query.StartCommitRequest + * @memberof query.PrepareRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StartCommitRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PrepareRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StartCommitRequest"; + return typeUrlPrefix + "/query.PrepareRequest"; }; - return StartCommitRequest; + return PrepareRequest; })(); - query.StartCommitResponse = (function() { + query.PrepareResponse = (function() { /** - * Properties of a StartCommitResponse. + * Properties of a PrepareResponse. * @memberof query - * @interface IStartCommitResponse + * @interface IPrepareResponse */ /** - * Constructs a new StartCommitResponse. + * Constructs a new PrepareResponse. * @memberof query - * @classdesc Represents a StartCommitResponse. - * @implements IStartCommitResponse + * @classdesc Represents a PrepareResponse. + * @implements IPrepareResponse * @constructor - * @param {query.IStartCommitResponse=} [properties] Properties to set + * @param {query.IPrepareResponse=} [properties] Properties to set */ - function StartCommitResponse(properties) { + function PrepareResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -83515,60 +84430,60 @@ export const query = $root.query = (() => { } /** - * Creates a new StartCommitResponse instance using the specified properties. + * Creates a new PrepareResponse instance using the specified properties. * @function create - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static - * @param {query.IStartCommitResponse=} [properties] Properties to set - * @returns {query.StartCommitResponse} StartCommitResponse instance + * @param {query.IPrepareResponse=} [properties] Properties to set + * @returns {query.PrepareResponse} PrepareResponse instance */ - StartCommitResponse.create = function create(properties) { - return new StartCommitResponse(properties); + PrepareResponse.create = function create(properties) { + return new PrepareResponse(properties); }; /** - * Encodes the specified StartCommitResponse message. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. + * Encodes the specified PrepareResponse message. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. * @function encode - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static - * @param {query.IStartCommitResponse} message StartCommitResponse message or plain object to encode + * @param {query.IPrepareResponse} message PrepareResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartCommitResponse.encode = function encode(message, writer) { + PrepareResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified StartCommitResponse message, length delimited. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. + * Encodes the specified PrepareResponse message, length delimited. Does not implicitly {@link query.PrepareResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static - * @param {query.IStartCommitResponse} message StartCommitResponse message or plain object to encode + * @param {query.IPrepareResponse} message PrepareResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartCommitResponse.encodeDelimited = function encodeDelimited(message, writer) { + PrepareResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StartCommitResponse message from the specified reader or buffer. + * Decodes a PrepareResponse message from the specified reader or buffer. * @function decode - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StartCommitResponse} StartCommitResponse + * @returns {query.PrepareResponse} PrepareResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartCommitResponse.decode = function decode(reader, length) { + PrepareResponse.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.query.StartCommitResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.PrepareResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -83581,113 +84496,112 @@ export const query = $root.query = (() => { }; /** - * Decodes a StartCommitResponse message from the specified reader or buffer, length delimited. + * Decodes a PrepareResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StartCommitResponse} StartCommitResponse + * @returns {query.PrepareResponse} PrepareResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartCommitResponse.decodeDelimited = function decodeDelimited(reader) { + PrepareResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StartCommitResponse message. + * Verifies a PrepareResponse message. * @function verify - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StartCommitResponse.verify = function verify(message) { + PrepareResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a StartCommitResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PrepareResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static * @param {Object.} object Plain object - * @returns {query.StartCommitResponse} StartCommitResponse + * @returns {query.PrepareResponse} PrepareResponse */ - StartCommitResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.StartCommitResponse) + PrepareResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.PrepareResponse) return object; - return new $root.query.StartCommitResponse(); + return new $root.query.PrepareResponse(); }; /** - * Creates a plain object from a StartCommitResponse message. Also converts values to other types if specified. + * Creates a plain object from a PrepareResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static - * @param {query.StartCommitResponse} message StartCommitResponse + * @param {query.PrepareResponse} message PrepareResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StartCommitResponse.toObject = function toObject() { + PrepareResponse.toObject = function toObject() { return {}; }; /** - * Converts this StartCommitResponse to JSON. + * Converts this PrepareResponse to JSON. * @function toJSON - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @instance * @returns {Object.} JSON object */ - StartCommitResponse.prototype.toJSON = function toJSON() { + PrepareResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StartCommitResponse + * Gets the default type url for PrepareResponse * @function getTypeUrl - * @memberof query.StartCommitResponse + * @memberof query.PrepareResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StartCommitResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PrepareResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StartCommitResponse"; + return typeUrlPrefix + "/query.PrepareResponse"; }; - return StartCommitResponse; + return PrepareResponse; })(); - query.SetRollbackRequest = (function() { + query.CommitPreparedRequest = (function() { /** - * Properties of a SetRollbackRequest. + * Properties of a CommitPreparedRequest. * @memberof query - * @interface ISetRollbackRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] SetRollbackRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] SetRollbackRequest immediate_caller_id - * @property {query.ITarget|null} [target] SetRollbackRequest target - * @property {number|Long|null} [transaction_id] SetRollbackRequest transaction_id - * @property {string|null} [dtid] SetRollbackRequest dtid + * @interface ICommitPreparedRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] CommitPreparedRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] CommitPreparedRequest immediate_caller_id + * @property {query.ITarget|null} [target] CommitPreparedRequest target + * @property {string|null} [dtid] CommitPreparedRequest dtid */ /** - * Constructs a new SetRollbackRequest. + * Constructs a new CommitPreparedRequest. * @memberof query - * @classdesc Represents a SetRollbackRequest. - * @implements ISetRollbackRequest + * @classdesc Represents a CommitPreparedRequest. + * @implements ICommitPreparedRequest * @constructor - * @param {query.ISetRollbackRequest=} [properties] Properties to set + * @param {query.ICommitPreparedRequest=} [properties] Properties to set */ - function SetRollbackRequest(properties) { + function CommitPreparedRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -83695,67 +84609,59 @@ export const query = $root.query = (() => { } /** - * SetRollbackRequest effective_caller_id. + * CommitPreparedRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @instance */ - SetRollbackRequest.prototype.effective_caller_id = null; + CommitPreparedRequest.prototype.effective_caller_id = null; /** - * SetRollbackRequest immediate_caller_id. + * CommitPreparedRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @instance */ - SetRollbackRequest.prototype.immediate_caller_id = null; + CommitPreparedRequest.prototype.immediate_caller_id = null; /** - * SetRollbackRequest target. + * CommitPreparedRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.SetRollbackRequest - * @instance - */ - SetRollbackRequest.prototype.target = null; - - /** - * SetRollbackRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @instance */ - SetRollbackRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + CommitPreparedRequest.prototype.target = null; /** - * SetRollbackRequest dtid. + * CommitPreparedRequest dtid. * @member {string} dtid - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @instance */ - SetRollbackRequest.prototype.dtid = ""; + CommitPreparedRequest.prototype.dtid = ""; /** - * Creates a new SetRollbackRequest instance using the specified properties. + * Creates a new CommitPreparedRequest instance using the specified properties. * @function create - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static - * @param {query.ISetRollbackRequest=} [properties] Properties to set - * @returns {query.SetRollbackRequest} SetRollbackRequest instance + * @param {query.ICommitPreparedRequest=} [properties] Properties to set + * @returns {query.CommitPreparedRequest} CommitPreparedRequest instance */ - SetRollbackRequest.create = function create(properties) { - return new SetRollbackRequest(properties); + CommitPreparedRequest.create = function create(properties) { + return new CommitPreparedRequest(properties); }; /** - * Encodes the specified SetRollbackRequest message. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. + * Encodes the specified CommitPreparedRequest message. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. * @function encode - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static - * @param {query.ISetRollbackRequest} message SetRollbackRequest message or plain object to encode + * @param {query.ICommitPreparedRequest} message CommitPreparedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetRollbackRequest.encode = function encode(message, writer) { + CommitPreparedRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -83764,41 +84670,39 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); + writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); return writer; }; /** - * Encodes the specified SetRollbackRequest message, length delimited. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. + * Encodes the specified CommitPreparedRequest message, length delimited. Does not implicitly {@link query.CommitPreparedRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static - * @param {query.ISetRollbackRequest} message SetRollbackRequest message or plain object to encode + * @param {query.ICommitPreparedRequest} message CommitPreparedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetRollbackRequest.encodeDelimited = function encodeDelimited(message, writer) { + CommitPreparedRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetRollbackRequest message from the specified reader or buffer. + * Decodes a CommitPreparedRequest message from the specified reader or buffer. * @function decode - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.SetRollbackRequest} SetRollbackRequest + * @returns {query.CommitPreparedRequest} CommitPreparedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetRollbackRequest.decode = function decode(reader, length) { + CommitPreparedRequest.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.query.SetRollbackRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CommitPreparedRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -83815,10 +84719,6 @@ export const query = $root.query = (() => { break; } case 4: { - message.transaction_id = reader.int64(); - break; - } - case 5: { message.dtid = reader.string(); break; } @@ -83831,30 +84731,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a SetRollbackRequest message from the specified reader or buffer, length delimited. + * Decodes a CommitPreparedRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.SetRollbackRequest} SetRollbackRequest + * @returns {query.CommitPreparedRequest} CommitPreparedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetRollbackRequest.decodeDelimited = function decodeDelimited(reader) { + CommitPreparedRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetRollbackRequest message. + * Verifies a CommitPreparedRequest message. * @function verify - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetRollbackRequest.verify = function verify(message) { + CommitPreparedRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -83872,9 +84772,6 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; if (message.dtid != null && message.hasOwnProperty("dtid")) if (!$util.isString(message.dtid)) return "dtid: string expected"; @@ -83882,56 +84779,47 @@ export const query = $root.query = (() => { }; /** - * Creates a SetRollbackRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CommitPreparedRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static * @param {Object.} object Plain object - * @returns {query.SetRollbackRequest} SetRollbackRequest + * @returns {query.CommitPreparedRequest} CommitPreparedRequest */ - SetRollbackRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.SetRollbackRequest) + CommitPreparedRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.CommitPreparedRequest) return object; - let message = new $root.query.SetRollbackRequest(); + let message = new $root.query.CommitPreparedRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.SetRollbackRequest.effective_caller_id: object expected"); + throw TypeError(".query.CommitPreparedRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.SetRollbackRequest.immediate_caller_id: object expected"); + throw TypeError(".query.CommitPreparedRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.SetRollbackRequest.target: object expected"); + throw TypeError(".query.CommitPreparedRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.dtid != null) message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a SetRollbackRequest message. Also converts values to other types if specified. + * Creates a plain object from a CommitPreparedRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static - * @param {query.SetRollbackRequest} message SetRollbackRequest + * @param {query.CommitPreparedRequest} message CommitPreparedRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetRollbackRequest.toObject = function toObject(message, options) { + CommitPreparedRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -83939,11 +84827,6 @@ export const query = $root.query = (() => { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) @@ -83952,62 +84835,57 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.dtid != null && message.hasOwnProperty("dtid")) object.dtid = message.dtid; return object; }; /** - * Converts this SetRollbackRequest to JSON. + * Converts this CommitPreparedRequest to JSON. * @function toJSON - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @instance * @returns {Object.} JSON object */ - SetRollbackRequest.prototype.toJSON = function toJSON() { + CommitPreparedRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetRollbackRequest + * Gets the default type url for CommitPreparedRequest * @function getTypeUrl - * @memberof query.SetRollbackRequest + * @memberof query.CommitPreparedRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetRollbackRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CommitPreparedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.SetRollbackRequest"; + return typeUrlPrefix + "/query.CommitPreparedRequest"; }; - return SetRollbackRequest; + return CommitPreparedRequest; })(); - query.SetRollbackResponse = (function() { + query.CommitPreparedResponse = (function() { /** - * Properties of a SetRollbackResponse. + * Properties of a CommitPreparedResponse. * @memberof query - * @interface ISetRollbackResponse + * @interface ICommitPreparedResponse */ /** - * Constructs a new SetRollbackResponse. + * Constructs a new CommitPreparedResponse. * @memberof query - * @classdesc Represents a SetRollbackResponse. - * @implements ISetRollbackResponse + * @classdesc Represents a CommitPreparedResponse. + * @implements ICommitPreparedResponse * @constructor - * @param {query.ISetRollbackResponse=} [properties] Properties to set + * @param {query.ICommitPreparedResponse=} [properties] Properties to set */ - function SetRollbackResponse(properties) { + function CommitPreparedResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -84015,60 +84893,60 @@ export const query = $root.query = (() => { } /** - * Creates a new SetRollbackResponse instance using the specified properties. + * Creates a new CommitPreparedResponse instance using the specified properties. * @function create - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static - * @param {query.ISetRollbackResponse=} [properties] Properties to set - * @returns {query.SetRollbackResponse} SetRollbackResponse instance + * @param {query.ICommitPreparedResponse=} [properties] Properties to set + * @returns {query.CommitPreparedResponse} CommitPreparedResponse instance */ - SetRollbackResponse.create = function create(properties) { - return new SetRollbackResponse(properties); + CommitPreparedResponse.create = function create(properties) { + return new CommitPreparedResponse(properties); }; /** - * Encodes the specified SetRollbackResponse message. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. + * Encodes the specified CommitPreparedResponse message. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. * @function encode - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static - * @param {query.ISetRollbackResponse} message SetRollbackResponse message or plain object to encode + * @param {query.ICommitPreparedResponse} message CommitPreparedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetRollbackResponse.encode = function encode(message, writer) { + CommitPreparedResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified SetRollbackResponse message, length delimited. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. + * Encodes the specified CommitPreparedResponse message, length delimited. Does not implicitly {@link query.CommitPreparedResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static - * @param {query.ISetRollbackResponse} message SetRollbackResponse message or plain object to encode + * @param {query.ICommitPreparedResponse} message CommitPreparedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetRollbackResponse.encodeDelimited = function encodeDelimited(message, writer) { + CommitPreparedResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetRollbackResponse message from the specified reader or buffer. + * Decodes a CommitPreparedResponse message from the specified reader or buffer. * @function decode - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.SetRollbackResponse} SetRollbackResponse + * @returns {query.CommitPreparedResponse} CommitPreparedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetRollbackResponse.decode = function decode(reader, length) { + CommitPreparedResponse.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.query.SetRollbackResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CommitPreparedResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -84081,112 +84959,113 @@ export const query = $root.query = (() => { }; /** - * Decodes a SetRollbackResponse message from the specified reader or buffer, length delimited. + * Decodes a CommitPreparedResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.SetRollbackResponse} SetRollbackResponse + * @returns {query.CommitPreparedResponse} CommitPreparedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetRollbackResponse.decodeDelimited = function decodeDelimited(reader) { + CommitPreparedResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetRollbackResponse message. + * Verifies a CommitPreparedResponse message. * @function verify - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetRollbackResponse.verify = function verify(message) { + CommitPreparedResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a SetRollbackResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CommitPreparedResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static * @param {Object.} object Plain object - * @returns {query.SetRollbackResponse} SetRollbackResponse + * @returns {query.CommitPreparedResponse} CommitPreparedResponse */ - SetRollbackResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.SetRollbackResponse) + CommitPreparedResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.CommitPreparedResponse) return object; - return new $root.query.SetRollbackResponse(); + return new $root.query.CommitPreparedResponse(); }; /** - * Creates a plain object from a SetRollbackResponse message. Also converts values to other types if specified. + * Creates a plain object from a CommitPreparedResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static - * @param {query.SetRollbackResponse} message SetRollbackResponse + * @param {query.CommitPreparedResponse} message CommitPreparedResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetRollbackResponse.toObject = function toObject() { + CommitPreparedResponse.toObject = function toObject() { return {}; }; /** - * Converts this SetRollbackResponse to JSON. + * Converts this CommitPreparedResponse to JSON. * @function toJSON - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @instance * @returns {Object.} JSON object */ - SetRollbackResponse.prototype.toJSON = function toJSON() { + CommitPreparedResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetRollbackResponse + * Gets the default type url for CommitPreparedResponse * @function getTypeUrl - * @memberof query.SetRollbackResponse + * @memberof query.CommitPreparedResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetRollbackResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CommitPreparedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.SetRollbackResponse"; + return typeUrlPrefix + "/query.CommitPreparedResponse"; }; - return SetRollbackResponse; + return CommitPreparedResponse; })(); - query.ConcludeTransactionRequest = (function() { + query.RollbackPreparedRequest = (function() { /** - * Properties of a ConcludeTransactionRequest. + * Properties of a RollbackPreparedRequest. * @memberof query - * @interface IConcludeTransactionRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ConcludeTransactionRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ConcludeTransactionRequest immediate_caller_id - * @property {query.ITarget|null} [target] ConcludeTransactionRequest target - * @property {string|null} [dtid] ConcludeTransactionRequest dtid + * @interface IRollbackPreparedRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] RollbackPreparedRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] RollbackPreparedRequest immediate_caller_id + * @property {query.ITarget|null} [target] RollbackPreparedRequest target + * @property {number|Long|null} [transaction_id] RollbackPreparedRequest transaction_id + * @property {string|null} [dtid] RollbackPreparedRequest dtid */ /** - * Constructs a new ConcludeTransactionRequest. + * Constructs a new RollbackPreparedRequest. * @memberof query - * @classdesc Represents a ConcludeTransactionRequest. - * @implements IConcludeTransactionRequest + * @classdesc Represents a RollbackPreparedRequest. + * @implements IRollbackPreparedRequest * @constructor - * @param {query.IConcludeTransactionRequest=} [properties] Properties to set + * @param {query.IRollbackPreparedRequest=} [properties] Properties to set */ - function ConcludeTransactionRequest(properties) { + function RollbackPreparedRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -84194,59 +85073,67 @@ export const query = $root.query = (() => { } /** - * ConcludeTransactionRequest effective_caller_id. + * RollbackPreparedRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @instance */ - ConcludeTransactionRequest.prototype.effective_caller_id = null; + RollbackPreparedRequest.prototype.effective_caller_id = null; /** - * ConcludeTransactionRequest immediate_caller_id. + * RollbackPreparedRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @instance */ - ConcludeTransactionRequest.prototype.immediate_caller_id = null; + RollbackPreparedRequest.prototype.immediate_caller_id = null; /** - * ConcludeTransactionRequest target. + * RollbackPreparedRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @instance */ - ConcludeTransactionRequest.prototype.target = null; + RollbackPreparedRequest.prototype.target = null; /** - * ConcludeTransactionRequest dtid. + * RollbackPreparedRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.RollbackPreparedRequest + * @instance + */ + RollbackPreparedRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * RollbackPreparedRequest dtid. * @member {string} dtid - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @instance */ - ConcludeTransactionRequest.prototype.dtid = ""; + RollbackPreparedRequest.prototype.dtid = ""; /** - * Creates a new ConcludeTransactionRequest instance using the specified properties. + * Creates a new RollbackPreparedRequest instance using the specified properties. * @function create - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static - * @param {query.IConcludeTransactionRequest=} [properties] Properties to set - * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest instance + * @param {query.IRollbackPreparedRequest=} [properties] Properties to set + * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest instance */ - ConcludeTransactionRequest.create = function create(properties) { - return new ConcludeTransactionRequest(properties); + RollbackPreparedRequest.create = function create(properties) { + return new RollbackPreparedRequest(properties); }; /** - * Encodes the specified ConcludeTransactionRequest message. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. + * Encodes the specified RollbackPreparedRequest message. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. * @function encode - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static - * @param {query.IConcludeTransactionRequest} message ConcludeTransactionRequest message or plain object to encode + * @param {query.IRollbackPreparedRequest} message RollbackPreparedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ConcludeTransactionRequest.encode = function encode(message, writer) { + RollbackPreparedRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -84255,39 +85142,41 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); + writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); return writer; }; /** - * Encodes the specified ConcludeTransactionRequest message, length delimited. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. + * Encodes the specified RollbackPreparedRequest message, length delimited. Does not implicitly {@link query.RollbackPreparedRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static - * @param {query.IConcludeTransactionRequest} message ConcludeTransactionRequest message or plain object to encode + * @param {query.IRollbackPreparedRequest} message RollbackPreparedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ConcludeTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { + RollbackPreparedRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ConcludeTransactionRequest message from the specified reader or buffer. + * Decodes a RollbackPreparedRequest message from the specified reader or buffer. * @function decode - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest + * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ConcludeTransactionRequest.decode = function decode(reader, length) { + RollbackPreparedRequest.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.query.ConcludeTransactionRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.RollbackPreparedRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -84304,6 +85193,10 @@ export const query = $root.query = (() => { break; } case 4: { + message.transaction_id = reader.int64(); + break; + } + case 5: { message.dtid = reader.string(); break; } @@ -84316,30 +85209,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ConcludeTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a RollbackPreparedRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest + * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ConcludeTransactionRequest.decodeDelimited = function decodeDelimited(reader) { + RollbackPreparedRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ConcludeTransactionRequest message. + * Verifies a RollbackPreparedRequest message. * @function verify - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ConcludeTransactionRequest.verify = function verify(message) { + RollbackPreparedRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -84357,6 +85250,9 @@ export const query = $root.query = (() => { if (error) return "target." + error; } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; if (message.dtid != null && message.hasOwnProperty("dtid")) if (!$util.isString(message.dtid)) return "dtid: string expected"; @@ -84364,47 +85260,56 @@ export const query = $root.query = (() => { }; /** - * Creates a ConcludeTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackPreparedRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static * @param {Object.} object Plain object - * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest + * @returns {query.RollbackPreparedRequest} RollbackPreparedRequest */ - ConcludeTransactionRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ConcludeTransactionRequest) + RollbackPreparedRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.RollbackPreparedRequest) return object; - let message = new $root.query.ConcludeTransactionRequest(); + let message = new $root.query.RollbackPreparedRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ConcludeTransactionRequest.effective_caller_id: object expected"); + throw TypeError(".query.RollbackPreparedRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ConcludeTransactionRequest.immediate_caller_id: object expected"); + throw TypeError(".query.RollbackPreparedRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ConcludeTransactionRequest.target: object expected"); + throw TypeError(".query.RollbackPreparedRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.dtid != null) message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a ConcludeTransactionRequest message. Also converts values to other types if specified. + * Creates a plain object from a RollbackPreparedRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static - * @param {query.ConcludeTransactionRequest} message ConcludeTransactionRequest + * @param {query.RollbackPreparedRequest} message RollbackPreparedRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ConcludeTransactionRequest.toObject = function toObject(message, options) { + RollbackPreparedRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -84412,6 +85317,11 @@ export const query = $root.query = (() => { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) @@ -84420,57 +85330,62 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.dtid != null && message.hasOwnProperty("dtid")) object.dtid = message.dtid; return object; }; /** - * Converts this ConcludeTransactionRequest to JSON. + * Converts this RollbackPreparedRequest to JSON. * @function toJSON - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @instance * @returns {Object.} JSON object */ - ConcludeTransactionRequest.prototype.toJSON = function toJSON() { + RollbackPreparedRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ConcludeTransactionRequest + * Gets the default type url for RollbackPreparedRequest * @function getTypeUrl - * @memberof query.ConcludeTransactionRequest + * @memberof query.RollbackPreparedRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ConcludeTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RollbackPreparedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ConcludeTransactionRequest"; + return typeUrlPrefix + "/query.RollbackPreparedRequest"; }; - return ConcludeTransactionRequest; + return RollbackPreparedRequest; })(); - query.ConcludeTransactionResponse = (function() { + query.RollbackPreparedResponse = (function() { /** - * Properties of a ConcludeTransactionResponse. + * Properties of a RollbackPreparedResponse. * @memberof query - * @interface IConcludeTransactionResponse + * @interface IRollbackPreparedResponse */ /** - * Constructs a new ConcludeTransactionResponse. + * Constructs a new RollbackPreparedResponse. * @memberof query - * @classdesc Represents a ConcludeTransactionResponse. - * @implements IConcludeTransactionResponse + * @classdesc Represents a RollbackPreparedResponse. + * @implements IRollbackPreparedResponse * @constructor - * @param {query.IConcludeTransactionResponse=} [properties] Properties to set + * @param {query.IRollbackPreparedResponse=} [properties] Properties to set */ - function ConcludeTransactionResponse(properties) { + function RollbackPreparedResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -84478,60 +85393,60 @@ export const query = $root.query = (() => { } /** - * Creates a new ConcludeTransactionResponse instance using the specified properties. + * Creates a new RollbackPreparedResponse instance using the specified properties. * @function create - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static - * @param {query.IConcludeTransactionResponse=} [properties] Properties to set - * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse instance + * @param {query.IRollbackPreparedResponse=} [properties] Properties to set + * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse instance */ - ConcludeTransactionResponse.create = function create(properties) { - return new ConcludeTransactionResponse(properties); + RollbackPreparedResponse.create = function create(properties) { + return new RollbackPreparedResponse(properties); }; /** - * Encodes the specified ConcludeTransactionResponse message. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. + * Encodes the specified RollbackPreparedResponse message. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. * @function encode - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static - * @param {query.IConcludeTransactionResponse} message ConcludeTransactionResponse message or plain object to encode + * @param {query.IRollbackPreparedResponse} message RollbackPreparedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ConcludeTransactionResponse.encode = function encode(message, writer) { + RollbackPreparedResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified ConcludeTransactionResponse message, length delimited. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. + * Encodes the specified RollbackPreparedResponse message, length delimited. Does not implicitly {@link query.RollbackPreparedResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static - * @param {query.IConcludeTransactionResponse} message ConcludeTransactionResponse message or plain object to encode + * @param {query.IRollbackPreparedResponse} message RollbackPreparedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ConcludeTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { + RollbackPreparedResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ConcludeTransactionResponse message from the specified reader or buffer. + * Decodes a RollbackPreparedResponse message from the specified reader or buffer. * @function decode - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse + * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ConcludeTransactionResponse.decode = function decode(reader, length) { + RollbackPreparedResponse.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.query.ConcludeTransactionResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.RollbackPreparedResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -84544,112 +85459,114 @@ export const query = $root.query = (() => { }; /** - * Decodes a ConcludeTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a RollbackPreparedResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse + * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ConcludeTransactionResponse.decodeDelimited = function decodeDelimited(reader) { + RollbackPreparedResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ConcludeTransactionResponse message. + * Verifies a RollbackPreparedResponse message. * @function verify - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ConcludeTransactionResponse.verify = function verify(message) { + RollbackPreparedResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a ConcludeTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RollbackPreparedResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static * @param {Object.} object Plain object - * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse + * @returns {query.RollbackPreparedResponse} RollbackPreparedResponse */ - ConcludeTransactionResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ConcludeTransactionResponse) + RollbackPreparedResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.RollbackPreparedResponse) return object; - return new $root.query.ConcludeTransactionResponse(); + return new $root.query.RollbackPreparedResponse(); }; /** - * Creates a plain object from a ConcludeTransactionResponse message. Also converts values to other types if specified. + * Creates a plain object from a RollbackPreparedResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static - * @param {query.ConcludeTransactionResponse} message ConcludeTransactionResponse + * @param {query.RollbackPreparedResponse} message RollbackPreparedResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ConcludeTransactionResponse.toObject = function toObject() { + RollbackPreparedResponse.toObject = function toObject() { return {}; }; /** - * Converts this ConcludeTransactionResponse to JSON. + * Converts this RollbackPreparedResponse to JSON. * @function toJSON - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @instance * @returns {Object.} JSON object */ - ConcludeTransactionResponse.prototype.toJSON = function toJSON() { + RollbackPreparedResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ConcludeTransactionResponse + * Gets the default type url for RollbackPreparedResponse * @function getTypeUrl - * @memberof query.ConcludeTransactionResponse + * @memberof query.RollbackPreparedResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ConcludeTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RollbackPreparedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ConcludeTransactionResponse"; + return typeUrlPrefix + "/query.RollbackPreparedResponse"; }; - return ConcludeTransactionResponse; + return RollbackPreparedResponse; })(); - query.ReadTransactionRequest = (function() { + query.CreateTransactionRequest = (function() { /** - * Properties of a ReadTransactionRequest. + * Properties of a CreateTransactionRequest. * @memberof query - * @interface IReadTransactionRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReadTransactionRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReadTransactionRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReadTransactionRequest target - * @property {string|null} [dtid] ReadTransactionRequest dtid + * @interface ICreateTransactionRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] CreateTransactionRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] CreateTransactionRequest immediate_caller_id + * @property {query.ITarget|null} [target] CreateTransactionRequest target + * @property {string|null} [dtid] CreateTransactionRequest dtid + * @property {Array.|null} [participants] CreateTransactionRequest participants */ /** - * Constructs a new ReadTransactionRequest. + * Constructs a new CreateTransactionRequest. * @memberof query - * @classdesc Represents a ReadTransactionRequest. - * @implements IReadTransactionRequest + * @classdesc Represents a CreateTransactionRequest. + * @implements ICreateTransactionRequest * @constructor - * @param {query.IReadTransactionRequest=} [properties] Properties to set + * @param {query.ICreateTransactionRequest=} [properties] Properties to set */ - function ReadTransactionRequest(properties) { + function CreateTransactionRequest(properties) { + this.participants = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -84657,59 +85574,67 @@ export const query = $root.query = (() => { } /** - * ReadTransactionRequest effective_caller_id. + * CreateTransactionRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @instance */ - ReadTransactionRequest.prototype.effective_caller_id = null; + CreateTransactionRequest.prototype.effective_caller_id = null; /** - * ReadTransactionRequest immediate_caller_id. + * CreateTransactionRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @instance */ - ReadTransactionRequest.prototype.immediate_caller_id = null; + CreateTransactionRequest.prototype.immediate_caller_id = null; /** - * ReadTransactionRequest target. + * CreateTransactionRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @instance */ - ReadTransactionRequest.prototype.target = null; + CreateTransactionRequest.prototype.target = null; /** - * ReadTransactionRequest dtid. + * CreateTransactionRequest dtid. * @member {string} dtid - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @instance */ - ReadTransactionRequest.prototype.dtid = ""; + CreateTransactionRequest.prototype.dtid = ""; /** - * Creates a new ReadTransactionRequest instance using the specified properties. + * CreateTransactionRequest participants. + * @member {Array.} participants + * @memberof query.CreateTransactionRequest + * @instance + */ + CreateTransactionRequest.prototype.participants = $util.emptyArray; + + /** + * Creates a new CreateTransactionRequest instance using the specified properties. * @function create - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static - * @param {query.IReadTransactionRequest=} [properties] Properties to set - * @returns {query.ReadTransactionRequest} ReadTransactionRequest instance + * @param {query.ICreateTransactionRequest=} [properties] Properties to set + * @returns {query.CreateTransactionRequest} CreateTransactionRequest instance */ - ReadTransactionRequest.create = function create(properties) { - return new ReadTransactionRequest(properties); + CreateTransactionRequest.create = function create(properties) { + return new CreateTransactionRequest(properties); }; /** - * Encodes the specified ReadTransactionRequest message. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. + * Encodes the specified CreateTransactionRequest message. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. * @function encode - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static - * @param {query.IReadTransactionRequest} message ReadTransactionRequest message or plain object to encode + * @param {query.ICreateTransactionRequest} message CreateTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadTransactionRequest.encode = function encode(message, writer) { + CreateTransactionRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -84720,37 +85645,40 @@ export const query = $root.query = (() => { $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); + if (message.participants != null && message.participants.length) + for (let i = 0; i < message.participants.length; ++i) + $root.query.Target.encode(message.participants[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReadTransactionRequest message, length delimited. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. + * Encodes the specified CreateTransactionRequest message, length delimited. Does not implicitly {@link query.CreateTransactionRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static - * @param {query.IReadTransactionRequest} message ReadTransactionRequest message or plain object to encode + * @param {query.ICreateTransactionRequest} message CreateTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { + CreateTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReadTransactionRequest message from the specified reader or buffer. + * Decodes a CreateTransactionRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReadTransactionRequest} ReadTransactionRequest + * @returns {query.CreateTransactionRequest} CreateTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadTransactionRequest.decode = function decode(reader, length) { + CreateTransactionRequest.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.query.ReadTransactionRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CreateTransactionRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -84770,6 +85698,12 @@ export const query = $root.query = (() => { message.dtid = reader.string(); break; } + case 5: { + if (!(message.participants && message.participants.length)) + message.participants = []; + message.participants.push($root.query.Target.decode(reader, reader.uint32())); + break; + } default: reader.skipType(tag & 7); break; @@ -84779,30 +85713,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReadTransactionRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateTransactionRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReadTransactionRequest} ReadTransactionRequest + * @returns {query.CreateTransactionRequest} CreateTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadTransactionRequest.decodeDelimited = function decodeDelimited(reader) { + CreateTransactionRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReadTransactionRequest message. + * Verifies a CreateTransactionRequest message. * @function verify - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReadTransactionRequest.verify = function verify(message) { + CreateTransactionRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -84823,54 +85757,75 @@ export const query = $root.query = (() => { if (message.dtid != null && message.hasOwnProperty("dtid")) if (!$util.isString(message.dtid)) return "dtid: string expected"; + if (message.participants != null && message.hasOwnProperty("participants")) { + if (!Array.isArray(message.participants)) + return "participants: array expected"; + for (let i = 0; i < message.participants.length; ++i) { + let error = $root.query.Target.verify(message.participants[i]); + if (error) + return "participants." + error; + } + } return null; }; /** - * Creates a ReadTransactionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateTransactionRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static * @param {Object.} object Plain object - * @returns {query.ReadTransactionRequest} ReadTransactionRequest + * @returns {query.CreateTransactionRequest} CreateTransactionRequest */ - ReadTransactionRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReadTransactionRequest) + CreateTransactionRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.CreateTransactionRequest) return object; - let message = new $root.query.ReadTransactionRequest(); + let message = new $root.query.CreateTransactionRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReadTransactionRequest.effective_caller_id: object expected"); + throw TypeError(".query.CreateTransactionRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReadTransactionRequest.immediate_caller_id: object expected"); + throw TypeError(".query.CreateTransactionRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReadTransactionRequest.target: object expected"); + throw TypeError(".query.CreateTransactionRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } if (object.dtid != null) message.dtid = String(object.dtid); + if (object.participants) { + if (!Array.isArray(object.participants)) + throw TypeError(".query.CreateTransactionRequest.participants: array expected"); + message.participants = []; + for (let i = 0; i < object.participants.length; ++i) { + if (typeof object.participants[i] !== "object") + throw TypeError(".query.CreateTransactionRequest.participants: object expected"); + message.participants[i] = $root.query.Target.fromObject(object.participants[i]); + } + } return message; }; /** - * Creates a plain object from a ReadTransactionRequest message. Also converts values to other types if specified. + * Creates a plain object from a CreateTransactionRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static - * @param {query.ReadTransactionRequest} message ReadTransactionRequest + * @param {query.CreateTransactionRequest} message CreateTransactionRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReadTransactionRequest.toObject = function toObject(message, options) { + CreateTransactionRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.participants = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; @@ -84885,56 +85840,60 @@ export const query = $root.query = (() => { object.target = $root.query.Target.toObject(message.target, options); if (message.dtid != null && message.hasOwnProperty("dtid")) object.dtid = message.dtid; + if (message.participants && message.participants.length) { + object.participants = []; + for (let j = 0; j < message.participants.length; ++j) + object.participants[j] = $root.query.Target.toObject(message.participants[j], options); + } return object; }; /** - * Converts this ReadTransactionRequest to JSON. + * Converts this CreateTransactionRequest to JSON. * @function toJSON - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @instance * @returns {Object.} JSON object */ - ReadTransactionRequest.prototype.toJSON = function toJSON() { + CreateTransactionRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReadTransactionRequest + * Gets the default type url for CreateTransactionRequest * @function getTypeUrl - * @memberof query.ReadTransactionRequest + * @memberof query.CreateTransactionRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReadTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReadTransactionRequest"; + return typeUrlPrefix + "/query.CreateTransactionRequest"; }; - return ReadTransactionRequest; + return CreateTransactionRequest; })(); - query.ReadTransactionResponse = (function() { + query.CreateTransactionResponse = (function() { /** - * Properties of a ReadTransactionResponse. + * Properties of a CreateTransactionResponse. * @memberof query - * @interface IReadTransactionResponse - * @property {query.ITransactionMetadata|null} [metadata] ReadTransactionResponse metadata + * @interface ICreateTransactionResponse */ /** - * Constructs a new ReadTransactionResponse. + * Constructs a new CreateTransactionResponse. * @memberof query - * @classdesc Represents a ReadTransactionResponse. - * @implements IReadTransactionResponse + * @classdesc Represents a CreateTransactionResponse. + * @implements ICreateTransactionResponse * @constructor - * @param {query.IReadTransactionResponse=} [properties] Properties to set + * @param {query.ICreateTransactionResponse=} [properties] Properties to set */ - function ReadTransactionResponse(properties) { + function CreateTransactionResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -84942,77 +85901,63 @@ export const query = $root.query = (() => { } /** - * ReadTransactionResponse metadata. - * @member {query.ITransactionMetadata|null|undefined} metadata - * @memberof query.ReadTransactionResponse - * @instance - */ - ReadTransactionResponse.prototype.metadata = null; - - /** - * Creates a new ReadTransactionResponse instance using the specified properties. + * Creates a new CreateTransactionResponse instance using the specified properties. * @function create - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static - * @param {query.IReadTransactionResponse=} [properties] Properties to set - * @returns {query.ReadTransactionResponse} ReadTransactionResponse instance + * @param {query.ICreateTransactionResponse=} [properties] Properties to set + * @returns {query.CreateTransactionResponse} CreateTransactionResponse instance */ - ReadTransactionResponse.create = function create(properties) { - return new ReadTransactionResponse(properties); + CreateTransactionResponse.create = function create(properties) { + return new CreateTransactionResponse(properties); }; /** - * Encodes the specified ReadTransactionResponse message. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. + * Encodes the specified CreateTransactionResponse message. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. * @function encode - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static - * @param {query.IReadTransactionResponse} message ReadTransactionResponse message or plain object to encode + * @param {query.ICreateTransactionResponse} message CreateTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadTransactionResponse.encode = function encode(message, writer) { + CreateTransactionResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.metadata != null && Object.hasOwnProperty.call(message, "metadata")) - $root.query.TransactionMetadata.encode(message.metadata, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReadTransactionResponse message, length delimited. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. + * Encodes the specified CreateTransactionResponse message, length delimited. Does not implicitly {@link query.CreateTransactionResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static - * @param {query.IReadTransactionResponse} message ReadTransactionResponse message or plain object to encode + * @param {query.ICreateTransactionResponse} message CreateTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { + CreateTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReadTransactionResponse message from the specified reader or buffer. + * Decodes a CreateTransactionResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReadTransactionResponse} ReadTransactionResponse + * @returns {query.CreateTransactionResponse} CreateTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadTransactionResponse.decode = function decode(reader, length) { + CreateTransactionResponse.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.query.ReadTransactionResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.CreateTransactionResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.metadata = $root.query.TransactionMetadata.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -85022,134 +85967,113 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReadTransactionResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateTransactionResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReadTransactionResponse} ReadTransactionResponse + * @returns {query.CreateTransactionResponse} CreateTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadTransactionResponse.decodeDelimited = function decodeDelimited(reader) { + CreateTransactionResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReadTransactionResponse message. + * Verifies a CreateTransactionResponse message. * @function verify - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReadTransactionResponse.verify = function verify(message) { + CreateTransactionResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.metadata != null && message.hasOwnProperty("metadata")) { - let error = $root.query.TransactionMetadata.verify(message.metadata); - if (error) - return "metadata." + error; - } return null; }; /** - * Creates a ReadTransactionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateTransactionResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static * @param {Object.} object Plain object - * @returns {query.ReadTransactionResponse} ReadTransactionResponse + * @returns {query.CreateTransactionResponse} CreateTransactionResponse */ - ReadTransactionResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReadTransactionResponse) + CreateTransactionResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.CreateTransactionResponse) return object; - let message = new $root.query.ReadTransactionResponse(); - if (object.metadata != null) { - if (typeof object.metadata !== "object") - throw TypeError(".query.ReadTransactionResponse.metadata: object expected"); - message.metadata = $root.query.TransactionMetadata.fromObject(object.metadata); - } - return message; + return new $root.query.CreateTransactionResponse(); }; /** - * Creates a plain object from a ReadTransactionResponse message. Also converts values to other types if specified. + * Creates a plain object from a CreateTransactionResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static - * @param {query.ReadTransactionResponse} message ReadTransactionResponse + * @param {query.CreateTransactionResponse} message CreateTransactionResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReadTransactionResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.metadata = null; - if (message.metadata != null && message.hasOwnProperty("metadata")) - object.metadata = $root.query.TransactionMetadata.toObject(message.metadata, options); - return object; + CreateTransactionResponse.toObject = function toObject() { + return {}; }; /** - * Converts this ReadTransactionResponse to JSON. + * Converts this CreateTransactionResponse to JSON. * @function toJSON - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @instance * @returns {Object.} JSON object */ - ReadTransactionResponse.prototype.toJSON = function toJSON() { + CreateTransactionResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReadTransactionResponse + * Gets the default type url for CreateTransactionResponse * @function getTypeUrl - * @memberof query.ReadTransactionResponse + * @memberof query.CreateTransactionResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReadTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReadTransactionResponse"; + return typeUrlPrefix + "/query.CreateTransactionResponse"; }; - return ReadTransactionResponse; + return CreateTransactionResponse; })(); - query.BeginExecuteRequest = (function() { + query.StartCommitRequest = (function() { /** - * Properties of a BeginExecuteRequest. + * Properties of a StartCommitRequest. * @memberof query - * @interface IBeginExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] BeginExecuteRequest target - * @property {query.IBoundQuery|null} [query] BeginExecuteRequest query - * @property {query.IExecuteOptions|null} [options] BeginExecuteRequest options - * @property {number|Long|null} [reserved_id] BeginExecuteRequest reserved_id - * @property {Array.|null} [pre_queries] BeginExecuteRequest pre_queries + * @interface IStartCommitRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] StartCommitRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] StartCommitRequest immediate_caller_id + * @property {query.ITarget|null} [target] StartCommitRequest target + * @property {number|Long|null} [transaction_id] StartCommitRequest transaction_id + * @property {string|null} [dtid] StartCommitRequest dtid */ /** - * Constructs a new BeginExecuteRequest. + * Constructs a new StartCommitRequest. * @memberof query - * @classdesc Represents a BeginExecuteRequest. - * @implements IBeginExecuteRequest + * @classdesc Represents a StartCommitRequest. + * @implements IStartCommitRequest * @constructor - * @param {query.IBeginExecuteRequest=} [properties] Properties to set + * @param {query.IStartCommitRequest=} [properties] Properties to set */ - function BeginExecuteRequest(properties) { - this.pre_queries = []; + function StartCommitRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -85157,83 +86081,67 @@ export const query = $root.query = (() => { } /** - * BeginExecuteRequest effective_caller_id. + * StartCommitRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @instance */ - BeginExecuteRequest.prototype.effective_caller_id = null; + StartCommitRequest.prototype.effective_caller_id = null; /** - * BeginExecuteRequest immediate_caller_id. + * StartCommitRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @instance */ - BeginExecuteRequest.prototype.immediate_caller_id = null; + StartCommitRequest.prototype.immediate_caller_id = null; /** - * BeginExecuteRequest target. + * StartCommitRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.BeginExecuteRequest - * @instance - */ - BeginExecuteRequest.prototype.target = null; - - /** - * BeginExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.BeginExecuteRequest - * @instance - */ - BeginExecuteRequest.prototype.query = null; - - /** - * BeginExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @instance */ - BeginExecuteRequest.prototype.options = null; + StartCommitRequest.prototype.target = null; /** - * BeginExecuteRequest reserved_id. - * @member {number|Long} reserved_id - * @memberof query.BeginExecuteRequest + * StartCommitRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.StartCommitRequest * @instance */ - BeginExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + StartCommitRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * BeginExecuteRequest pre_queries. - * @member {Array.} pre_queries - * @memberof query.BeginExecuteRequest + * StartCommitRequest dtid. + * @member {string} dtid + * @memberof query.StartCommitRequest * @instance */ - BeginExecuteRequest.prototype.pre_queries = $util.emptyArray; + StartCommitRequest.prototype.dtid = ""; /** - * Creates a new BeginExecuteRequest instance using the specified properties. + * Creates a new StartCommitRequest instance using the specified properties. * @function create - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static - * @param {query.IBeginExecuteRequest=} [properties] Properties to set - * @returns {query.BeginExecuteRequest} BeginExecuteRequest instance + * @param {query.IStartCommitRequest=} [properties] Properties to set + * @returns {query.StartCommitRequest} StartCommitRequest instance */ - BeginExecuteRequest.create = function create(properties) { - return new BeginExecuteRequest(properties); + StartCommitRequest.create = function create(properties) { + return new StartCommitRequest(properties); }; /** - * Encodes the specified BeginExecuteRequest message. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. + * Encodes the specified StartCommitRequest message. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. * @function encode - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static - * @param {query.IBeginExecuteRequest} message BeginExecuteRequest message or plain object to encode + * @param {query.IStartCommitRequest} message StartCommitRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginExecuteRequest.encode = function encode(message, writer) { + StartCommitRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -85242,46 +86150,41 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.reserved_id); - if (message.pre_queries != null && message.pre_queries.length) - for (let i = 0; i < message.pre_queries.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.pre_queries[i]); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); + if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); return writer; }; /** - * Encodes the specified BeginExecuteRequest message, length delimited. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. + * Encodes the specified StartCommitRequest message, length delimited. Does not implicitly {@link query.StartCommitRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static - * @param {query.IBeginExecuteRequest} message BeginExecuteRequest message or plain object to encode + * @param {query.IStartCommitRequest} message StartCommitRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + StartCommitRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginExecuteRequest message from the specified reader or buffer. + * Decodes a StartCommitRequest message from the specified reader or buffer. * @function decode - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginExecuteRequest} BeginExecuteRequest + * @returns {query.StartCommitRequest} StartCommitRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginExecuteRequest.decode = function decode(reader, length) { + StartCommitRequest.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.query.BeginExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StartCommitRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -85298,21 +86201,11 @@ export const query = $root.query = (() => { break; } case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + message.transaction_id = reader.int64(); break; } case 5: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); - break; - } - case 6: { - message.reserved_id = reader.int64(); - break; - } - case 7: { - if (!(message.pre_queries && message.pre_queries.length)) - message.pre_queries = []; - message.pre_queries.push(reader.string()); + message.dtid = reader.string(); break; } default: @@ -85324,30 +86217,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a StartCommitRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginExecuteRequest} BeginExecuteRequest + * @returns {query.StartCommitRequest} StartCommitRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + StartCommitRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginExecuteRequest message. + * Verifies a StartCommitRequest message. * @function verify - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginExecuteRequest.verify = function verify(message) { + StartCommitRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -85365,111 +86258,79 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; - } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; - if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { - if (!Array.isArray(message.pre_queries)) - return "pre_queries: array expected"; - for (let i = 0; i < message.pre_queries.length; ++i) - if (!$util.isString(message.pre_queries[i])) - return "pre_queries: string[] expected"; - } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.dtid != null && message.hasOwnProperty("dtid")) + if (!$util.isString(message.dtid)) + return "dtid: string expected"; return null; }; /** - * Creates a BeginExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StartCommitRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static * @param {Object.} object Plain object - * @returns {query.BeginExecuteRequest} BeginExecuteRequest + * @returns {query.StartCommitRequest} StartCommitRequest */ - BeginExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginExecuteRequest) + StartCommitRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.StartCommitRequest) return object; - let message = new $root.query.BeginExecuteRequest(); + let message = new $root.query.StartCommitRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.BeginExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.StartCommitRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.BeginExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.StartCommitRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.BeginExecuteRequest.target: object expected"); + throw TypeError(".query.StartCommitRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.BeginExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.BeginExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); - } - if (object.reserved_id != null) + if (object.transaction_id != null) if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); - if (object.pre_queries) { - if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.BeginExecuteRequest.pre_queries: array expected"); - message.pre_queries = []; - for (let i = 0; i < object.pre_queries.length; ++i) - message.pre_queries[i] = String(object.pre_queries[i]); - } + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.dtid != null) + message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a BeginExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a StartCommitRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static - * @param {query.BeginExecuteRequest} message BeginExecuteRequest + * @param {query.StartCommitRequest} message StartCommitRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginExecuteRequest.toObject = function toObject(message, options) { + StartCommitRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.pre_queries = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.query = null; - object.options = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.reserved_id = options.longs === String ? "0" : 0; + object.transaction_id = options.longs === String ? "0" : 0; + object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -85477,74 +86338,62 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; - if (message.pre_queries && message.pre_queries.length) { - object.pre_queries = []; - for (let j = 0; j < message.pre_queries.length; ++j) - object.pre_queries[j] = message.pre_queries[j]; - } + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.dtid != null && message.hasOwnProperty("dtid")) + object.dtid = message.dtid; return object; }; /** - * Converts this BeginExecuteRequest to JSON. + * Converts this StartCommitRequest to JSON. * @function toJSON - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @instance * @returns {Object.} JSON object */ - BeginExecuteRequest.prototype.toJSON = function toJSON() { + StartCommitRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginExecuteRequest + * Gets the default type url for StartCommitRequest * @function getTypeUrl - * @memberof query.BeginExecuteRequest + * @memberof query.StartCommitRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StartCommitRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginExecuteRequest"; + return typeUrlPrefix + "/query.StartCommitRequest"; }; - return BeginExecuteRequest; + return StartCommitRequest; })(); - query.BeginExecuteResponse = (function() { + query.StartCommitResponse = (function() { /** - * Properties of a BeginExecuteResponse. + * Properties of a StartCommitResponse. * @memberof query - * @interface IBeginExecuteResponse - * @property {vtrpc.IRPCError|null} [error] BeginExecuteResponse error - * @property {query.IQueryResult|null} [result] BeginExecuteResponse result - * @property {number|Long|null} [transaction_id] BeginExecuteResponse transaction_id - * @property {topodata.ITabletAlias|null} [tablet_alias] BeginExecuteResponse tablet_alias - * @property {string|null} [session_state_changes] BeginExecuteResponse session_state_changes + * @interface IStartCommitResponse */ /** - * Constructs a new BeginExecuteResponse. + * Constructs a new StartCommitResponse. * @memberof query - * @classdesc Represents a BeginExecuteResponse. - * @implements IBeginExecuteResponse + * @classdesc Represents a StartCommitResponse. + * @implements IStartCommitResponse * @constructor - * @param {query.IBeginExecuteResponse=} [properties] Properties to set + * @param {query.IStartCommitResponse=} [properties] Properties to set */ - function BeginExecuteResponse(properties) { + function StartCommitResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -85552,133 +86401,63 @@ export const query = $root.query = (() => { } /** - * BeginExecuteResponse error. - * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.BeginExecuteResponse - * @instance - */ - BeginExecuteResponse.prototype.error = null; - - /** - * BeginExecuteResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.BeginExecuteResponse - * @instance - */ - BeginExecuteResponse.prototype.result = null; - - /** - * BeginExecuteResponse transaction_id. - * @member {number|Long} transaction_id - * @memberof query.BeginExecuteResponse - * @instance - */ - BeginExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * BeginExecuteResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.BeginExecuteResponse - * @instance - */ - BeginExecuteResponse.prototype.tablet_alias = null; - - /** - * BeginExecuteResponse session_state_changes. - * @member {string} session_state_changes - * @memberof query.BeginExecuteResponse - * @instance - */ - BeginExecuteResponse.prototype.session_state_changes = ""; - - /** - * Creates a new BeginExecuteResponse instance using the specified properties. + * Creates a new StartCommitResponse instance using the specified properties. * @function create - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static - * @param {query.IBeginExecuteResponse=} [properties] Properties to set - * @returns {query.BeginExecuteResponse} BeginExecuteResponse instance + * @param {query.IStartCommitResponse=} [properties] Properties to set + * @returns {query.StartCommitResponse} StartCommitResponse instance */ - BeginExecuteResponse.create = function create(properties) { - return new BeginExecuteResponse(properties); + StartCommitResponse.create = function create(properties) { + return new StartCommitResponse(properties); }; /** - * Encodes the specified BeginExecuteResponse message. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. + * Encodes the specified StartCommitResponse message. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. * @function encode - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static - * @param {query.IBeginExecuteResponse} message BeginExecuteResponse message or plain object to encode + * @param {query.IStartCommitResponse} message StartCommitResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginExecuteResponse.encode = function encode(message, writer) { + StartCommitResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.session_state_changes); return writer; }; /** - * Encodes the specified BeginExecuteResponse message, length delimited. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. + * Encodes the specified StartCommitResponse message, length delimited. Does not implicitly {@link query.StartCommitResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static - * @param {query.IBeginExecuteResponse} message BeginExecuteResponse message or plain object to encode + * @param {query.IStartCommitResponse} message StartCommitResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + StartCommitResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginExecuteResponse message from the specified reader or buffer. + * Decodes a StartCommitResponse message from the specified reader or buffer. * @function decode - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginExecuteResponse} BeginExecuteResponse + * @returns {query.StartCommitResponse} StartCommitResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginExecuteResponse.decode = function decode(reader, length) { + StartCommitResponse.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.query.BeginExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StartCommitResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); - break; - } - case 2: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); - break; - } - case 3: { - message.transaction_id = reader.int64(); - break; - } - case 4: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 5: { - message.session_state_changes = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -85688,191 +86467,113 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a StartCommitResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginExecuteResponse} BeginExecuteResponse + * @returns {query.StartCommitResponse} StartCommitResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + StartCommitResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginExecuteResponse message. + * Verifies a StartCommitResponse message. * @function verify - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginExecuteResponse.verify = function verify(message) { + StartCommitResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.vtrpc.RPCError.verify(message.error); - if (error) - return "error." + error; - } - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; - } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; return null; }; /** - * Creates a BeginExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StartCommitResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static * @param {Object.} object Plain object - * @returns {query.BeginExecuteResponse} BeginExecuteResponse + * @returns {query.StartCommitResponse} StartCommitResponse */ - BeginExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginExecuteResponse) + StartCommitResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.StartCommitResponse) return object; - let message = new $root.query.BeginExecuteResponse(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".query.BeginExecuteResponse.error: object expected"); - message.error = $root.vtrpc.RPCError.fromObject(object.error); - } - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.BeginExecuteResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); - } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".query.BeginExecuteResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); - return message; + return new $root.query.StartCommitResponse(); }; /** - * Creates a plain object from a BeginExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a StartCommitResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static - * @param {query.BeginExecuteResponse} message BeginExecuteResponse + * @param {query.StartCommitResponse} message StartCommitResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginExecuteResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.error = null; - object.result = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - object.tablet_alias = null; - object.session_state_changes = ""; - } - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.vtrpc.RPCError.toObject(message.error, options); - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; - return object; + StartCommitResponse.toObject = function toObject() { + return {}; }; /** - * Converts this BeginExecuteResponse to JSON. + * Converts this StartCommitResponse to JSON. * @function toJSON - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @instance * @returns {Object.} JSON object */ - BeginExecuteResponse.prototype.toJSON = function toJSON() { + StartCommitResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginExecuteResponse + * Gets the default type url for StartCommitResponse * @function getTypeUrl - * @memberof query.BeginExecuteResponse + * @memberof query.StartCommitResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StartCommitResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginExecuteResponse"; + return typeUrlPrefix + "/query.StartCommitResponse"; }; - return BeginExecuteResponse; + return StartCommitResponse; })(); - query.BeginStreamExecuteRequest = (function() { + query.SetRollbackRequest = (function() { /** - * Properties of a BeginStreamExecuteRequest. + * Properties of a SetRollbackRequest. * @memberof query - * @interface IBeginStreamExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginStreamExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginStreamExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] BeginStreamExecuteRequest target - * @property {query.IBoundQuery|null} [query] BeginStreamExecuteRequest query - * @property {query.IExecuteOptions|null} [options] BeginStreamExecuteRequest options - * @property {Array.|null} [pre_queries] BeginStreamExecuteRequest pre_queries - * @property {number|Long|null} [reserved_id] BeginStreamExecuteRequest reserved_id + * @interface ISetRollbackRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] SetRollbackRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] SetRollbackRequest immediate_caller_id + * @property {query.ITarget|null} [target] SetRollbackRequest target + * @property {number|Long|null} [transaction_id] SetRollbackRequest transaction_id + * @property {string|null} [dtid] SetRollbackRequest dtid */ /** - * Constructs a new BeginStreamExecuteRequest. + * Constructs a new SetRollbackRequest. * @memberof query - * @classdesc Represents a BeginStreamExecuteRequest. - * @implements IBeginStreamExecuteRequest + * @classdesc Represents a SetRollbackRequest. + * @implements ISetRollbackRequest * @constructor - * @param {query.IBeginStreamExecuteRequest=} [properties] Properties to set + * @param {query.ISetRollbackRequest=} [properties] Properties to set */ - function BeginStreamExecuteRequest(properties) { - this.pre_queries = []; + function SetRollbackRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -85880,83 +86581,67 @@ export const query = $root.query = (() => { } /** - * BeginStreamExecuteRequest effective_caller_id. + * SetRollbackRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @instance */ - BeginStreamExecuteRequest.prototype.effective_caller_id = null; + SetRollbackRequest.prototype.effective_caller_id = null; /** - * BeginStreamExecuteRequest immediate_caller_id. + * SetRollbackRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @instance */ - BeginStreamExecuteRequest.prototype.immediate_caller_id = null; + SetRollbackRequest.prototype.immediate_caller_id = null; /** - * BeginStreamExecuteRequest target. + * SetRollbackRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.BeginStreamExecuteRequest - * @instance - */ - BeginStreamExecuteRequest.prototype.target = null; - - /** - * BeginStreamExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.BeginStreamExecuteRequest - * @instance - */ - BeginStreamExecuteRequest.prototype.query = null; - - /** - * BeginStreamExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @instance */ - BeginStreamExecuteRequest.prototype.options = null; + SetRollbackRequest.prototype.target = null; /** - * BeginStreamExecuteRequest pre_queries. - * @member {Array.} pre_queries - * @memberof query.BeginStreamExecuteRequest + * SetRollbackRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.SetRollbackRequest * @instance */ - BeginStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; + SetRollbackRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * BeginStreamExecuteRequest reserved_id. - * @member {number|Long} reserved_id - * @memberof query.BeginStreamExecuteRequest + * SetRollbackRequest dtid. + * @member {string} dtid + * @memberof query.SetRollbackRequest * @instance */ - BeginStreamExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + SetRollbackRequest.prototype.dtid = ""; /** - * Creates a new BeginStreamExecuteRequest instance using the specified properties. + * Creates a new SetRollbackRequest instance using the specified properties. * @function create - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static - * @param {query.IBeginStreamExecuteRequest=} [properties] Properties to set - * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest instance + * @param {query.ISetRollbackRequest=} [properties] Properties to set + * @returns {query.SetRollbackRequest} SetRollbackRequest instance */ - BeginStreamExecuteRequest.create = function create(properties) { - return new BeginStreamExecuteRequest(properties); + SetRollbackRequest.create = function create(properties) { + return new SetRollbackRequest(properties); }; /** - * Encodes the specified BeginStreamExecuteRequest message. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. + * Encodes the specified SetRollbackRequest message. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. * @function encode - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static - * @param {query.IBeginStreamExecuteRequest} message BeginStreamExecuteRequest message or plain object to encode + * @param {query.ISetRollbackRequest} message SetRollbackRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginStreamExecuteRequest.encode = function encode(message, writer) { + SetRollbackRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -85965,46 +86650,41 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.pre_queries != null && message.pre_queries.length) - for (let i = 0; i < message.pre_queries.length; ++i) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); + if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.dtid); return writer; }; /** - * Encodes the specified BeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. + * Encodes the specified SetRollbackRequest message, length delimited. Does not implicitly {@link query.SetRollbackRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static - * @param {query.IBeginStreamExecuteRequest} message BeginStreamExecuteRequest message or plain object to encode + * @param {query.ISetRollbackRequest} message SetRollbackRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetRollbackRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer. + * Decodes a SetRollbackRequest message from the specified reader or buffer. * @function decode - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest + * @returns {query.SetRollbackRequest} SetRollbackRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginStreamExecuteRequest.decode = function decode(reader, length) { + SetRollbackRequest.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.query.BeginStreamExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.SetRollbackRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -86021,21 +86701,11 @@ export const query = $root.query = (() => { break; } case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + message.transaction_id = reader.int64(); break; } case 5: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); - break; - } - case 6: { - if (!(message.pre_queries && message.pre_queries.length)) - message.pre_queries = []; - message.pre_queries.push(reader.string()); - break; - } - case 7: { - message.reserved_id = reader.int64(); + message.dtid = reader.string(); break; } default: @@ -86047,30 +86717,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a SetRollbackRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest + * @returns {query.SetRollbackRequest} SetRollbackRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + SetRollbackRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginStreamExecuteRequest message. + * Verifies a SetRollbackRequest message. * @function verify - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginStreamExecuteRequest.verify = function verify(message) { + SetRollbackRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -86088,111 +86758,79 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; - } - if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { - if (!Array.isArray(message.pre_queries)) - return "pre_queries: array expected"; - for (let i = 0; i < message.pre_queries.length; ++i) - if (!$util.isString(message.pre_queries[i])) - return "pre_queries: string[] expected"; - } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.dtid != null && message.hasOwnProperty("dtid")) + if (!$util.isString(message.dtid)) + return "dtid: string expected"; return null; }; /** - * Creates a BeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetRollbackRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static * @param {Object.} object Plain object - * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest + * @returns {query.SetRollbackRequest} SetRollbackRequest */ - BeginStreamExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginStreamExecuteRequest) + SetRollbackRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.SetRollbackRequest) return object; - let message = new $root.query.BeginStreamExecuteRequest(); + let message = new $root.query.SetRollbackRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.BeginStreamExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.SetRollbackRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.BeginStreamExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.SetRollbackRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.BeginStreamExecuteRequest.target: object expected"); + throw TypeError(".query.SetRollbackRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.BeginStreamExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.BeginStreamExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); - } - if (object.pre_queries) { - if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.BeginStreamExecuteRequest.pre_queries: array expected"); - message.pre_queries = []; - for (let i = 0; i < object.pre_queries.length; ++i) - message.pre_queries[i] = String(object.pre_queries[i]); - } - if (object.reserved_id != null) + if (object.transaction_id != null) if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.dtid != null) + message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a BeginStreamExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetRollbackRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static - * @param {query.BeginStreamExecuteRequest} message BeginStreamExecuteRequest + * @param {query.SetRollbackRequest} message SetRollbackRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginStreamExecuteRequest.toObject = function toObject(message, options) { + SetRollbackRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.pre_queries = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.query = null; - object.options = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.reserved_id = options.longs === String ? "0" : 0; + object.transaction_id = options.longs === String ? "0" : 0; + object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -86200,74 +86838,62 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.pre_queries && message.pre_queries.length) { - object.pre_queries = []; - for (let j = 0; j < message.pre_queries.length; ++j) - object.pre_queries[j] = message.pre_queries[j]; - } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.dtid != null && message.hasOwnProperty("dtid")) + object.dtid = message.dtid; return object; }; /** - * Converts this BeginStreamExecuteRequest to JSON. + * Converts this SetRollbackRequest to JSON. * @function toJSON - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @instance * @returns {Object.} JSON object */ - BeginStreamExecuteRequest.prototype.toJSON = function toJSON() { + SetRollbackRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginStreamExecuteRequest + * Gets the default type url for SetRollbackRequest * @function getTypeUrl - * @memberof query.BeginStreamExecuteRequest + * @memberof query.SetRollbackRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetRollbackRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginStreamExecuteRequest"; + return typeUrlPrefix + "/query.SetRollbackRequest"; }; - return BeginStreamExecuteRequest; + return SetRollbackRequest; })(); - query.BeginStreamExecuteResponse = (function() { + query.SetRollbackResponse = (function() { /** - * Properties of a BeginStreamExecuteResponse. + * Properties of a SetRollbackResponse. * @memberof query - * @interface IBeginStreamExecuteResponse - * @property {vtrpc.IRPCError|null} [error] BeginStreamExecuteResponse error - * @property {query.IQueryResult|null} [result] BeginStreamExecuteResponse result - * @property {number|Long|null} [transaction_id] BeginStreamExecuteResponse transaction_id - * @property {topodata.ITabletAlias|null} [tablet_alias] BeginStreamExecuteResponse tablet_alias - * @property {string|null} [session_state_changes] BeginStreamExecuteResponse session_state_changes + * @interface ISetRollbackResponse */ /** - * Constructs a new BeginStreamExecuteResponse. + * Constructs a new SetRollbackResponse. * @memberof query - * @classdesc Represents a BeginStreamExecuteResponse. - * @implements IBeginStreamExecuteResponse + * @classdesc Represents a SetRollbackResponse. + * @implements ISetRollbackResponse * @constructor - * @param {query.IBeginStreamExecuteResponse=} [properties] Properties to set + * @param {query.ISetRollbackResponse=} [properties] Properties to set */ - function BeginStreamExecuteResponse(properties) { + function SetRollbackResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -86275,133 +86901,63 @@ export const query = $root.query = (() => { } /** - * BeginStreamExecuteResponse error. - * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.BeginStreamExecuteResponse - * @instance - */ - BeginStreamExecuteResponse.prototype.error = null; - - /** - * BeginStreamExecuteResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.BeginStreamExecuteResponse - * @instance - */ - BeginStreamExecuteResponse.prototype.result = null; - - /** - * BeginStreamExecuteResponse transaction_id. - * @member {number|Long} transaction_id - * @memberof query.BeginStreamExecuteResponse - * @instance - */ - BeginStreamExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * BeginStreamExecuteResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.BeginStreamExecuteResponse - * @instance - */ - BeginStreamExecuteResponse.prototype.tablet_alias = null; - - /** - * BeginStreamExecuteResponse session_state_changes. - * @member {string} session_state_changes - * @memberof query.BeginStreamExecuteResponse - * @instance - */ - BeginStreamExecuteResponse.prototype.session_state_changes = ""; - - /** - * Creates a new BeginStreamExecuteResponse instance using the specified properties. + * Creates a new SetRollbackResponse instance using the specified properties. * @function create - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static - * @param {query.IBeginStreamExecuteResponse=} [properties] Properties to set - * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse instance + * @param {query.ISetRollbackResponse=} [properties] Properties to set + * @returns {query.SetRollbackResponse} SetRollbackResponse instance */ - BeginStreamExecuteResponse.create = function create(properties) { - return new BeginStreamExecuteResponse(properties); + SetRollbackResponse.create = function create(properties) { + return new SetRollbackResponse(properties); }; /** - * Encodes the specified BeginStreamExecuteResponse message. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. + * Encodes the specified SetRollbackResponse message. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. * @function encode - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static - * @param {query.IBeginStreamExecuteResponse} message BeginStreamExecuteResponse message or plain object to encode + * @param {query.ISetRollbackResponse} message SetRollbackResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginStreamExecuteResponse.encode = function encode(message, writer) { + SetRollbackResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.session_state_changes); return writer; }; /** - * Encodes the specified BeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. + * Encodes the specified SetRollbackResponse message, length delimited. Does not implicitly {@link query.SetRollbackResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static - * @param {query.IBeginStreamExecuteResponse} message BeginStreamExecuteResponse message or plain object to encode + * @param {query.ISetRollbackResponse} message SetRollbackResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BeginStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetRollbackResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer. + * Decodes a SetRollbackResponse message from the specified reader or buffer. * @function decode - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse + * @returns {query.SetRollbackResponse} SetRollbackResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginStreamExecuteResponse.decode = function decode(reader, length) { + SetRollbackResponse.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.query.BeginStreamExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.SetRollbackResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); - break; - } - case 2: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); - break; - } - case 3: { - message.transaction_id = reader.int64(); - break; - } - case 4: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 5: { - message.session_state_changes = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -86411,187 +86967,112 @@ export const query = $root.query = (() => { }; /** - * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a SetRollbackResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse + * @returns {query.SetRollbackResponse} SetRollbackResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BeginStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + SetRollbackResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BeginStreamExecuteResponse message. + * Verifies a SetRollbackResponse message. * @function verify - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BeginStreamExecuteResponse.verify = function verify(message) { + SetRollbackResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.vtrpc.RPCError.verify(message.error); - if (error) - return "error." + error; - } - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; - } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; return null; }; /** - * Creates a BeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetRollbackResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static * @param {Object.} object Plain object - * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse + * @returns {query.SetRollbackResponse} SetRollbackResponse */ - BeginStreamExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.BeginStreamExecuteResponse) + SetRollbackResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.SetRollbackResponse) return object; - let message = new $root.query.BeginStreamExecuteResponse(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".query.BeginStreamExecuteResponse.error: object expected"); - message.error = $root.vtrpc.RPCError.fromObject(object.error); - } - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.BeginStreamExecuteResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); - } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".query.BeginStreamExecuteResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); - return message; + return new $root.query.SetRollbackResponse(); }; /** - * Creates a plain object from a BeginStreamExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetRollbackResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static - * @param {query.BeginStreamExecuteResponse} message BeginStreamExecuteResponse + * @param {query.SetRollbackResponse} message SetRollbackResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BeginStreamExecuteResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.error = null; - object.result = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - object.tablet_alias = null; - object.session_state_changes = ""; - } - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.vtrpc.RPCError.toObject(message.error, options); - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; - return object; + SetRollbackResponse.toObject = function toObject() { + return {}; }; /** - * Converts this BeginStreamExecuteResponse to JSON. + * Converts this SetRollbackResponse to JSON. * @function toJSON - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @instance * @returns {Object.} JSON object */ - BeginStreamExecuteResponse.prototype.toJSON = function toJSON() { + SetRollbackResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BeginStreamExecuteResponse + * Gets the default type url for SetRollbackResponse * @function getTypeUrl - * @memberof query.BeginStreamExecuteResponse + * @memberof query.SetRollbackResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BeginStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetRollbackResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.BeginStreamExecuteResponse"; + return typeUrlPrefix + "/query.SetRollbackResponse"; }; - return BeginStreamExecuteResponse; + return SetRollbackResponse; })(); - query.MessageStreamRequest = (function() { + query.ConcludeTransactionRequest = (function() { /** - * Properties of a MessageStreamRequest. + * Properties of a ConcludeTransactionRequest. * @memberof query - * @interface IMessageStreamRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] MessageStreamRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] MessageStreamRequest immediate_caller_id - * @property {query.ITarget|null} [target] MessageStreamRequest target - * @property {string|null} [name] MessageStreamRequest name + * @interface IConcludeTransactionRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ConcludeTransactionRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ConcludeTransactionRequest immediate_caller_id + * @property {query.ITarget|null} [target] ConcludeTransactionRequest target + * @property {string|null} [dtid] ConcludeTransactionRequest dtid */ /** - * Constructs a new MessageStreamRequest. + * Constructs a new ConcludeTransactionRequest. * @memberof query - * @classdesc Represents a MessageStreamRequest. - * @implements IMessageStreamRequest + * @classdesc Represents a ConcludeTransactionRequest. + * @implements IConcludeTransactionRequest * @constructor - * @param {query.IMessageStreamRequest=} [properties] Properties to set + * @param {query.IConcludeTransactionRequest=} [properties] Properties to set */ - function MessageStreamRequest(properties) { + function ConcludeTransactionRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -86599,59 +87080,59 @@ export const query = $root.query = (() => { } /** - * MessageStreamRequest effective_caller_id. + * ConcludeTransactionRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @instance */ - MessageStreamRequest.prototype.effective_caller_id = null; + ConcludeTransactionRequest.prototype.effective_caller_id = null; /** - * MessageStreamRequest immediate_caller_id. + * ConcludeTransactionRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @instance */ - MessageStreamRequest.prototype.immediate_caller_id = null; + ConcludeTransactionRequest.prototype.immediate_caller_id = null; /** - * MessageStreamRequest target. + * ConcludeTransactionRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @instance */ - MessageStreamRequest.prototype.target = null; + ConcludeTransactionRequest.prototype.target = null; /** - * MessageStreamRequest name. - * @member {string} name - * @memberof query.MessageStreamRequest + * ConcludeTransactionRequest dtid. + * @member {string} dtid + * @memberof query.ConcludeTransactionRequest * @instance */ - MessageStreamRequest.prototype.name = ""; + ConcludeTransactionRequest.prototype.dtid = ""; /** - * Creates a new MessageStreamRequest instance using the specified properties. + * Creates a new ConcludeTransactionRequest instance using the specified properties. * @function create - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static - * @param {query.IMessageStreamRequest=} [properties] Properties to set - * @returns {query.MessageStreamRequest} MessageStreamRequest instance + * @param {query.IConcludeTransactionRequest=} [properties] Properties to set + * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest instance */ - MessageStreamRequest.create = function create(properties) { - return new MessageStreamRequest(properties); + ConcludeTransactionRequest.create = function create(properties) { + return new ConcludeTransactionRequest(properties); }; /** - * Encodes the specified MessageStreamRequest message. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. + * Encodes the specified ConcludeTransactionRequest message. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. * @function encode - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static - * @param {query.IMessageStreamRequest} message MessageStreamRequest message or plain object to encode + * @param {query.IConcludeTransactionRequest} message ConcludeTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageStreamRequest.encode = function encode(message, writer) { + ConcludeTransactionRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -86660,39 +87141,39 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.name); + if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); return writer; }; /** - * Encodes the specified MessageStreamRequest message, length delimited. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. + * Encodes the specified ConcludeTransactionRequest message, length delimited. Does not implicitly {@link query.ConcludeTransactionRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static - * @param {query.IMessageStreamRequest} message MessageStreamRequest message or plain object to encode + * @param {query.IConcludeTransactionRequest} message ConcludeTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageStreamRequest.encodeDelimited = function encodeDelimited(message, writer) { + ConcludeTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MessageStreamRequest message from the specified reader or buffer. + * Decodes a ConcludeTransactionRequest message from the specified reader or buffer. * @function decode - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.MessageStreamRequest} MessageStreamRequest + * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageStreamRequest.decode = function decode(reader, length) { + ConcludeTransactionRequest.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.query.MessageStreamRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ConcludeTransactionRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -86709,7 +87190,7 @@ export const query = $root.query = (() => { break; } case 4: { - message.name = reader.string(); + message.dtid = reader.string(); break; } default: @@ -86721,30 +87202,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a MessageStreamRequest message from the specified reader or buffer, length delimited. + * Decodes a ConcludeTransactionRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.MessageStreamRequest} MessageStreamRequest + * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageStreamRequest.decodeDelimited = function decodeDelimited(reader) { + ConcludeTransactionRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MessageStreamRequest message. + * Verifies a ConcludeTransactionRequest message. * @function verify - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MessageStreamRequest.verify = function verify(message) { + ConcludeTransactionRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -86762,54 +87243,54 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; + if (message.dtid != null && message.hasOwnProperty("dtid")) + if (!$util.isString(message.dtid)) + return "dtid: string expected"; return null; }; /** - * Creates a MessageStreamRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ConcludeTransactionRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static * @param {Object.} object Plain object - * @returns {query.MessageStreamRequest} MessageStreamRequest + * @returns {query.ConcludeTransactionRequest} ConcludeTransactionRequest */ - MessageStreamRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.MessageStreamRequest) + ConcludeTransactionRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ConcludeTransactionRequest) return object; - let message = new $root.query.MessageStreamRequest(); + let message = new $root.query.ConcludeTransactionRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.MessageStreamRequest.effective_caller_id: object expected"); + throw TypeError(".query.ConcludeTransactionRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.MessageStreamRequest.immediate_caller_id: object expected"); + throw TypeError(".query.ConcludeTransactionRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.MessageStreamRequest.target: object expected"); + throw TypeError(".query.ConcludeTransactionRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.name != null) - message.name = String(object.name); + if (object.dtid != null) + message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a MessageStreamRequest message. Also converts values to other types if specified. + * Creates a plain object from a ConcludeTransactionRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static - * @param {query.MessageStreamRequest} message MessageStreamRequest + * @param {query.ConcludeTransactionRequest} message ConcludeTransactionRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MessageStreamRequest.toObject = function toObject(message, options) { + ConcludeTransactionRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -86817,7 +87298,7 @@ export const query = $root.query = (() => { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.name = ""; + object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -86825,58 +87306,57 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; + if (message.dtid != null && message.hasOwnProperty("dtid")) + object.dtid = message.dtid; return object; }; /** - * Converts this MessageStreamRequest to JSON. + * Converts this ConcludeTransactionRequest to JSON. * @function toJSON - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @instance * @returns {Object.} JSON object */ - MessageStreamRequest.prototype.toJSON = function toJSON() { + ConcludeTransactionRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MessageStreamRequest + * Gets the default type url for ConcludeTransactionRequest * @function getTypeUrl - * @memberof query.MessageStreamRequest + * @memberof query.ConcludeTransactionRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MessageStreamRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ConcludeTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.MessageStreamRequest"; + return typeUrlPrefix + "/query.ConcludeTransactionRequest"; }; - return MessageStreamRequest; + return ConcludeTransactionRequest; })(); - query.MessageStreamResponse = (function() { + query.ConcludeTransactionResponse = (function() { /** - * Properties of a MessageStreamResponse. + * Properties of a ConcludeTransactionResponse. * @memberof query - * @interface IMessageStreamResponse - * @property {query.IQueryResult|null} [result] MessageStreamResponse result + * @interface IConcludeTransactionResponse */ /** - * Constructs a new MessageStreamResponse. + * Constructs a new ConcludeTransactionResponse. * @memberof query - * @classdesc Represents a MessageStreamResponse. - * @implements IMessageStreamResponse + * @classdesc Represents a ConcludeTransactionResponse. + * @implements IConcludeTransactionResponse * @constructor - * @param {query.IMessageStreamResponse=} [properties] Properties to set + * @param {query.IConcludeTransactionResponse=} [properties] Properties to set */ - function MessageStreamResponse(properties) { + function ConcludeTransactionResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -86884,77 +87364,63 @@ export const query = $root.query = (() => { } /** - * MessageStreamResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.MessageStreamResponse - * @instance - */ - MessageStreamResponse.prototype.result = null; - - /** - * Creates a new MessageStreamResponse instance using the specified properties. + * Creates a new ConcludeTransactionResponse instance using the specified properties. * @function create - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static - * @param {query.IMessageStreamResponse=} [properties] Properties to set - * @returns {query.MessageStreamResponse} MessageStreamResponse instance + * @param {query.IConcludeTransactionResponse=} [properties] Properties to set + * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse instance */ - MessageStreamResponse.create = function create(properties) { - return new MessageStreamResponse(properties); + ConcludeTransactionResponse.create = function create(properties) { + return new ConcludeTransactionResponse(properties); }; /** - * Encodes the specified MessageStreamResponse message. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. + * Encodes the specified ConcludeTransactionResponse message. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. * @function encode - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static - * @param {query.IMessageStreamResponse} message MessageStreamResponse message or plain object to encode + * @param {query.IConcludeTransactionResponse} message ConcludeTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageStreamResponse.encode = function encode(message, writer) { + ConcludeTransactionResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified MessageStreamResponse message, length delimited. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. + * Encodes the specified ConcludeTransactionResponse message, length delimited. Does not implicitly {@link query.ConcludeTransactionResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static - * @param {query.IMessageStreamResponse} message MessageStreamResponse message or plain object to encode + * @param {query.IConcludeTransactionResponse} message ConcludeTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageStreamResponse.encodeDelimited = function encodeDelimited(message, writer) { + ConcludeTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MessageStreamResponse message from the specified reader or buffer. + * Decodes a ConcludeTransactionResponse message from the specified reader or buffer. * @function decode - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.MessageStreamResponse} MessageStreamResponse + * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageStreamResponse.decode = function decode(reader, length) { + ConcludeTransactionResponse.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.query.MessageStreamResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ConcludeTransactionResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -86964,132 +87430,112 @@ export const query = $root.query = (() => { }; /** - * Decodes a MessageStreamResponse message from the specified reader or buffer, length delimited. + * Decodes a ConcludeTransactionResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.MessageStreamResponse} MessageStreamResponse + * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageStreamResponse.decodeDelimited = function decodeDelimited(reader) { + ConcludeTransactionResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MessageStreamResponse message. + * Verifies a ConcludeTransactionResponse message. * @function verify - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MessageStreamResponse.verify = function verify(message) { + ConcludeTransactionResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; - } return null; }; /** - * Creates a MessageStreamResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ConcludeTransactionResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static * @param {Object.} object Plain object - * @returns {query.MessageStreamResponse} MessageStreamResponse + * @returns {query.ConcludeTransactionResponse} ConcludeTransactionResponse */ - MessageStreamResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.MessageStreamResponse) + ConcludeTransactionResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ConcludeTransactionResponse) return object; - let message = new $root.query.MessageStreamResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.MessageStreamResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); - } - return message; + return new $root.query.ConcludeTransactionResponse(); }; /** - * Creates a plain object from a MessageStreamResponse message. Also converts values to other types if specified. + * Creates a plain object from a ConcludeTransactionResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static - * @param {query.MessageStreamResponse} message MessageStreamResponse + * @param {query.ConcludeTransactionResponse} message ConcludeTransactionResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MessageStreamResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); - return object; + ConcludeTransactionResponse.toObject = function toObject() { + return {}; }; /** - * Converts this MessageStreamResponse to JSON. + * Converts this ConcludeTransactionResponse to JSON. * @function toJSON - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @instance * @returns {Object.} JSON object */ - MessageStreamResponse.prototype.toJSON = function toJSON() { + ConcludeTransactionResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MessageStreamResponse + * Gets the default type url for ConcludeTransactionResponse * @function getTypeUrl - * @memberof query.MessageStreamResponse + * @memberof query.ConcludeTransactionResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MessageStreamResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ConcludeTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.MessageStreamResponse"; + return typeUrlPrefix + "/query.ConcludeTransactionResponse"; }; - return MessageStreamResponse; + return ConcludeTransactionResponse; })(); - query.MessageAckRequest = (function() { + query.ReadTransactionRequest = (function() { /** - * Properties of a MessageAckRequest. + * Properties of a ReadTransactionRequest. * @memberof query - * @interface IMessageAckRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] MessageAckRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] MessageAckRequest immediate_caller_id - * @property {query.ITarget|null} [target] MessageAckRequest target - * @property {string|null} [name] MessageAckRequest name - * @property {Array.|null} [ids] MessageAckRequest ids + * @interface IReadTransactionRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReadTransactionRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReadTransactionRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReadTransactionRequest target + * @property {string|null} [dtid] ReadTransactionRequest dtid */ /** - * Constructs a new MessageAckRequest. + * Constructs a new ReadTransactionRequest. * @memberof query - * @classdesc Represents a MessageAckRequest. - * @implements IMessageAckRequest + * @classdesc Represents a ReadTransactionRequest. + * @implements IReadTransactionRequest * @constructor - * @param {query.IMessageAckRequest=} [properties] Properties to set + * @param {query.IReadTransactionRequest=} [properties] Properties to set */ - function MessageAckRequest(properties) { - this.ids = []; + function ReadTransactionRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -87097,67 +87543,59 @@ export const query = $root.query = (() => { } /** - * MessageAckRequest effective_caller_id. + * ReadTransactionRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @instance */ - MessageAckRequest.prototype.effective_caller_id = null; + ReadTransactionRequest.prototype.effective_caller_id = null; /** - * MessageAckRequest immediate_caller_id. + * ReadTransactionRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @instance */ - MessageAckRequest.prototype.immediate_caller_id = null; + ReadTransactionRequest.prototype.immediate_caller_id = null; /** - * MessageAckRequest target. + * ReadTransactionRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.MessageAckRequest - * @instance - */ - MessageAckRequest.prototype.target = null; - - /** - * MessageAckRequest name. - * @member {string} name - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @instance */ - MessageAckRequest.prototype.name = ""; + ReadTransactionRequest.prototype.target = null; /** - * MessageAckRequest ids. - * @member {Array.} ids - * @memberof query.MessageAckRequest + * ReadTransactionRequest dtid. + * @member {string} dtid + * @memberof query.ReadTransactionRequest * @instance */ - MessageAckRequest.prototype.ids = $util.emptyArray; + ReadTransactionRequest.prototype.dtid = ""; /** - * Creates a new MessageAckRequest instance using the specified properties. + * Creates a new ReadTransactionRequest instance using the specified properties. * @function create - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static - * @param {query.IMessageAckRequest=} [properties] Properties to set - * @returns {query.MessageAckRequest} MessageAckRequest instance + * @param {query.IReadTransactionRequest=} [properties] Properties to set + * @returns {query.ReadTransactionRequest} ReadTransactionRequest instance */ - MessageAckRequest.create = function create(properties) { - return new MessageAckRequest(properties); + ReadTransactionRequest.create = function create(properties) { + return new ReadTransactionRequest(properties); }; /** - * Encodes the specified MessageAckRequest message. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. + * Encodes the specified ReadTransactionRequest message. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. * @function encode - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static - * @param {query.IMessageAckRequest} message MessageAckRequest message or plain object to encode + * @param {query.IReadTransactionRequest} message ReadTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageAckRequest.encode = function encode(message, writer) { + ReadTransactionRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -87166,42 +87604,39 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.name); - if (message.ids != null && message.ids.length) - for (let i = 0; i < message.ids.length; ++i) - $root.query.Value.encode(message.ids[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.dtid); return writer; }; /** - * Encodes the specified MessageAckRequest message, length delimited. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. + * Encodes the specified ReadTransactionRequest message, length delimited. Does not implicitly {@link query.ReadTransactionRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static - * @param {query.IMessageAckRequest} message MessageAckRequest message or plain object to encode + * @param {query.IReadTransactionRequest} message ReadTransactionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageAckRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReadTransactionRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MessageAckRequest message from the specified reader or buffer. + * Decodes a ReadTransactionRequest message from the specified reader or buffer. * @function decode - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.MessageAckRequest} MessageAckRequest + * @returns {query.ReadTransactionRequest} ReadTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageAckRequest.decode = function decode(reader, length) { + ReadTransactionRequest.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.query.MessageAckRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReadTransactionRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -87218,13 +87653,7 @@ export const query = $root.query = (() => { break; } case 4: { - message.name = reader.string(); - break; - } - case 5: { - if (!(message.ids && message.ids.length)) - message.ids = []; - message.ids.push($root.query.Value.decode(reader, reader.uint32())); + message.dtid = reader.string(); break; } default: @@ -87236,30 +87665,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a MessageAckRequest message from the specified reader or buffer, length delimited. + * Decodes a ReadTransactionRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.MessageAckRequest} MessageAckRequest + * @returns {query.ReadTransactionRequest} ReadTransactionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageAckRequest.decodeDelimited = function decodeDelimited(reader) { + ReadTransactionRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MessageAckRequest message. + * Verifies a ReadTransactionRequest message. * @function verify - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MessageAckRequest.verify = function verify(message) { + ReadTransactionRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -87277,83 +87706,62 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.ids != null && message.hasOwnProperty("ids")) { - if (!Array.isArray(message.ids)) - return "ids: array expected"; - for (let i = 0; i < message.ids.length; ++i) { - let error = $root.query.Value.verify(message.ids[i]); - if (error) - return "ids." + error; - } - } + if (message.dtid != null && message.hasOwnProperty("dtid")) + if (!$util.isString(message.dtid)) + return "dtid: string expected"; return null; }; /** - * Creates a MessageAckRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReadTransactionRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static * @param {Object.} object Plain object - * @returns {query.MessageAckRequest} MessageAckRequest + * @returns {query.ReadTransactionRequest} ReadTransactionRequest */ - MessageAckRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.MessageAckRequest) + ReadTransactionRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReadTransactionRequest) return object; - let message = new $root.query.MessageAckRequest(); + let message = new $root.query.ReadTransactionRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.MessageAckRequest.effective_caller_id: object expected"); + throw TypeError(".query.ReadTransactionRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.MessageAckRequest.immediate_caller_id: object expected"); + throw TypeError(".query.ReadTransactionRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.MessageAckRequest.target: object expected"); + throw TypeError(".query.ReadTransactionRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.name != null) - message.name = String(object.name); - if (object.ids) { - if (!Array.isArray(object.ids)) - throw TypeError(".query.MessageAckRequest.ids: array expected"); - message.ids = []; - for (let i = 0; i < object.ids.length; ++i) { - if (typeof object.ids[i] !== "object") - throw TypeError(".query.MessageAckRequest.ids: object expected"); - message.ids[i] = $root.query.Value.fromObject(object.ids[i]); - } - } + if (object.dtid != null) + message.dtid = String(object.dtid); return message; }; /** - * Creates a plain object from a MessageAckRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReadTransactionRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static - * @param {query.MessageAckRequest} message MessageAckRequest + * @param {query.ReadTransactionRequest} message ReadTransactionRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MessageAckRequest.toObject = function toObject(message, options) { + ReadTransactionRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.ids = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.name = ""; + object.dtid = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -87361,63 +87769,58 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.ids && message.ids.length) { - object.ids = []; - for (let j = 0; j < message.ids.length; ++j) - object.ids[j] = $root.query.Value.toObject(message.ids[j], options); - } + if (message.dtid != null && message.hasOwnProperty("dtid")) + object.dtid = message.dtid; return object; }; /** - * Converts this MessageAckRequest to JSON. + * Converts this ReadTransactionRequest to JSON. * @function toJSON - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @instance * @returns {Object.} JSON object */ - MessageAckRequest.prototype.toJSON = function toJSON() { + ReadTransactionRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MessageAckRequest + * Gets the default type url for ReadTransactionRequest * @function getTypeUrl - * @memberof query.MessageAckRequest + * @memberof query.ReadTransactionRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MessageAckRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReadTransactionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.MessageAckRequest"; + return typeUrlPrefix + "/query.ReadTransactionRequest"; }; - return MessageAckRequest; + return ReadTransactionRequest; })(); - query.MessageAckResponse = (function() { + query.ReadTransactionResponse = (function() { /** - * Properties of a MessageAckResponse. + * Properties of a ReadTransactionResponse. * @memberof query - * @interface IMessageAckResponse - * @property {query.IQueryResult|null} [result] MessageAckResponse result + * @interface IReadTransactionResponse + * @property {query.ITransactionMetadata|null} [metadata] ReadTransactionResponse metadata */ /** - * Constructs a new MessageAckResponse. + * Constructs a new ReadTransactionResponse. * @memberof query - * @classdesc Represents a MessageAckResponse. - * @implements IMessageAckResponse + * @classdesc Represents a ReadTransactionResponse. + * @implements IReadTransactionResponse * @constructor - * @param {query.IMessageAckResponse=} [properties] Properties to set + * @param {query.IReadTransactionResponse=} [properties] Properties to set */ - function MessageAckResponse(properties) { + function ReadTransactionResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -87425,75 +87828,75 @@ export const query = $root.query = (() => { } /** - * MessageAckResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof query.MessageAckResponse + * ReadTransactionResponse metadata. + * @member {query.ITransactionMetadata|null|undefined} metadata + * @memberof query.ReadTransactionResponse * @instance */ - MessageAckResponse.prototype.result = null; + ReadTransactionResponse.prototype.metadata = null; /** - * Creates a new MessageAckResponse instance using the specified properties. + * Creates a new ReadTransactionResponse instance using the specified properties. * @function create - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static - * @param {query.IMessageAckResponse=} [properties] Properties to set - * @returns {query.MessageAckResponse} MessageAckResponse instance + * @param {query.IReadTransactionResponse=} [properties] Properties to set + * @returns {query.ReadTransactionResponse} ReadTransactionResponse instance */ - MessageAckResponse.create = function create(properties) { - return new MessageAckResponse(properties); + ReadTransactionResponse.create = function create(properties) { + return new ReadTransactionResponse(properties); }; /** - * Encodes the specified MessageAckResponse message. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. + * Encodes the specified ReadTransactionResponse message. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. * @function encode - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static - * @param {query.IMessageAckResponse} message MessageAckResponse message or plain object to encode + * @param {query.IReadTransactionResponse} message ReadTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageAckResponse.encode = function encode(message, writer) { + ReadTransactionResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.metadata != null && Object.hasOwnProperty.call(message, "metadata")) + $root.query.TransactionMetadata.encode(message.metadata, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified MessageAckResponse message, length delimited. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. + * Encodes the specified ReadTransactionResponse message, length delimited. Does not implicitly {@link query.ReadTransactionResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static - * @param {query.IMessageAckResponse} message MessageAckResponse message or plain object to encode + * @param {query.IReadTransactionResponse} message ReadTransactionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MessageAckResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReadTransactionResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MessageAckResponse message from the specified reader or buffer. + * Decodes a ReadTransactionResponse message from the specified reader or buffer. * @function decode - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.MessageAckResponse} MessageAckResponse + * @returns {query.ReadTransactionResponse} ReadTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageAckResponse.decode = function decode(reader, length) { + ReadTransactionResponse.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.query.MessageAckResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReadTransactionResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + message.metadata = $root.query.TransactionMetadata.decode(reader, reader.uint32()); break; } default: @@ -87505,133 +87908,133 @@ export const query = $root.query = (() => { }; /** - * Decodes a MessageAckResponse message from the specified reader or buffer, length delimited. + * Decodes a ReadTransactionResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.MessageAckResponse} MessageAckResponse + * @returns {query.ReadTransactionResponse} ReadTransactionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MessageAckResponse.decodeDelimited = function decodeDelimited(reader) { + ReadTransactionResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MessageAckResponse message. + * Verifies a ReadTransactionResponse message. * @function verify - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MessageAckResponse.verify = function verify(message) { + ReadTransactionResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); + if (message.metadata != null && message.hasOwnProperty("metadata")) { + let error = $root.query.TransactionMetadata.verify(message.metadata); if (error) - return "result." + error; + return "metadata." + error; } return null; }; /** - * Creates a MessageAckResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReadTransactionResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static * @param {Object.} object Plain object - * @returns {query.MessageAckResponse} MessageAckResponse + * @returns {query.ReadTransactionResponse} ReadTransactionResponse */ - MessageAckResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.MessageAckResponse) + ReadTransactionResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReadTransactionResponse) return object; - let message = new $root.query.MessageAckResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".query.MessageAckResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.query.ReadTransactionResponse(); + if (object.metadata != null) { + if (typeof object.metadata !== "object") + throw TypeError(".query.ReadTransactionResponse.metadata: object expected"); + message.metadata = $root.query.TransactionMetadata.fromObject(object.metadata); } return message; }; /** - * Creates a plain object from a MessageAckResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReadTransactionResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static - * @param {query.MessageAckResponse} message MessageAckResponse + * @param {query.ReadTransactionResponse} message ReadTransactionResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MessageAckResponse.toObject = function toObject(message, options) { + ReadTransactionResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); + object.metadata = null; + if (message.metadata != null && message.hasOwnProperty("metadata")) + object.metadata = $root.query.TransactionMetadata.toObject(message.metadata, options); return object; }; /** - * Converts this MessageAckResponse to JSON. + * Converts this ReadTransactionResponse to JSON. * @function toJSON - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @instance * @returns {Object.} JSON object */ - MessageAckResponse.prototype.toJSON = function toJSON() { + ReadTransactionResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MessageAckResponse + * Gets the default type url for ReadTransactionResponse * @function getTypeUrl - * @memberof query.MessageAckResponse + * @memberof query.ReadTransactionResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MessageAckResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReadTransactionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.MessageAckResponse"; + return typeUrlPrefix + "/query.ReadTransactionResponse"; }; - return MessageAckResponse; + return ReadTransactionResponse; })(); - query.ReserveExecuteRequest = (function() { + query.BeginExecuteRequest = (function() { /** - * Properties of a ReserveExecuteRequest. + * Properties of a BeginExecuteRequest. * @memberof query - * @interface IReserveExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReserveExecuteRequest target - * @property {query.IBoundQuery|null} [query] ReserveExecuteRequest query - * @property {number|Long|null} [transaction_id] ReserveExecuteRequest transaction_id - * @property {query.IExecuteOptions|null} [options] ReserveExecuteRequest options - * @property {Array.|null} [pre_queries] ReserveExecuteRequest pre_queries + * @interface IBeginExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] BeginExecuteRequest target + * @property {query.IBoundQuery|null} [query] BeginExecuteRequest query + * @property {query.IExecuteOptions|null} [options] BeginExecuteRequest options + * @property {number|Long|null} [reserved_id] BeginExecuteRequest reserved_id + * @property {Array.|null} [pre_queries] BeginExecuteRequest pre_queries */ /** - * Constructs a new ReserveExecuteRequest. + * Constructs a new BeginExecuteRequest. * @memberof query - * @classdesc Represents a ReserveExecuteRequest. - * @implements IReserveExecuteRequest + * @classdesc Represents a BeginExecuteRequest. + * @implements IBeginExecuteRequest * @constructor - * @param {query.IReserveExecuteRequest=} [properties] Properties to set + * @param {query.IBeginExecuteRequest=} [properties] Properties to set */ - function ReserveExecuteRequest(properties) { + function BeginExecuteRequest(properties) { this.pre_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) @@ -87640,83 +88043,83 @@ export const query = $root.query = (() => { } /** - * ReserveExecuteRequest effective_caller_id. + * BeginExecuteRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.effective_caller_id = null; + BeginExecuteRequest.prototype.effective_caller_id = null; /** - * ReserveExecuteRequest immediate_caller_id. + * BeginExecuteRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.immediate_caller_id = null; + BeginExecuteRequest.prototype.immediate_caller_id = null; /** - * ReserveExecuteRequest target. + * BeginExecuteRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.target = null; + BeginExecuteRequest.prototype.target = null; /** - * ReserveExecuteRequest query. + * BeginExecuteRequest query. * @member {query.IBoundQuery|null|undefined} query - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.query = null; + BeginExecuteRequest.prototype.query = null; /** - * ReserveExecuteRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.ReserveExecuteRequest + * BeginExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + BeginExecuteRequest.prototype.options = null; /** - * ReserveExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.ReserveExecuteRequest + * BeginExecuteRequest reserved_id. + * @member {number|Long} reserved_id + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.options = null; + BeginExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ReserveExecuteRequest pre_queries. + * BeginExecuteRequest pre_queries. * @member {Array.} pre_queries - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance */ - ReserveExecuteRequest.prototype.pre_queries = $util.emptyArray; + BeginExecuteRequest.prototype.pre_queries = $util.emptyArray; /** - * Creates a new ReserveExecuteRequest instance using the specified properties. + * Creates a new BeginExecuteRequest instance using the specified properties. * @function create - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static - * @param {query.IReserveExecuteRequest=} [properties] Properties to set - * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest instance + * @param {query.IBeginExecuteRequest=} [properties] Properties to set + * @returns {query.BeginExecuteRequest} BeginExecuteRequest instance */ - ReserveExecuteRequest.create = function create(properties) { - return new ReserveExecuteRequest(properties); + BeginExecuteRequest.create = function create(properties) { + return new BeginExecuteRequest(properties); }; /** - * Encodes the specified ReserveExecuteRequest message. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. + * Encodes the specified BeginExecuteRequest message. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static - * @param {query.IReserveExecuteRequest} message ReserveExecuteRequest message or plain object to encode + * @param {query.IBeginExecuteRequest} message BeginExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveExecuteRequest.encode = function encode(message, writer) { + BeginExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -87727,10 +88130,10 @@ export const query = $root.query = (() => { $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); if (message.query != null && Object.hasOwnProperty.call(message, "query")) $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.transaction_id); if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.reserved_id); if (message.pre_queries != null && message.pre_queries.length) for (let i = 0; i < message.pre_queries.length; ++i) writer.uint32(/* id 7, wireType 2 =*/58).string(message.pre_queries[i]); @@ -87738,33 +88141,33 @@ export const query = $root.query = (() => { }; /** - * Encodes the specified ReserveExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. + * Encodes the specified BeginExecuteRequest message, length delimited. Does not implicitly {@link query.BeginExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static - * @param {query.IReserveExecuteRequest} message ReserveExecuteRequest message or plain object to encode + * @param {query.IBeginExecuteRequest} message BeginExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + BeginExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveExecuteRequest message from the specified reader or buffer. + * Decodes a BeginExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest + * @returns {query.BeginExecuteRequest} BeginExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveExecuteRequest.decode = function decode(reader, length) { + BeginExecuteRequest.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.query.ReserveExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -87785,11 +88188,11 @@ export const query = $root.query = (() => { break; } case 5: { - message.transaction_id = reader.int64(); + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); break; } case 6: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + message.reserved_id = reader.int64(); break; } case 7: { @@ -87807,30 +88210,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest + * @returns {query.BeginExecuteRequest} BeginExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + BeginExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveExecuteRequest message. + * Verifies a BeginExecuteRequest message. * @function verify - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveExecuteRequest.verify = function verify(message) { + BeginExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -87853,14 +88256,14 @@ export const query = $root.query = (() => { if (error) return "query." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; if (message.options != null && message.hasOwnProperty("options")) { let error = $root.query.ExecuteOptions.verify(message.options); if (error) return "options." + error; } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { if (!Array.isArray(message.pre_queries)) return "pre_queries: array expected"; @@ -87872,54 +88275,54 @@ export const query = $root.query = (() => { }; /** - * Creates a ReserveExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest + * @returns {query.BeginExecuteRequest} BeginExecuteRequest */ - ReserveExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveExecuteRequest) + BeginExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginExecuteRequest) return object; - let message = new $root.query.ReserveExecuteRequest(); + let message = new $root.query.BeginExecuteRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReserveExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.BeginExecuteRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReserveExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.BeginExecuteRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReserveExecuteRequest.target: object expected"); + throw TypeError(".query.BeginExecuteRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } if (object.query != null) { if (typeof object.query !== "object") - throw TypeError(".query.ReserveExecuteRequest.query: object expected"); + throw TypeError(".query.BeginExecuteRequest.query: object expected"); message.query = $root.query.BoundQuery.fromObject(object.query); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.options != null) { if (typeof object.options !== "object") - throw TypeError(".query.ReserveExecuteRequest.options: object expected"); + throw TypeError(".query.BeginExecuteRequest.options: object expected"); message.options = $root.query.ExecuteOptions.fromObject(object.options); } + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); if (object.pre_queries) { if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.ReserveExecuteRequest.pre_queries: array expected"); + throw TypeError(".query.BeginExecuteRequest.pre_queries: array expected"); message.pre_queries = []; for (let i = 0; i < object.pre_queries.length; ++i) message.pre_queries[i] = String(object.pre_queries[i]); @@ -87928,15 +88331,15 @@ export const query = $root.query = (() => { }; /** - * Creates a plain object from a ReserveExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a BeginExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static - * @param {query.ReserveExecuteRequest} message ReserveExecuteRequest + * @param {query.BeginExecuteRequest} message BeginExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveExecuteRequest.toObject = function toObject(message, options) { + BeginExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -87947,12 +88350,12 @@ export const query = $root.query = (() => { object.immediate_caller_id = null; object.target = null; object.query = null; + object.options = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.transaction_id = options.longs === String ? "0" : 0; - object.options = null; + object.reserved_id = options.longs === String ? "0" : 0; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -87962,13 +88365,13 @@ export const query = $root.query = (() => { object.target = $root.query.Target.toObject(message.target, options); if (message.query != null && message.hasOwnProperty("query")) object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.options != null && message.hasOwnProperty("options")) object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; if (message.pre_queries && message.pre_queries.length) { object.pre_queries = []; for (let j = 0; j < message.pre_queries.length; ++j) @@ -87978,55 +88381,56 @@ export const query = $root.query = (() => { }; /** - * Converts this ReserveExecuteRequest to JSON. + * Converts this BeginExecuteRequest to JSON. * @function toJSON - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @instance * @returns {Object.} JSON object */ - ReserveExecuteRequest.prototype.toJSON = function toJSON() { + BeginExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveExecuteRequest + * Gets the default type url for BeginExecuteRequest * @function getTypeUrl - * @memberof query.ReserveExecuteRequest + * @memberof query.BeginExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveExecuteRequest"; + return typeUrlPrefix + "/query.BeginExecuteRequest"; }; - return ReserveExecuteRequest; + return BeginExecuteRequest; })(); - query.ReserveExecuteResponse = (function() { + query.BeginExecuteResponse = (function() { /** - * Properties of a ReserveExecuteResponse. + * Properties of a BeginExecuteResponse. * @memberof query - * @interface IReserveExecuteResponse - * @property {vtrpc.IRPCError|null} [error] ReserveExecuteResponse error - * @property {query.IQueryResult|null} [result] ReserveExecuteResponse result - * @property {number|Long|null} [reserved_id] ReserveExecuteResponse reserved_id - * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveExecuteResponse tablet_alias + * @interface IBeginExecuteResponse + * @property {vtrpc.IRPCError|null} [error] BeginExecuteResponse error + * @property {query.IQueryResult|null} [result] BeginExecuteResponse result + * @property {number|Long|null} [transaction_id] BeginExecuteResponse transaction_id + * @property {topodata.ITabletAlias|null} [tablet_alias] BeginExecuteResponse tablet_alias + * @property {string|null} [session_state_changes] BeginExecuteResponse session_state_changes */ /** - * Constructs a new ReserveExecuteResponse. + * Constructs a new BeginExecuteResponse. * @memberof query - * @classdesc Represents a ReserveExecuteResponse. - * @implements IReserveExecuteResponse + * @classdesc Represents a BeginExecuteResponse. + * @implements IBeginExecuteResponse * @constructor - * @param {query.IReserveExecuteResponse=} [properties] Properties to set + * @param {query.IBeginExecuteResponse=} [properties] Properties to set */ - function ReserveExecuteResponse(properties) { + function BeginExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -88034,100 +88438,110 @@ export const query = $root.query = (() => { } /** - * ReserveExecuteResponse error. + * BeginExecuteResponse error. * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @instance */ - ReserveExecuteResponse.prototype.error = null; + BeginExecuteResponse.prototype.error = null; /** - * ReserveExecuteResponse result. + * BeginExecuteResponse result. * @member {query.IQueryResult|null|undefined} result - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @instance */ - ReserveExecuteResponse.prototype.result = null; + BeginExecuteResponse.prototype.result = null; /** - * ReserveExecuteResponse reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ReserveExecuteResponse + * BeginExecuteResponse transaction_id. + * @member {number|Long} transaction_id + * @memberof query.BeginExecuteResponse * @instance */ - ReserveExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + BeginExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ReserveExecuteResponse tablet_alias. + * BeginExecuteResponse tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @instance */ - ReserveExecuteResponse.prototype.tablet_alias = null; + BeginExecuteResponse.prototype.tablet_alias = null; /** - * Creates a new ReserveExecuteResponse instance using the specified properties. + * BeginExecuteResponse session_state_changes. + * @member {string} session_state_changes + * @memberof query.BeginExecuteResponse + * @instance + */ + BeginExecuteResponse.prototype.session_state_changes = ""; + + /** + * Creates a new BeginExecuteResponse instance using the specified properties. * @function create - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static - * @param {query.IReserveExecuteResponse=} [properties] Properties to set - * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse instance + * @param {query.IBeginExecuteResponse=} [properties] Properties to set + * @returns {query.BeginExecuteResponse} BeginExecuteResponse instance */ - ReserveExecuteResponse.create = function create(properties) { - return new ReserveExecuteResponse(properties); + BeginExecuteResponse.create = function create(properties) { + return new BeginExecuteResponse(properties); }; /** - * Encodes the specified ReserveExecuteResponse message. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. + * Encodes the specified BeginExecuteResponse message. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static - * @param {query.IReserveExecuteResponse} message ReserveExecuteResponse message or plain object to encode + * @param {query.IBeginExecuteResponse} message BeginExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveExecuteResponse.encode = function encode(message, writer) { + BeginExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.error != null && Object.hasOwnProperty.call(message, "error")) $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.result != null && Object.hasOwnProperty.call(message, "result")) $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.reserved_id); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.session_state_changes); return writer; }; /** - * Encodes the specified ReserveExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. + * Encodes the specified BeginExecuteResponse message, length delimited. Does not implicitly {@link query.BeginExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static - * @param {query.IReserveExecuteResponse} message ReserveExecuteResponse message or plain object to encode + * @param {query.IBeginExecuteResponse} message BeginExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + BeginExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveExecuteResponse message from the specified reader or buffer. + * Decodes a BeginExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse + * @returns {query.BeginExecuteResponse} BeginExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveExecuteResponse.decode = function decode(reader, length) { + BeginExecuteResponse.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.query.ReserveExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -88140,13 +88554,17 @@ export const query = $root.query = (() => { break; } case 3: { - message.reserved_id = reader.int64(); + message.transaction_id = reader.int64(); break; } case 4: { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } + case 5: { + message.session_state_changes = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -88156,30 +88574,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse + * @returns {query.BeginExecuteResponse} BeginExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + BeginExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveExecuteResponse message. + * Verifies a BeginExecuteResponse message. * @function verify - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveExecuteResponse.verify = function verify(message) { + BeginExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.error != null && message.hasOwnProperty("error")) { @@ -88192,66 +88610,71 @@ export const query = $root.query = (() => { if (error) return "result." + error; } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) return "tablet_alias." + error; } + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a ReserveExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse + * @returns {query.BeginExecuteResponse} BeginExecuteResponse */ - ReserveExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveExecuteResponse) + BeginExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginExecuteResponse) return object; - let message = new $root.query.ReserveExecuteResponse(); + let message = new $root.query.BeginExecuteResponse(); if (object.error != null) { if (typeof object.error !== "object") - throw TypeError(".query.ReserveExecuteResponse.error: object expected"); + throw TypeError(".query.BeginExecuteResponse.error: object expected"); message.error = $root.vtrpc.RPCError.fromObject(object.error); } if (object.result != null) { if (typeof object.result !== "object") - throw TypeError(".query.ReserveExecuteResponse.result: object expected"); + throw TypeError(".query.BeginExecuteResponse.result: object expected"); message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.reserved_id != null) + if (object.transaction_id != null) if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".query.ReserveExecuteResponse.tablet_alias: object expected"); + throw TypeError(".query.BeginExecuteResponse.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a ReserveExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a BeginExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static - * @param {query.ReserveExecuteResponse} message ReserveExecuteResponse + * @param {query.BeginExecuteResponse} message BeginExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveExecuteResponse.toObject = function toObject(message, options) { + BeginExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -88260,78 +88683,81 @@ export const query = $root.query = (() => { object.result = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.reserved_id = options.longs === String ? "0" : 0; + object.transaction_id = options.longs === String ? "0" : 0; object.tablet_alias = null; + object.session_state_changes = ""; } if (message.error != null && message.hasOwnProperty("error")) object.error = $root.vtrpc.RPCError.toObject(message.error, options); if (message.result != null && message.hasOwnProperty("result")) object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this ReserveExecuteResponse to JSON. + * Converts this BeginExecuteResponse to JSON. * @function toJSON - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @instance * @returns {Object.} JSON object */ - ReserveExecuteResponse.prototype.toJSON = function toJSON() { + BeginExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveExecuteResponse + * Gets the default type url for BeginExecuteResponse * @function getTypeUrl - * @memberof query.ReserveExecuteResponse + * @memberof query.BeginExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveExecuteResponse"; + return typeUrlPrefix + "/query.BeginExecuteResponse"; }; - return ReserveExecuteResponse; + return BeginExecuteResponse; })(); - query.ReserveStreamExecuteRequest = (function() { + query.BeginStreamExecuteRequest = (function() { /** - * Properties of a ReserveStreamExecuteRequest. + * Properties of a BeginStreamExecuteRequest. * @memberof query - * @interface IReserveStreamExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveStreamExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveStreamExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReserveStreamExecuteRequest target - * @property {query.IBoundQuery|null} [query] ReserveStreamExecuteRequest query - * @property {query.IExecuteOptions|null} [options] ReserveStreamExecuteRequest options - * @property {number|Long|null} [transaction_id] ReserveStreamExecuteRequest transaction_id - * @property {Array.|null} [pre_queries] ReserveStreamExecuteRequest pre_queries + * @interface IBeginStreamExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] BeginStreamExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] BeginStreamExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] BeginStreamExecuteRequest target + * @property {query.IBoundQuery|null} [query] BeginStreamExecuteRequest query + * @property {query.IExecuteOptions|null} [options] BeginStreamExecuteRequest options + * @property {Array.|null} [pre_queries] BeginStreamExecuteRequest pre_queries + * @property {number|Long|null} [reserved_id] BeginStreamExecuteRequest reserved_id */ /** - * Constructs a new ReserveStreamExecuteRequest. + * Constructs a new BeginStreamExecuteRequest. * @memberof query - * @classdesc Represents a ReserveStreamExecuteRequest. - * @implements IReserveStreamExecuteRequest + * @classdesc Represents a BeginStreamExecuteRequest. + * @implements IBeginStreamExecuteRequest * @constructor - * @param {query.IReserveStreamExecuteRequest=} [properties] Properties to set + * @param {query.IBeginStreamExecuteRequest=} [properties] Properties to set */ - function ReserveStreamExecuteRequest(properties) { + function BeginStreamExecuteRequest(properties) { this.pre_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) @@ -88340,83 +88766,83 @@ export const query = $root.query = (() => { } /** - * ReserveStreamExecuteRequest effective_caller_id. + * BeginStreamExecuteRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.effective_caller_id = null; + BeginStreamExecuteRequest.prototype.effective_caller_id = null; /** - * ReserveStreamExecuteRequest immediate_caller_id. + * BeginStreamExecuteRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.immediate_caller_id = null; + BeginStreamExecuteRequest.prototype.immediate_caller_id = null; /** - * ReserveStreamExecuteRequest target. + * BeginStreamExecuteRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.target = null; + BeginStreamExecuteRequest.prototype.target = null; /** - * ReserveStreamExecuteRequest query. + * BeginStreamExecuteRequest query. * @member {query.IBoundQuery|null|undefined} query - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.query = null; + BeginStreamExecuteRequest.prototype.query = null; /** - * ReserveStreamExecuteRequest options. + * BeginStreamExecuteRequest options. * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.options = null; + BeginStreamExecuteRequest.prototype.options = null; /** - * ReserveStreamExecuteRequest transaction_id. - * @member {number|Long} transaction_id - * @memberof query.ReserveStreamExecuteRequest + * BeginStreamExecuteRequest pre_queries. + * @member {Array.} pre_queries + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + BeginStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; /** - * ReserveStreamExecuteRequest pre_queries. - * @member {Array.} pre_queries - * @memberof query.ReserveStreamExecuteRequest + * BeginStreamExecuteRequest reserved_id. + * @member {number|Long} reserved_id + * @memberof query.BeginStreamExecuteRequest * @instance */ - ReserveStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; + BeginStreamExecuteRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new ReserveStreamExecuteRequest instance using the specified properties. + * Creates a new BeginStreamExecuteRequest instance using the specified properties. * @function create - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static - * @param {query.IReserveStreamExecuteRequest=} [properties] Properties to set - * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest instance + * @param {query.IBeginStreamExecuteRequest=} [properties] Properties to set + * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest instance */ - ReserveStreamExecuteRequest.create = function create(properties) { - return new ReserveStreamExecuteRequest(properties); + BeginStreamExecuteRequest.create = function create(properties) { + return new BeginStreamExecuteRequest(properties); }; /** - * Encodes the specified ReserveStreamExecuteRequest message. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. + * Encodes the specified BeginStreamExecuteRequest message. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static - * @param {query.IReserveStreamExecuteRequest} message ReserveStreamExecuteRequest message or plain object to encode + * @param {query.IBeginStreamExecuteRequest} message BeginStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveStreamExecuteRequest.encode = function encode(message, writer) { + BeginStreamExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -88429,42 +88855,42 @@ export const query = $root.query = (() => { $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); if (message.options != null && Object.hasOwnProperty.call(message, "options")) $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.transaction_id); if (message.pre_queries != null && message.pre_queries.length) for (let i = 0; i < message.pre_queries.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.pre_queries[i]); + writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 7, wireType 0 =*/56).int64(message.reserved_id); return writer; }; /** - * Encodes the specified ReserveStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. + * Encodes the specified BeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.BeginStreamExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static - * @param {query.IReserveStreamExecuteRequest} message ReserveStreamExecuteRequest message or plain object to encode + * @param {query.IBeginStreamExecuteRequest} message BeginStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + BeginStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer. + * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest + * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveStreamExecuteRequest.decode = function decode(reader, length) { + BeginStreamExecuteRequest.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.query.ReserveStreamExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginStreamExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -88489,15 +88915,15 @@ export const query = $root.query = (() => { break; } case 6: { - message.transaction_id = reader.int64(); - break; - } - case 7: { if (!(message.pre_queries && message.pre_queries.length)) message.pre_queries = []; message.pre_queries.push(reader.string()); break; } + case 7: { + message.reserved_id = reader.int64(); + break; + } default: reader.skipType(tag & 7); break; @@ -88507,30 +88933,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a BeginStreamExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest + * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + BeginStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveStreamExecuteRequest message. + * Verifies a BeginStreamExecuteRequest message. * @function verify - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveStreamExecuteRequest.verify = function verify(message) { + BeginStreamExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -88558,9 +88984,6 @@ export const query = $root.query = (() => { if (error) return "options." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { if (!Array.isArray(message.pre_queries)) return "pre_queries: array expected"; @@ -88568,75 +88991,78 @@ export const query = $root.query = (() => { if (!$util.isString(message.pre_queries[i])) return "pre_queries: string[] expected"; } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; return null; }; /** - * Creates a ReserveStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest + * @returns {query.BeginStreamExecuteRequest} BeginStreamExecuteRequest */ - ReserveStreamExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveStreamExecuteRequest) + BeginStreamExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginStreamExecuteRequest) return object; - let message = new $root.query.ReserveStreamExecuteRequest(); + let message = new $root.query.BeginStreamExecuteRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReserveStreamExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.BeginStreamExecuteRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReserveStreamExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.BeginStreamExecuteRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReserveStreamExecuteRequest.target: object expected"); + throw TypeError(".query.BeginStreamExecuteRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } if (object.query != null) { if (typeof object.query !== "object") - throw TypeError(".query.ReserveStreamExecuteRequest.query: object expected"); + throw TypeError(".query.BeginStreamExecuteRequest.query: object expected"); message.query = $root.query.BoundQuery.fromObject(object.query); } if (object.options != null) { if (typeof object.options !== "object") - throw TypeError(".query.ReserveStreamExecuteRequest.options: object expected"); + throw TypeError(".query.BeginStreamExecuteRequest.options: object expected"); message.options = $root.query.ExecuteOptions.fromObject(object.options); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.pre_queries) { if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.ReserveStreamExecuteRequest.pre_queries: array expected"); + throw TypeError(".query.BeginStreamExecuteRequest.pre_queries: array expected"); message.pre_queries = []; for (let i = 0; i < object.pre_queries.length; ++i) message.pre_queries[i] = String(object.pre_queries[i]); } + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a ReserveStreamExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a BeginStreamExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static - * @param {query.ReserveStreamExecuteRequest} message ReserveStreamExecuteRequest + * @param {query.BeginStreamExecuteRequest} message BeginStreamExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveStreamExecuteRequest.toObject = function toObject(message, options) { + BeginStreamExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -88650,9 +89076,9 @@ export const query = $root.query = (() => { object.options = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.transaction_id = options.longs === String ? "0" : 0; + object.reserved_id = options.longs === String ? "0" : 0; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -88664,69 +89090,70 @@ export const query = $root.query = (() => { object.query = $root.query.BoundQuery.toObject(message.query, options); if (message.options != null && message.hasOwnProperty("options")) object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.pre_queries && message.pre_queries.length) { object.pre_queries = []; for (let j = 0; j < message.pre_queries.length; ++j) object.pre_queries[j] = message.pre_queries[j]; } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; return object; }; /** - * Converts this ReserveStreamExecuteRequest to JSON. + * Converts this BeginStreamExecuteRequest to JSON. * @function toJSON - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @instance * @returns {Object.} JSON object */ - ReserveStreamExecuteRequest.prototype.toJSON = function toJSON() { + BeginStreamExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveStreamExecuteRequest + * Gets the default type url for BeginStreamExecuteRequest * @function getTypeUrl - * @memberof query.ReserveStreamExecuteRequest + * @memberof query.BeginStreamExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveStreamExecuteRequest"; + return typeUrlPrefix + "/query.BeginStreamExecuteRequest"; }; - return ReserveStreamExecuteRequest; + return BeginStreamExecuteRequest; })(); - query.ReserveStreamExecuteResponse = (function() { + query.BeginStreamExecuteResponse = (function() { /** - * Properties of a ReserveStreamExecuteResponse. + * Properties of a BeginStreamExecuteResponse. * @memberof query - * @interface IReserveStreamExecuteResponse - * @property {vtrpc.IRPCError|null} [error] ReserveStreamExecuteResponse error - * @property {query.IQueryResult|null} [result] ReserveStreamExecuteResponse result - * @property {number|Long|null} [reserved_id] ReserveStreamExecuteResponse reserved_id - * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveStreamExecuteResponse tablet_alias + * @interface IBeginStreamExecuteResponse + * @property {vtrpc.IRPCError|null} [error] BeginStreamExecuteResponse error + * @property {query.IQueryResult|null} [result] BeginStreamExecuteResponse result + * @property {number|Long|null} [transaction_id] BeginStreamExecuteResponse transaction_id + * @property {topodata.ITabletAlias|null} [tablet_alias] BeginStreamExecuteResponse tablet_alias + * @property {string|null} [session_state_changes] BeginStreamExecuteResponse session_state_changes */ /** - * Constructs a new ReserveStreamExecuteResponse. + * Constructs a new BeginStreamExecuteResponse. * @memberof query - * @classdesc Represents a ReserveStreamExecuteResponse. - * @implements IReserveStreamExecuteResponse + * @classdesc Represents a BeginStreamExecuteResponse. + * @implements IBeginStreamExecuteResponse * @constructor - * @param {query.IReserveStreamExecuteResponse=} [properties] Properties to set + * @param {query.IBeginStreamExecuteResponse=} [properties] Properties to set */ - function ReserveStreamExecuteResponse(properties) { + function BeginStreamExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -88734,100 +89161,110 @@ export const query = $root.query = (() => { } /** - * ReserveStreamExecuteResponse error. + * BeginStreamExecuteResponse error. * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @instance */ - ReserveStreamExecuteResponse.prototype.error = null; + BeginStreamExecuteResponse.prototype.error = null; /** - * ReserveStreamExecuteResponse result. + * BeginStreamExecuteResponse result. * @member {query.IQueryResult|null|undefined} result - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @instance */ - ReserveStreamExecuteResponse.prototype.result = null; + BeginStreamExecuteResponse.prototype.result = null; /** - * ReserveStreamExecuteResponse reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ReserveStreamExecuteResponse + * BeginStreamExecuteResponse transaction_id. + * @member {number|Long} transaction_id + * @memberof query.BeginStreamExecuteResponse * @instance */ - ReserveStreamExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + BeginStreamExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ReserveStreamExecuteResponse tablet_alias. + * BeginStreamExecuteResponse tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @instance */ - ReserveStreamExecuteResponse.prototype.tablet_alias = null; + BeginStreamExecuteResponse.prototype.tablet_alias = null; /** - * Creates a new ReserveStreamExecuteResponse instance using the specified properties. + * BeginStreamExecuteResponse session_state_changes. + * @member {string} session_state_changes + * @memberof query.BeginStreamExecuteResponse + * @instance + */ + BeginStreamExecuteResponse.prototype.session_state_changes = ""; + + /** + * Creates a new BeginStreamExecuteResponse instance using the specified properties. * @function create - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static - * @param {query.IReserveStreamExecuteResponse=} [properties] Properties to set - * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse instance + * @param {query.IBeginStreamExecuteResponse=} [properties] Properties to set + * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse instance */ - ReserveStreamExecuteResponse.create = function create(properties) { - return new ReserveStreamExecuteResponse(properties); + BeginStreamExecuteResponse.create = function create(properties) { + return new BeginStreamExecuteResponse(properties); }; /** - * Encodes the specified ReserveStreamExecuteResponse message. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. + * Encodes the specified BeginStreamExecuteResponse message. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static - * @param {query.IReserveStreamExecuteResponse} message ReserveStreamExecuteResponse message or plain object to encode + * @param {query.IBeginStreamExecuteResponse} message BeginStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveStreamExecuteResponse.encode = function encode(message, writer) { + BeginStreamExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.error != null && Object.hasOwnProperty.call(message, "error")) $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.result != null && Object.hasOwnProperty.call(message, "result")) $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.reserved_id); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.session_state_changes); return writer; }; /** - * Encodes the specified ReserveStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. + * Encodes the specified BeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.BeginStreamExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static - * @param {query.IReserveStreamExecuteResponse} message ReserveStreamExecuteResponse message or plain object to encode + * @param {query.IBeginStreamExecuteResponse} message BeginStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + BeginStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer. + * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse + * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveStreamExecuteResponse.decode = function decode(reader, length) { + BeginStreamExecuteResponse.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.query.ReserveStreamExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.BeginStreamExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -88840,13 +89277,17 @@ export const query = $root.query = (() => { break; } case 3: { - message.reserved_id = reader.int64(); + message.transaction_id = reader.int64(); break; } case 4: { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } + case 5: { + message.session_state_changes = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -88856,30 +89297,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a BeginStreamExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse + * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + BeginStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveStreamExecuteResponse message. + * Verifies a BeginStreamExecuteResponse message. * @function verify - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveStreamExecuteResponse.verify = function verify(message) { + BeginStreamExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.error != null && message.hasOwnProperty("error")) { @@ -88892,66 +89333,71 @@ export const query = $root.query = (() => { if (error) return "result." + error; } - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) return "tablet_alias." + error; } + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a ReserveStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse + * @returns {query.BeginStreamExecuteResponse} BeginStreamExecuteResponse */ - ReserveStreamExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveStreamExecuteResponse) + BeginStreamExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.BeginStreamExecuteResponse) return object; - let message = new $root.query.ReserveStreamExecuteResponse(); + let message = new $root.query.BeginStreamExecuteResponse(); if (object.error != null) { if (typeof object.error !== "object") - throw TypeError(".query.ReserveStreamExecuteResponse.error: object expected"); + throw TypeError(".query.BeginStreamExecuteResponse.error: object expected"); message.error = $root.vtrpc.RPCError.fromObject(object.error); } if (object.result != null) { if (typeof object.result !== "object") - throw TypeError(".query.ReserveStreamExecuteResponse.result: object expected"); + throw TypeError(".query.BeginStreamExecuteResponse.result: object expected"); message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.reserved_id != null) + if (object.transaction_id != null) if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".query.ReserveStreamExecuteResponse.tablet_alias: object expected"); + throw TypeError(".query.BeginStreamExecuteResponse.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a ReserveStreamExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a BeginStreamExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static - * @param {query.ReserveStreamExecuteResponse} message ReserveStreamExecuteResponse + * @param {query.BeginStreamExecuteResponse} message BeginStreamExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveStreamExecuteResponse.toObject = function toObject(message, options) { + BeginStreamExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -88960,80 +89406,78 @@ export const query = $root.query = (() => { object.result = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.reserved_id = options.longs === String ? "0" : 0; + object.transaction_id = options.longs === String ? "0" : 0; object.tablet_alias = null; + object.session_state_changes = ""; } if (message.error != null && message.hasOwnProperty("error")) object.error = $root.vtrpc.RPCError.toObject(message.error, options); if (message.result != null && message.hasOwnProperty("result")) object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this ReserveStreamExecuteResponse to JSON. + * Converts this BeginStreamExecuteResponse to JSON. * @function toJSON - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @instance * @returns {Object.} JSON object */ - ReserveStreamExecuteResponse.prototype.toJSON = function toJSON() { + BeginStreamExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveStreamExecuteResponse + * Gets the default type url for BeginStreamExecuteResponse * @function getTypeUrl - * @memberof query.ReserveStreamExecuteResponse + * @memberof query.BeginStreamExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BeginStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveStreamExecuteResponse"; + return typeUrlPrefix + "/query.BeginStreamExecuteResponse"; }; - return ReserveStreamExecuteResponse; + return BeginStreamExecuteResponse; })(); - query.ReserveBeginExecuteRequest = (function() { + query.MessageStreamRequest = (function() { /** - * Properties of a ReserveBeginExecuteRequest. + * Properties of a MessageStreamRequest. * @memberof query - * @interface IReserveBeginExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveBeginExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveBeginExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReserveBeginExecuteRequest target - * @property {query.IBoundQuery|null} [query] ReserveBeginExecuteRequest query - * @property {query.IExecuteOptions|null} [options] ReserveBeginExecuteRequest options - * @property {Array.|null} [pre_queries] ReserveBeginExecuteRequest pre_queries - * @property {Array.|null} [post_begin_queries] ReserveBeginExecuteRequest post_begin_queries + * @interface IMessageStreamRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] MessageStreamRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] MessageStreamRequest immediate_caller_id + * @property {query.ITarget|null} [target] MessageStreamRequest target + * @property {string|null} [name] MessageStreamRequest name */ /** - * Constructs a new ReserveBeginExecuteRequest. + * Constructs a new MessageStreamRequest. * @memberof query - * @classdesc Represents a ReserveBeginExecuteRequest. - * @implements IReserveBeginExecuteRequest + * @classdesc Represents a MessageStreamRequest. + * @implements IMessageStreamRequest * @constructor - * @param {query.IReserveBeginExecuteRequest=} [properties] Properties to set + * @param {query.IMessageStreamRequest=} [properties] Properties to set */ - function ReserveBeginExecuteRequest(properties) { - this.pre_queries = []; - this.post_begin_queries = []; + function MessageStreamRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -89041,83 +89485,59 @@ export const query = $root.query = (() => { } /** - * ReserveBeginExecuteRequest effective_caller_id. + * MessageStreamRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @instance */ - ReserveBeginExecuteRequest.prototype.effective_caller_id = null; + MessageStreamRequest.prototype.effective_caller_id = null; /** - * ReserveBeginExecuteRequest immediate_caller_id. + * MessageStreamRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @instance */ - ReserveBeginExecuteRequest.prototype.immediate_caller_id = null; + MessageStreamRequest.prototype.immediate_caller_id = null; /** - * ReserveBeginExecuteRequest target. + * MessageStreamRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReserveBeginExecuteRequest - * @instance - */ - ReserveBeginExecuteRequest.prototype.target = null; - - /** - * ReserveBeginExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.ReserveBeginExecuteRequest - * @instance - */ - ReserveBeginExecuteRequest.prototype.query = null; - - /** - * ReserveBeginExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.ReserveBeginExecuteRequest - * @instance - */ - ReserveBeginExecuteRequest.prototype.options = null; - - /** - * ReserveBeginExecuteRequest pre_queries. - * @member {Array.} pre_queries - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @instance */ - ReserveBeginExecuteRequest.prototype.pre_queries = $util.emptyArray; + MessageStreamRequest.prototype.target = null; /** - * ReserveBeginExecuteRequest post_begin_queries. - * @member {Array.} post_begin_queries - * @memberof query.ReserveBeginExecuteRequest + * MessageStreamRequest name. + * @member {string} name + * @memberof query.MessageStreamRequest * @instance */ - ReserveBeginExecuteRequest.prototype.post_begin_queries = $util.emptyArray; + MessageStreamRequest.prototype.name = ""; /** - * Creates a new ReserveBeginExecuteRequest instance using the specified properties. + * Creates a new MessageStreamRequest instance using the specified properties. * @function create - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static - * @param {query.IReserveBeginExecuteRequest=} [properties] Properties to set - * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest instance + * @param {query.IMessageStreamRequest=} [properties] Properties to set + * @returns {query.MessageStreamRequest} MessageStreamRequest instance */ - ReserveBeginExecuteRequest.create = function create(properties) { - return new ReserveBeginExecuteRequest(properties); + MessageStreamRequest.create = function create(properties) { + return new MessageStreamRequest(properties); }; /** - * Encodes the specified ReserveBeginExecuteRequest message. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. + * Encodes the specified MessageStreamRequest message. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. * @function encode - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static - * @param {query.IReserveBeginExecuteRequest} message ReserveBeginExecuteRequest message or plain object to encode + * @param {query.IMessageStreamRequest} message MessageStreamRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginExecuteRequest.encode = function encode(message, writer) { + MessageStreamRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -89126,47 +89546,39 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.pre_queries != null && message.pre_queries.length) - for (let i = 0; i < message.pre_queries.length; ++i) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); - if (message.post_begin_queries != null && message.post_begin_queries.length) - for (let i = 0; i < message.post_begin_queries.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.post_begin_queries[i]); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.name); return writer; }; /** - * Encodes the specified ReserveBeginExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. + * Encodes the specified MessageStreamRequest message, length delimited. Does not implicitly {@link query.MessageStreamRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static - * @param {query.IReserveBeginExecuteRequest} message ReserveBeginExecuteRequest message or plain object to encode + * @param {query.IMessageStreamRequest} message MessageStreamRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + MessageStreamRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer. + * Decodes a MessageStreamRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest + * @returns {query.MessageStreamRequest} MessageStreamRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginExecuteRequest.decode = function decode(reader, length) { + MessageStreamRequest.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.query.ReserveBeginExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.MessageStreamRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -89183,23 +89595,7 @@ export const query = $root.query = (() => { break; } case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); - break; - } - case 5: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); - break; - } - case 6: { - if (!(message.pre_queries && message.pre_queries.length)) - message.pre_queries = []; - message.pre_queries.push(reader.string()); - break; - } - case 7: { - if (!(message.post_begin_queries && message.post_begin_queries.length)) - message.post_begin_queries = []; - message.post_begin_queries.push(reader.string()); + message.name = reader.string(); break; } default: @@ -89211,30 +89607,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a MessageStreamRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest + * @returns {query.MessageStreamRequest} MessageStreamRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + MessageStreamRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveBeginExecuteRequest message. + * Verifies a MessageStreamRequest message. * @function verify - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveBeginExecuteRequest.verify = function verify(message) { + MessageStreamRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -89252,110 +89648,62 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; - } - if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { - if (!Array.isArray(message.pre_queries)) - return "pre_queries: array expected"; - for (let i = 0; i < message.pre_queries.length; ++i) - if (!$util.isString(message.pre_queries[i])) - return "pre_queries: string[] expected"; - } - if (message.post_begin_queries != null && message.hasOwnProperty("post_begin_queries")) { - if (!Array.isArray(message.post_begin_queries)) - return "post_begin_queries: array expected"; - for (let i = 0; i < message.post_begin_queries.length; ++i) - if (!$util.isString(message.post_begin_queries[i])) - return "post_begin_queries: string[] expected"; - } + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; return null; }; /** - * Creates a ReserveBeginExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MessageStreamRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static * @param {Object.} object Plain object - * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest + * @returns {query.MessageStreamRequest} MessageStreamRequest */ - ReserveBeginExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveBeginExecuteRequest) + MessageStreamRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.MessageStreamRequest) return object; - let message = new $root.query.ReserveBeginExecuteRequest(); + let message = new $root.query.MessageStreamRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReserveBeginExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.MessageStreamRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReserveBeginExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.MessageStreamRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReserveBeginExecuteRequest.target: object expected"); + throw TypeError(".query.MessageStreamRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.ReserveBeginExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.ReserveBeginExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); - } - if (object.pre_queries) { - if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.ReserveBeginExecuteRequest.pre_queries: array expected"); - message.pre_queries = []; - for (let i = 0; i < object.pre_queries.length; ++i) - message.pre_queries[i] = String(object.pre_queries[i]); - } - if (object.post_begin_queries) { - if (!Array.isArray(object.post_begin_queries)) - throw TypeError(".query.ReserveBeginExecuteRequest.post_begin_queries: array expected"); - message.post_begin_queries = []; - for (let i = 0; i < object.post_begin_queries.length; ++i) - message.post_begin_queries[i] = String(object.post_begin_queries[i]); - } + if (object.name != null) + message.name = String(object.name); return message; }; /** - * Creates a plain object from a ReserveBeginExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a MessageStreamRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static - * @param {query.ReserveBeginExecuteRequest} message ReserveBeginExecuteRequest + * @param {query.MessageStreamRequest} message MessageStreamRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveBeginExecuteRequest.toObject = function toObject(message, options) { + MessageStreamRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.pre_queries = []; - object.post_begin_queries = []; - } if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.query = null; - object.options = null; + object.name = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -89363,75 +89711,58 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.pre_queries && message.pre_queries.length) { - object.pre_queries = []; - for (let j = 0; j < message.pre_queries.length; ++j) - object.pre_queries[j] = message.pre_queries[j]; - } - if (message.post_begin_queries && message.post_begin_queries.length) { - object.post_begin_queries = []; - for (let j = 0; j < message.post_begin_queries.length; ++j) - object.post_begin_queries[j] = message.post_begin_queries[j]; - } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; return object; }; /** - * Converts this ReserveBeginExecuteRequest to JSON. + * Converts this MessageStreamRequest to JSON. * @function toJSON - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @instance * @returns {Object.} JSON object */ - ReserveBeginExecuteRequest.prototype.toJSON = function toJSON() { + MessageStreamRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveBeginExecuteRequest + * Gets the default type url for MessageStreamRequest * @function getTypeUrl - * @memberof query.ReserveBeginExecuteRequest + * @memberof query.MessageStreamRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveBeginExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MessageStreamRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveBeginExecuteRequest"; + return typeUrlPrefix + "/query.MessageStreamRequest"; }; - return ReserveBeginExecuteRequest; + return MessageStreamRequest; })(); - query.ReserveBeginExecuteResponse = (function() { + query.MessageStreamResponse = (function() { /** - * Properties of a ReserveBeginExecuteResponse. + * Properties of a MessageStreamResponse. * @memberof query - * @interface IReserveBeginExecuteResponse - * @property {vtrpc.IRPCError|null} [error] ReserveBeginExecuteResponse error - * @property {query.IQueryResult|null} [result] ReserveBeginExecuteResponse result - * @property {number|Long|null} [transaction_id] ReserveBeginExecuteResponse transaction_id - * @property {number|Long|null} [reserved_id] ReserveBeginExecuteResponse reserved_id - * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveBeginExecuteResponse tablet_alias - * @property {string|null} [session_state_changes] ReserveBeginExecuteResponse session_state_changes + * @interface IMessageStreamResponse + * @property {query.IQueryResult|null} [result] MessageStreamResponse result */ /** - * Constructs a new ReserveBeginExecuteResponse. + * Constructs a new MessageStreamResponse. * @memberof query - * @classdesc Represents a ReserveBeginExecuteResponse. - * @implements IReserveBeginExecuteResponse + * @classdesc Represents a MessageStreamResponse. + * @implements IMessageStreamResponse * @constructor - * @param {query.IReserveBeginExecuteResponse=} [properties] Properties to set + * @param {query.IMessageStreamResponse=} [properties] Properties to set */ - function ReserveBeginExecuteResponse(properties) { + function MessageStreamResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -89439,147 +89770,77 @@ export const query = $root.query = (() => { } /** - * ReserveBeginExecuteResponse error. - * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.ReserveBeginExecuteResponse - * @instance - */ - ReserveBeginExecuteResponse.prototype.error = null; - - /** - * ReserveBeginExecuteResponse result. + * MessageStreamResponse result. * @member {query.IQueryResult|null|undefined} result - * @memberof query.ReserveBeginExecuteResponse - * @instance - */ - ReserveBeginExecuteResponse.prototype.result = null; - - /** - * ReserveBeginExecuteResponse transaction_id. - * @member {number|Long} transaction_id - * @memberof query.ReserveBeginExecuteResponse - * @instance - */ - ReserveBeginExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * ReserveBeginExecuteResponse reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ReserveBeginExecuteResponse - * @instance - */ - ReserveBeginExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * ReserveBeginExecuteResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.ReserveBeginExecuteResponse - * @instance - */ - ReserveBeginExecuteResponse.prototype.tablet_alias = null; - - /** - * ReserveBeginExecuteResponse session_state_changes. - * @member {string} session_state_changes - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @instance */ - ReserveBeginExecuteResponse.prototype.session_state_changes = ""; + MessageStreamResponse.prototype.result = null; /** - * Creates a new ReserveBeginExecuteResponse instance using the specified properties. + * Creates a new MessageStreamResponse instance using the specified properties. * @function create - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static - * @param {query.IReserveBeginExecuteResponse=} [properties] Properties to set - * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse instance + * @param {query.IMessageStreamResponse=} [properties] Properties to set + * @returns {query.MessageStreamResponse} MessageStreamResponse instance */ - ReserveBeginExecuteResponse.create = function create(properties) { - return new ReserveBeginExecuteResponse(properties); + MessageStreamResponse.create = function create(properties) { + return new MessageStreamResponse(properties); }; /** - * Encodes the specified ReserveBeginExecuteResponse message. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. + * Encodes the specified MessageStreamResponse message. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. * @function encode - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static - * @param {query.IReserveBeginExecuteResponse} message ReserveBeginExecuteResponse message or plain object to encode + * @param {query.IMessageStreamResponse} message MessageStreamResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginExecuteResponse.encode = function encode(message, writer) { + MessageStreamResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.reserved_id); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.session_state_changes); + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReserveBeginExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. + * Encodes the specified MessageStreamResponse message, length delimited. Does not implicitly {@link query.MessageStreamResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static - * @param {query.IReserveBeginExecuteResponse} message ReserveBeginExecuteResponse message or plain object to encode + * @param {query.IMessageStreamResponse} message MessageStreamResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + MessageStreamResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer. + * Decodes a MessageStreamResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse + * @returns {query.MessageStreamResponse} MessageStreamResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginExecuteResponse.decode = function decode(reader, length) { + MessageStreamResponse.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.query.ReserveBeginExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.MessageStreamResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); - break; - } - case 2: { message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } - case 3: { - message.transaction_id = reader.int64(); - break; - } - case 4: { - message.reserved_id = reader.int64(); - break; - } - case 5: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 6: { - message.session_state_changes = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -89589,214 +89850,132 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a MessageStreamResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse + * @returns {query.MessageStreamResponse} MessageStreamResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + MessageStreamResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveBeginExecuteResponse message. + * Verifies a MessageStreamResponse message. * @function verify - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveBeginExecuteResponse.verify = function verify(message) { + MessageStreamResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.vtrpc.RPCError.verify(message.error); - if (error) - return "error." + error; - } if (message.result != null && message.hasOwnProperty("result")) { let error = $root.query.QueryResult.verify(message.result); if (error) return "result." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; return null; }; /** - * Creates a ReserveBeginExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MessageStreamResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static * @param {Object.} object Plain object - * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse + * @returns {query.MessageStreamResponse} MessageStreamResponse */ - ReserveBeginExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveBeginExecuteResponse) + MessageStreamResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.MessageStreamResponse) return object; - let message = new $root.query.ReserveBeginExecuteResponse(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".query.ReserveBeginExecuteResponse.error: object expected"); - message.error = $root.vtrpc.RPCError.fromObject(object.error); - } + let message = new $root.query.MessageStreamResponse(); if (object.result != null) { if (typeof object.result !== "object") - throw TypeError(".query.ReserveBeginExecuteResponse.result: object expected"); + throw TypeError(".query.MessageStreamResponse.result: object expected"); message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".query.ReserveBeginExecuteResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a ReserveBeginExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a MessageStreamResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static - * @param {query.ReserveBeginExecuteResponse} message ReserveBeginExecuteResponse + * @param {query.MessageStreamResponse} message MessageStreamResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveBeginExecuteResponse.toObject = function toObject(message, options) { + MessageStreamResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.error = null; + if (options.defaults) object.result = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.reserved_id = options.longs === String ? "0" : 0; - object.tablet_alias = null; - object.session_state_changes = ""; - } - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.vtrpc.RPCError.toObject(message.error, options); if (message.result != null && message.hasOwnProperty("result")) object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; - else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this ReserveBeginExecuteResponse to JSON. + * Converts this MessageStreamResponse to JSON. * @function toJSON - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @instance * @returns {Object.} JSON object */ - ReserveBeginExecuteResponse.prototype.toJSON = function toJSON() { + MessageStreamResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveBeginExecuteResponse + * Gets the default type url for MessageStreamResponse * @function getTypeUrl - * @memberof query.ReserveBeginExecuteResponse + * @memberof query.MessageStreamResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveBeginExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MessageStreamResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveBeginExecuteResponse"; + return typeUrlPrefix + "/query.MessageStreamResponse"; }; - return ReserveBeginExecuteResponse; + return MessageStreamResponse; })(); - query.ReserveBeginStreamExecuteRequest = (function() { + query.MessageAckRequest = (function() { /** - * Properties of a ReserveBeginStreamExecuteRequest. + * Properties of a MessageAckRequest. * @memberof query - * @interface IReserveBeginStreamExecuteRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveBeginStreamExecuteRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveBeginStreamExecuteRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReserveBeginStreamExecuteRequest target - * @property {query.IBoundQuery|null} [query] ReserveBeginStreamExecuteRequest query - * @property {query.IExecuteOptions|null} [options] ReserveBeginStreamExecuteRequest options - * @property {Array.|null} [pre_queries] ReserveBeginStreamExecuteRequest pre_queries - * @property {Array.|null} [post_begin_queries] ReserveBeginStreamExecuteRequest post_begin_queries + * @interface IMessageAckRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] MessageAckRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] MessageAckRequest immediate_caller_id + * @property {query.ITarget|null} [target] MessageAckRequest target + * @property {string|null} [name] MessageAckRequest name + * @property {Array.|null} [ids] MessageAckRequest ids */ /** - * Constructs a new ReserveBeginStreamExecuteRequest. + * Constructs a new MessageAckRequest. * @memberof query - * @classdesc Represents a ReserveBeginStreamExecuteRequest. - * @implements IReserveBeginStreamExecuteRequest + * @classdesc Represents a MessageAckRequest. + * @implements IMessageAckRequest * @constructor - * @param {query.IReserveBeginStreamExecuteRequest=} [properties] Properties to set + * @param {query.IMessageAckRequest=} [properties] Properties to set */ - function ReserveBeginStreamExecuteRequest(properties) { - this.pre_queries = []; - this.post_begin_queries = []; + function MessageAckRequest(properties) { + this.ids = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -89804,83 +89983,67 @@ export const query = $root.query = (() => { } /** - * ReserveBeginStreamExecuteRequest effective_caller_id. + * MessageAckRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @instance */ - ReserveBeginStreamExecuteRequest.prototype.effective_caller_id = null; + MessageAckRequest.prototype.effective_caller_id = null; /** - * ReserveBeginStreamExecuteRequest immediate_caller_id. + * MessageAckRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @instance */ - ReserveBeginStreamExecuteRequest.prototype.immediate_caller_id = null; + MessageAckRequest.prototype.immediate_caller_id = null; /** - * ReserveBeginStreamExecuteRequest target. + * MessageAckRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReserveBeginStreamExecuteRequest - * @instance - */ - ReserveBeginStreamExecuteRequest.prototype.target = null; - - /** - * ReserveBeginStreamExecuteRequest query. - * @member {query.IBoundQuery|null|undefined} query - * @memberof query.ReserveBeginStreamExecuteRequest - * @instance - */ - ReserveBeginStreamExecuteRequest.prototype.query = null; - - /** - * ReserveBeginStreamExecuteRequest options. - * @member {query.IExecuteOptions|null|undefined} options - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @instance */ - ReserveBeginStreamExecuteRequest.prototype.options = null; + MessageAckRequest.prototype.target = null; /** - * ReserveBeginStreamExecuteRequest pre_queries. - * @member {Array.} pre_queries - * @memberof query.ReserveBeginStreamExecuteRequest + * MessageAckRequest name. + * @member {string} name + * @memberof query.MessageAckRequest * @instance */ - ReserveBeginStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; + MessageAckRequest.prototype.name = ""; /** - * ReserveBeginStreamExecuteRequest post_begin_queries. - * @member {Array.} post_begin_queries - * @memberof query.ReserveBeginStreamExecuteRequest + * MessageAckRequest ids. + * @member {Array.} ids + * @memberof query.MessageAckRequest * @instance */ - ReserveBeginStreamExecuteRequest.prototype.post_begin_queries = $util.emptyArray; + MessageAckRequest.prototype.ids = $util.emptyArray; /** - * Creates a new ReserveBeginStreamExecuteRequest instance using the specified properties. + * Creates a new MessageAckRequest instance using the specified properties. * @function create - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static - * @param {query.IReserveBeginStreamExecuteRequest=} [properties] Properties to set - * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest instance + * @param {query.IMessageAckRequest=} [properties] Properties to set + * @returns {query.MessageAckRequest} MessageAckRequest instance */ - ReserveBeginStreamExecuteRequest.create = function create(properties) { - return new ReserveBeginStreamExecuteRequest(properties); + MessageAckRequest.create = function create(properties) { + return new MessageAckRequest(properties); }; /** - * Encodes the specified ReserveBeginStreamExecuteRequest message. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. + * Encodes the specified MessageAckRequest message. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. * @function encode - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static - * @param {query.IReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest message or plain object to encode + * @param {query.IMessageAckRequest} message MessageAckRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginStreamExecuteRequest.encode = function encode(message, writer) { + MessageAckRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -89889,47 +90052,42 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.pre_queries != null && message.pre_queries.length) - for (let i = 0; i < message.pre_queries.length; ++i) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); - if (message.post_begin_queries != null && message.post_begin_queries.length) - for (let i = 0; i < message.post_begin_queries.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.post_begin_queries[i]); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.name); + if (message.ids != null && message.ids.length) + for (let i = 0; i < message.ids.length; ++i) + $root.query.Value.encode(message.ids[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReserveBeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. + * Encodes the specified MessageAckRequest message, length delimited. Does not implicitly {@link query.MessageAckRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static - * @param {query.IReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest message or plain object to encode + * @param {query.IMessageAckRequest} message MessageAckRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { + MessageAckRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer. + * Decodes a MessageAckRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest + * @returns {query.MessageAckRequest} MessageAckRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginStreamExecuteRequest.decode = function decode(reader, length) { + MessageAckRequest.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.query.ReserveBeginStreamExecuteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.MessageAckRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -89946,23 +90104,13 @@ export const query = $root.query = (() => { break; } case 4: { - message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + message.name = reader.string(); break; } case 5: { - message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); - break; - } - case 6: { - if (!(message.pre_queries && message.pre_queries.length)) - message.pre_queries = []; - message.pre_queries.push(reader.string()); - break; - } - case 7: { - if (!(message.post_begin_queries && message.post_begin_queries.length)) - message.post_begin_queries = []; - message.post_begin_queries.push(reader.string()); + if (!(message.ids && message.ids.length)) + message.ids = []; + message.ids.push($root.query.Value.decode(reader, reader.uint32())); break; } default: @@ -89974,30 +90122,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer, length delimited. + * Decodes a MessageAckRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest + * @returns {query.MessageAckRequest} MessageAckRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { + MessageAckRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveBeginStreamExecuteRequest message. + * Verifies a MessageAckRequest message. * @function verify - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveBeginStreamExecuteRequest.verify = function verify(message) { + MessageAckRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -90015,110 +90163,83 @@ export const query = $root.query = (() => { if (error) return "target." + error; } - if (message.query != null && message.hasOwnProperty("query")) { - let error = $root.query.BoundQuery.verify(message.query); - if (error) - return "query." + error; - } - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.query.ExecuteOptions.verify(message.options); - if (error) - return "options." + error; - } - if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { - if (!Array.isArray(message.pre_queries)) - return "pre_queries: array expected"; - for (let i = 0; i < message.pre_queries.length; ++i) - if (!$util.isString(message.pre_queries[i])) - return "pre_queries: string[] expected"; - } - if (message.post_begin_queries != null && message.hasOwnProperty("post_begin_queries")) { - if (!Array.isArray(message.post_begin_queries)) - return "post_begin_queries: array expected"; - for (let i = 0; i < message.post_begin_queries.length; ++i) - if (!$util.isString(message.post_begin_queries[i])) - return "post_begin_queries: string[] expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.ids != null && message.hasOwnProperty("ids")) { + if (!Array.isArray(message.ids)) + return "ids: array expected"; + for (let i = 0; i < message.ids.length; ++i) { + let error = $root.query.Value.verify(message.ids[i]); + if (error) + return "ids." + error; + } } return null; }; /** - * Creates a ReserveBeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MessageAckRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static * @param {Object.} object Plain object - * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest + * @returns {query.MessageAckRequest} MessageAckRequest */ - ReserveBeginStreamExecuteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveBeginStreamExecuteRequest) + MessageAckRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.MessageAckRequest) return object; - let message = new $root.query.ReserveBeginStreamExecuteRequest(); + let message = new $root.query.MessageAckRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteRequest.effective_caller_id: object expected"); + throw TypeError(".query.MessageAckRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteRequest.immediate_caller_id: object expected"); + throw TypeError(".query.MessageAckRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteRequest.target: object expected"); + throw TypeError(".query.MessageAckRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } - if (object.query != null) { - if (typeof object.query !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteRequest.query: object expected"); - message.query = $root.query.BoundQuery.fromObject(object.query); - } - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteRequest.options: object expected"); - message.options = $root.query.ExecuteOptions.fromObject(object.options); - } - if (object.pre_queries) { - if (!Array.isArray(object.pre_queries)) - throw TypeError(".query.ReserveBeginStreamExecuteRequest.pre_queries: array expected"); - message.pre_queries = []; - for (let i = 0; i < object.pre_queries.length; ++i) - message.pre_queries[i] = String(object.pre_queries[i]); - } - if (object.post_begin_queries) { - if (!Array.isArray(object.post_begin_queries)) - throw TypeError(".query.ReserveBeginStreamExecuteRequest.post_begin_queries: array expected"); - message.post_begin_queries = []; - for (let i = 0; i < object.post_begin_queries.length; ++i) - message.post_begin_queries[i] = String(object.post_begin_queries[i]); + if (object.name != null) + message.name = String(object.name); + if (object.ids) { + if (!Array.isArray(object.ids)) + throw TypeError(".query.MessageAckRequest.ids: array expected"); + message.ids = []; + for (let i = 0; i < object.ids.length; ++i) { + if (typeof object.ids[i] !== "object") + throw TypeError(".query.MessageAckRequest.ids: object expected"); + message.ids[i] = $root.query.Value.fromObject(object.ids[i]); + } } return message; }; /** - * Creates a plain object from a ReserveBeginStreamExecuteRequest message. Also converts values to other types if specified. + * Creates a plain object from a MessageAckRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static - * @param {query.ReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest + * @param {query.MessageAckRequest} message MessageAckRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveBeginStreamExecuteRequest.toObject = function toObject(message, options) { + MessageAckRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.pre_queries = []; - object.post_begin_queries = []; - } + if (options.arrays || options.defaults) + object.ids = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; - object.query = null; - object.options = null; + object.name = ""; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -90126,75 +90247,63 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = $root.query.BoundQuery.toObject(message.query, options); - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.query.ExecuteOptions.toObject(message.options, options); - if (message.pre_queries && message.pre_queries.length) { - object.pre_queries = []; - for (let j = 0; j < message.pre_queries.length; ++j) - object.pre_queries[j] = message.pre_queries[j]; - } - if (message.post_begin_queries && message.post_begin_queries.length) { - object.post_begin_queries = []; - for (let j = 0; j < message.post_begin_queries.length; ++j) - object.post_begin_queries[j] = message.post_begin_queries[j]; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.ids && message.ids.length) { + object.ids = []; + for (let j = 0; j < message.ids.length; ++j) + object.ids[j] = $root.query.Value.toObject(message.ids[j], options); } return object; }; /** - * Converts this ReserveBeginStreamExecuteRequest to JSON. + * Converts this MessageAckRequest to JSON. * @function toJSON - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @instance * @returns {Object.} JSON object */ - ReserveBeginStreamExecuteRequest.prototype.toJSON = function toJSON() { + MessageAckRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveBeginStreamExecuteRequest + * Gets the default type url for MessageAckRequest * @function getTypeUrl - * @memberof query.ReserveBeginStreamExecuteRequest + * @memberof query.MessageAckRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveBeginStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MessageAckRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveBeginStreamExecuteRequest"; + return typeUrlPrefix + "/query.MessageAckRequest"; }; - return ReserveBeginStreamExecuteRequest; + return MessageAckRequest; })(); - query.ReserveBeginStreamExecuteResponse = (function() { + query.MessageAckResponse = (function() { /** - * Properties of a ReserveBeginStreamExecuteResponse. + * Properties of a MessageAckResponse. * @memberof query - * @interface IReserveBeginStreamExecuteResponse - * @property {vtrpc.IRPCError|null} [error] ReserveBeginStreamExecuteResponse error - * @property {query.IQueryResult|null} [result] ReserveBeginStreamExecuteResponse result - * @property {number|Long|null} [transaction_id] ReserveBeginStreamExecuteResponse transaction_id - * @property {number|Long|null} [reserved_id] ReserveBeginStreamExecuteResponse reserved_id - * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveBeginStreamExecuteResponse tablet_alias - * @property {string|null} [session_state_changes] ReserveBeginStreamExecuteResponse session_state_changes + * @interface IMessageAckResponse + * @property {query.IQueryResult|null} [result] MessageAckResponse result */ /** - * Constructs a new ReserveBeginStreamExecuteResponse. + * Constructs a new MessageAckResponse. * @memberof query - * @classdesc Represents a ReserveBeginStreamExecuteResponse. - * @implements IReserveBeginStreamExecuteResponse + * @classdesc Represents a MessageAckResponse. + * @implements IMessageAckResponse * @constructor - * @param {query.IReserveBeginStreamExecuteResponse=} [properties] Properties to set + * @param {query.IMessageAckResponse=} [properties] Properties to set */ - function ReserveBeginStreamExecuteResponse(properties) { + function MessageAckResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -90202,147 +90311,77 @@ export const query = $root.query = (() => { } /** - * ReserveBeginStreamExecuteResponse error. - * @member {vtrpc.IRPCError|null|undefined} error - * @memberof query.ReserveBeginStreamExecuteResponse - * @instance - */ - ReserveBeginStreamExecuteResponse.prototype.error = null; - - /** - * ReserveBeginStreamExecuteResponse result. + * MessageAckResponse result. * @member {query.IQueryResult|null|undefined} result - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @instance */ - ReserveBeginStreamExecuteResponse.prototype.result = null; + MessageAckResponse.prototype.result = null; /** - * ReserveBeginStreamExecuteResponse transaction_id. - * @member {number|Long} transaction_id - * @memberof query.ReserveBeginStreamExecuteResponse - * @instance + * Creates a new MessageAckResponse instance using the specified properties. + * @function create + * @memberof query.MessageAckResponse + * @static + * @param {query.IMessageAckResponse=} [properties] Properties to set + * @returns {query.MessageAckResponse} MessageAckResponse instance */ - ReserveBeginStreamExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + MessageAckResponse.create = function create(properties) { + return new MessageAckResponse(properties); + }; /** - * ReserveBeginStreamExecuteResponse reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ReserveBeginStreamExecuteResponse - * @instance + * Encodes the specified MessageAckResponse message. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. + * @function encode + * @memberof query.MessageAckResponse + * @static + * @param {query.IMessageAckResponse} message MessageAckResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer */ - ReserveBeginStreamExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + MessageAckResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; /** - * ReserveBeginStreamExecuteResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.ReserveBeginStreamExecuteResponse - * @instance - */ - ReserveBeginStreamExecuteResponse.prototype.tablet_alias = null; - - /** - * ReserveBeginStreamExecuteResponse session_state_changes. - * @member {string} session_state_changes - * @memberof query.ReserveBeginStreamExecuteResponse - * @instance - */ - ReserveBeginStreamExecuteResponse.prototype.session_state_changes = ""; - - /** - * Creates a new ReserveBeginStreamExecuteResponse instance using the specified properties. - * @function create - * @memberof query.ReserveBeginStreamExecuteResponse - * @static - * @param {query.IReserveBeginStreamExecuteResponse=} [properties] Properties to set - * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse instance - */ - ReserveBeginStreamExecuteResponse.create = function create(properties) { - return new ReserveBeginStreamExecuteResponse(properties); - }; - - /** - * Encodes the specified ReserveBeginStreamExecuteResponse message. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. - * @function encode - * @memberof query.ReserveBeginStreamExecuteResponse - * @static - * @param {query.IReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ReserveBeginStreamExecuteResponse.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.reserved_id); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.session_state_changes); - return writer; - }; - - /** - * Encodes the specified ReserveBeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. + * Encodes the specified MessageAckResponse message, length delimited. Does not implicitly {@link query.MessageAckResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static - * @param {query.IReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse message or plain object to encode + * @param {query.IMessageAckResponse} message MessageAckResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReserveBeginStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { + MessageAckResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer. + * Decodes a MessageAckResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse + * @returns {query.MessageAckResponse} MessageAckResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginStreamExecuteResponse.decode = function decode(reader, length) { + MessageAckResponse.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.query.ReserveBeginStreamExecuteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.MessageAckResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); - break; - } - case 2: { message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } - case 3: { - message.transaction_id = reader.int64(); - break; - } - case 4: { - message.reserved_id = reader.int64(); - break; - } - case 5: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 6: { - message.session_state_changes = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -90352,210 +90391,134 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer, length delimited. + * Decodes a MessageAckResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse + * @returns {query.MessageAckResponse} MessageAckResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReserveBeginStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { + MessageAckResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReserveBeginStreamExecuteResponse message. + * Verifies a MessageAckResponse message. * @function verify - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReserveBeginStreamExecuteResponse.verify = function verify(message) { + MessageAckResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.vtrpc.RPCError.verify(message.error); - if (error) - return "error." + error; - } if (message.result != null && message.hasOwnProperty("result")) { let error = $root.query.QueryResult.verify(message.result); if (error) return "result." + error; } - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) - return "transaction_id: integer|Long expected"; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - if (!$util.isString(message.session_state_changes)) - return "session_state_changes: string expected"; return null; }; /** - * Creates a ReserveBeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MessageAckResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static * @param {Object.} object Plain object - * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse + * @returns {query.MessageAckResponse} MessageAckResponse */ - ReserveBeginStreamExecuteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReserveBeginStreamExecuteResponse) + MessageAckResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.MessageAckResponse) return object; - let message = new $root.query.ReserveBeginStreamExecuteResponse(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteResponse.error: object expected"); - message.error = $root.vtrpc.RPCError.fromObject(object.error); - } + let message = new $root.query.MessageAckResponse(); if (object.result != null) { if (typeof object.result !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteResponse.result: object expected"); + throw TypeError(".query.MessageAckResponse.result: object expected"); message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.transaction_id != null) - if ($util.Long) - (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; - else if (typeof object.transaction_id === "string") - message.transaction_id = parseInt(object.transaction_id, 10); - else if (typeof object.transaction_id === "number") - message.transaction_id = object.transaction_id; - else if (typeof object.transaction_id === "object") - message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".query.ReserveBeginStreamExecuteResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.session_state_changes != null) - message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a ReserveBeginStreamExecuteResponse message. Also converts values to other types if specified. + * Creates a plain object from a MessageAckResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static - * @param {query.ReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse + * @param {query.MessageAckResponse} message MessageAckResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReserveBeginStreamExecuteResponse.toObject = function toObject(message, options) { + MessageAckResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.error = null; + if (options.defaults) object.result = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.transaction_id = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.reserved_id = options.longs === String ? "0" : 0; - object.tablet_alias = null; - object.session_state_changes = ""; - } - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.vtrpc.RPCError.toObject(message.error, options); if (message.result != null && message.hasOwnProperty("result")) object.result = $root.query.QueryResult.toObject(message.result, options); - if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) - if (typeof message.transaction_id === "number") - object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; - else - object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; - else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) - object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this ReserveBeginStreamExecuteResponse to JSON. + * Converts this MessageAckResponse to JSON. * @function toJSON - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @instance * @returns {Object.} JSON object */ - ReserveBeginStreamExecuteResponse.prototype.toJSON = function toJSON() { + MessageAckResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReserveBeginStreamExecuteResponse + * Gets the default type url for MessageAckResponse * @function getTypeUrl - * @memberof query.ReserveBeginStreamExecuteResponse + * @memberof query.MessageAckResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReserveBeginStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MessageAckResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReserveBeginStreamExecuteResponse"; + return typeUrlPrefix + "/query.MessageAckResponse"; }; - return ReserveBeginStreamExecuteResponse; + return MessageAckResponse; })(); - query.ReleaseRequest = (function() { + query.ReserveExecuteRequest = (function() { /** - * Properties of a ReleaseRequest. + * Properties of a ReserveExecuteRequest. * @memberof query - * @interface IReleaseRequest - * @property {vtrpc.ICallerID|null} [effective_caller_id] ReleaseRequest effective_caller_id - * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReleaseRequest immediate_caller_id - * @property {query.ITarget|null} [target] ReleaseRequest target - * @property {number|Long|null} [transaction_id] ReleaseRequest transaction_id - * @property {number|Long|null} [reserved_id] ReleaseRequest reserved_id + * @interface IReserveExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReserveExecuteRequest target + * @property {query.IBoundQuery|null} [query] ReserveExecuteRequest query + * @property {number|Long|null} [transaction_id] ReserveExecuteRequest transaction_id + * @property {query.IExecuteOptions|null} [options] ReserveExecuteRequest options + * @property {Array.|null} [pre_queries] ReserveExecuteRequest pre_queries */ /** - * Constructs a new ReleaseRequest. + * Constructs a new ReserveExecuteRequest. * @memberof query - * @classdesc Represents a ReleaseRequest. - * @implements IReleaseRequest + * @classdesc Represents a ReserveExecuteRequest. + * @implements IReserveExecuteRequest * @constructor - * @param {query.IReleaseRequest=} [properties] Properties to set + * @param {query.IReserveExecuteRequest=} [properties] Properties to set */ - function ReleaseRequest(properties) { + function ReserveExecuteRequest(properties) { + this.pre_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -90563,67 +90526,83 @@ export const query = $root.query = (() => { } /** - * ReleaseRequest effective_caller_id. + * ReserveExecuteRequest effective_caller_id. * @member {vtrpc.ICallerID|null|undefined} effective_caller_id - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @instance */ - ReleaseRequest.prototype.effective_caller_id = null; + ReserveExecuteRequest.prototype.effective_caller_id = null; /** - * ReleaseRequest immediate_caller_id. + * ReserveExecuteRequest immediate_caller_id. * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @instance */ - ReleaseRequest.prototype.immediate_caller_id = null; + ReserveExecuteRequest.prototype.immediate_caller_id = null; /** - * ReleaseRequest target. + * ReserveExecuteRequest target. * @member {query.ITarget|null|undefined} target - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @instance */ - ReleaseRequest.prototype.target = null; + ReserveExecuteRequest.prototype.target = null; /** - * ReleaseRequest transaction_id. + * ReserveExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.ReserveExecuteRequest + * @instance + */ + ReserveExecuteRequest.prototype.query = null; + + /** + * ReserveExecuteRequest transaction_id. * @member {number|Long} transaction_id - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @instance */ - ReleaseRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ReserveExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ReleaseRequest reserved_id. - * @member {number|Long} reserved_id - * @memberof query.ReleaseRequest + * ReserveExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.ReserveExecuteRequest * @instance */ - ReleaseRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ReserveExecuteRequest.prototype.options = null; /** - * Creates a new ReleaseRequest instance using the specified properties. + * ReserveExecuteRequest pre_queries. + * @member {Array.} pre_queries + * @memberof query.ReserveExecuteRequest + * @instance + */ + ReserveExecuteRequest.prototype.pre_queries = $util.emptyArray; + + /** + * Creates a new ReserveExecuteRequest instance using the specified properties. * @function create - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static - * @param {query.IReleaseRequest=} [properties] Properties to set - * @returns {query.ReleaseRequest} ReleaseRequest instance + * @param {query.IReserveExecuteRequest=} [properties] Properties to set + * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest instance */ - ReleaseRequest.create = function create(properties) { - return new ReleaseRequest(properties); + ReserveExecuteRequest.create = function create(properties) { + return new ReserveExecuteRequest(properties); }; /** - * Encodes the specified ReleaseRequest message. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. + * Encodes the specified ReserveExecuteRequest message. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static - * @param {query.IReleaseRequest} message ReleaseRequest message or plain object to encode + * @param {query.IReserveExecuteRequest} message ReserveExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReleaseRequest.encode = function encode(message, writer) { + ReserveExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) @@ -90632,41 +90611,46 @@ export const query = $root.query = (() => { $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.target != null && Object.hasOwnProperty.call(message, "target")) $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); - if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.reserved_id); + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.transaction_id); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.pre_queries != null && message.pre_queries.length) + for (let i = 0; i < message.pre_queries.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.pre_queries[i]); return writer; }; /** - * Encodes the specified ReleaseRequest message, length delimited. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. + * Encodes the specified ReserveExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static - * @param {query.IReleaseRequest} message ReleaseRequest message or plain object to encode + * @param {query.IReserveExecuteRequest} message ReserveExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReleaseRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReserveExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReleaseRequest message from the specified reader or buffer. + * Decodes a ReserveExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReleaseRequest} ReleaseRequest + * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReleaseRequest.decode = function decode(reader, length) { + ReserveExecuteRequest.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.query.ReleaseRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -90683,11 +90667,21 @@ export const query = $root.query = (() => { break; } case 4: { - message.transaction_id = reader.int64(); + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); break; } case 5: { - message.reserved_id = reader.int64(); + message.transaction_id = reader.int64(); + break; + } + case 6: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 7: { + if (!(message.pre_queries && message.pre_queries.length)) + message.pre_queries = []; + message.pre_queries.push(reader.string()); break; } default: @@ -90699,30 +90693,30 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReleaseRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReleaseRequest} ReleaseRequest + * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReleaseRequest.decodeDelimited = function decodeDelimited(reader) { + ReserveExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReleaseRequest message. + * Verifies a ReserveExecuteRequest message. * @function verify - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReleaseRequest.verify = function verify(message) { + ReserveExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { @@ -90740,42 +90734,61 @@ export const query = $root.query = (() => { if (error) return "target." + error; } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) return "transaction_id: integer|Long expected"; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) - return "reserved_id: integer|Long expected"; + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { + if (!Array.isArray(message.pre_queries)) + return "pre_queries: array expected"; + for (let i = 0; i < message.pre_queries.length; ++i) + if (!$util.isString(message.pre_queries[i])) + return "pre_queries: string[] expected"; + } return null; }; /** - * Creates a ReleaseRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.ReleaseRequest} ReleaseRequest + * @returns {query.ReserveExecuteRequest} ReserveExecuteRequest */ - ReleaseRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReleaseRequest) + ReserveExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveExecuteRequest) return object; - let message = new $root.query.ReleaseRequest(); + let message = new $root.query.ReserveExecuteRequest(); if (object.effective_caller_id != null) { if (typeof object.effective_caller_id !== "object") - throw TypeError(".query.ReleaseRequest.effective_caller_id: object expected"); + throw TypeError(".query.ReserveExecuteRequest.effective_caller_id: object expected"); message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } if (object.immediate_caller_id != null) { if (typeof object.immediate_caller_id !== "object") - throw TypeError(".query.ReleaseRequest.immediate_caller_id: object expected"); + throw TypeError(".query.ReserveExecuteRequest.immediate_caller_id: object expected"); message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); } if (object.target != null) { if (typeof object.target !== "object") - throw TypeError(".query.ReleaseRequest.target: object expected"); + throw TypeError(".query.ReserveExecuteRequest.target: object expected"); message.target = $root.query.Target.fromObject(object.target); } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.ReserveExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } if (object.transaction_id != null) if ($util.Long) (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; @@ -90785,45 +90798,47 @@ export const query = $root.query = (() => { message.transaction_id = object.transaction_id; else if (typeof object.transaction_id === "object") message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); - if (object.reserved_id != null) - if ($util.Long) - (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; - else if (typeof object.reserved_id === "string") - message.reserved_id = parseInt(object.reserved_id, 10); - else if (typeof object.reserved_id === "number") - message.reserved_id = object.reserved_id; - else if (typeof object.reserved_id === "object") - message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.ReserveExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.pre_queries) { + if (!Array.isArray(object.pre_queries)) + throw TypeError(".query.ReserveExecuteRequest.pre_queries: array expected"); + message.pre_queries = []; + for (let i = 0; i < object.pre_queries.length; ++i) + message.pre_queries[i] = String(object.pre_queries[i]); + } return message; }; /** - * Creates a plain object from a ReleaseRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReserveExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static - * @param {query.ReleaseRequest} message ReleaseRequest + * @param {query.ReserveExecuteRequest} message ReserveExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReleaseRequest.toObject = function toObject(message, options) { + ReserveExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.pre_queries = []; if (options.defaults) { object.effective_caller_id = null; object.immediate_caller_id = null; object.target = null; + object.query = null; if ($util.Long) { let long = new $util.Long(0, 0, false); object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else object.transaction_id = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.reserved_id = options.longs === String ? "0" : 0; + object.options = null; } if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); @@ -90831,65 +90846,73 @@ export const query = $root.query = (() => { object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); if (message.target != null && message.hasOwnProperty("target")) object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) if (typeof message.transaction_id === "number") object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; - if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) - if (typeof message.reserved_id === "number") - object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; - else - object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.pre_queries && message.pre_queries.length) { + object.pre_queries = []; + for (let j = 0; j < message.pre_queries.length; ++j) + object.pre_queries[j] = message.pre_queries[j]; + } return object; }; /** - * Converts this ReleaseRequest to JSON. + * Converts this ReserveExecuteRequest to JSON. * @function toJSON - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @instance * @returns {Object.} JSON object */ - ReleaseRequest.prototype.toJSON = function toJSON() { + ReserveExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReleaseRequest + * Gets the default type url for ReserveExecuteRequest * @function getTypeUrl - * @memberof query.ReleaseRequest + * @memberof query.ReserveExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReleaseRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReleaseRequest"; + return typeUrlPrefix + "/query.ReserveExecuteRequest"; }; - return ReleaseRequest; + return ReserveExecuteRequest; })(); - query.ReleaseResponse = (function() { + query.ReserveExecuteResponse = (function() { /** - * Properties of a ReleaseResponse. + * Properties of a ReserveExecuteResponse. * @memberof query - * @interface IReleaseResponse + * @interface IReserveExecuteResponse + * @property {vtrpc.IRPCError|null} [error] ReserveExecuteResponse error + * @property {query.IQueryResult|null} [result] ReserveExecuteResponse result + * @property {number|Long|null} [reserved_id] ReserveExecuteResponse reserved_id + * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveExecuteResponse tablet_alias */ /** - * Constructs a new ReleaseResponse. + * Constructs a new ReserveExecuteResponse. * @memberof query - * @classdesc Represents a ReleaseResponse. - * @implements IReleaseResponse + * @classdesc Represents a ReserveExecuteResponse. + * @implements IReserveExecuteResponse * @constructor - * @param {query.IReleaseResponse=} [properties] Properties to set + * @param {query.IReserveExecuteResponse=} [properties] Properties to set */ - function ReleaseResponse(properties) { + function ReserveExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -90897,63 +90920,119 @@ export const query = $root.query = (() => { } /** - * Creates a new ReleaseResponse instance using the specified properties. - * @function create - * @memberof query.ReleaseResponse - * @static - * @param {query.IReleaseResponse=} [properties] Properties to set - * @returns {query.ReleaseResponse} ReleaseResponse instance - */ - ReleaseResponse.create = function create(properties) { - return new ReleaseResponse(properties); + * ReserveExecuteResponse error. + * @member {vtrpc.IRPCError|null|undefined} error + * @memberof query.ReserveExecuteResponse + * @instance + */ + ReserveExecuteResponse.prototype.error = null; + + /** + * ReserveExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ReserveExecuteResponse + * @instance + */ + ReserveExecuteResponse.prototype.result = null; + + /** + * ReserveExecuteResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ReserveExecuteResponse + * @instance + */ + ReserveExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ReserveExecuteResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof query.ReserveExecuteResponse + * @instance + */ + ReserveExecuteResponse.prototype.tablet_alias = null; + + /** + * Creates a new ReserveExecuteResponse instance using the specified properties. + * @function create + * @memberof query.ReserveExecuteResponse + * @static + * @param {query.IReserveExecuteResponse=} [properties] Properties to set + * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse instance + */ + ReserveExecuteResponse.create = function create(properties) { + return new ReserveExecuteResponse(properties); }; /** - * Encodes the specified ReleaseResponse message. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * Encodes the specified ReserveExecuteResponse message. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static - * @param {query.IReleaseResponse} message ReleaseResponse message or plain object to encode + * @param {query.IReserveExecuteResponse} message ReserveExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReleaseResponse.encode = function encode(message, writer) { + ReserveExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.reserved_id); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReleaseResponse message, length delimited. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * Encodes the specified ReserveExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static - * @param {query.IReleaseResponse} message ReleaseResponse message or plain object to encode + * @param {query.IReserveExecuteResponse} message ReserveExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReleaseResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReserveExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReleaseResponse message from the specified reader or buffer. + * Decodes a ReserveExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.ReleaseResponse} ReleaseResponse + * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReleaseResponse.decode = function decode(reader, length) { + ReserveExecuteResponse.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.query.ReleaseResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); + break; + } + case 2: { + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } + case 3: { + message.reserved_id = reader.int64(); + break; + } + case 4: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -90963,108 +91042,183 @@ export const query = $root.query = (() => { }; /** - * Decodes a ReleaseResponse message from the specified reader or buffer, length delimited. + * Decodes a ReserveExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.ReleaseResponse} ReleaseResponse + * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReleaseResponse.decodeDelimited = function decodeDelimited(reader) { + ReserveExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReleaseResponse message. + * Verifies a ReserveExecuteResponse message. * @function verify - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReleaseResponse.verify = function verify(message) { + ReserveExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.vtrpc.RPCError.verify(message.error); + if (error) + return "error." + error; + } + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a ReleaseResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.ReleaseResponse} ReleaseResponse + * @returns {query.ReserveExecuteResponse} ReserveExecuteResponse */ - ReleaseResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.ReleaseResponse) + ReserveExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveExecuteResponse) return object; - return new $root.query.ReleaseResponse(); + let message = new $root.query.ReserveExecuteResponse(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".query.ReserveExecuteResponse.error: object expected"); + message.error = $root.vtrpc.RPCError.fromObject(object.error); + } + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ReserveExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".query.ReserveExecuteResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + return message; }; /** - * Creates a plain object from a ReleaseResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReserveExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static - * @param {query.ReleaseResponse} message ReleaseResponse + * @param {query.ReserveExecuteResponse} message ReserveExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReleaseResponse.toObject = function toObject() { - return {}; + ReserveExecuteResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.error = null; + object.result = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; + object.tablet_alias = null; + } + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.vtrpc.RPCError.toObject(message.error, options); + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + return object; }; /** - * Converts this ReleaseResponse to JSON. + * Converts this ReserveExecuteResponse to JSON. * @function toJSON - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @instance * @returns {Object.} JSON object */ - ReleaseResponse.prototype.toJSON = function toJSON() { + ReserveExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReleaseResponse + * Gets the default type url for ReserveExecuteResponse * @function getTypeUrl - * @memberof query.ReleaseResponse + * @memberof query.ReserveExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReleaseResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.ReleaseResponse"; + return typeUrlPrefix + "/query.ReserveExecuteResponse"; }; - return ReleaseResponse; + return ReserveExecuteResponse; })(); - query.StreamHealthRequest = (function() { + query.ReserveStreamExecuteRequest = (function() { /** - * Properties of a StreamHealthRequest. + * Properties of a ReserveStreamExecuteRequest. * @memberof query - * @interface IStreamHealthRequest + * @interface IReserveStreamExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveStreamExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveStreamExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReserveStreamExecuteRequest target + * @property {query.IBoundQuery|null} [query] ReserveStreamExecuteRequest query + * @property {query.IExecuteOptions|null} [options] ReserveStreamExecuteRequest options + * @property {number|Long|null} [transaction_id] ReserveStreamExecuteRequest transaction_id + * @property {Array.|null} [pre_queries] ReserveStreamExecuteRequest pre_queries */ /** - * Constructs a new StreamHealthRequest. + * Constructs a new ReserveStreamExecuteRequest. * @memberof query - * @classdesc Represents a StreamHealthRequest. - * @implements IStreamHealthRequest + * @classdesc Represents a ReserveStreamExecuteRequest. + * @implements IReserveStreamExecuteRequest * @constructor - * @param {query.IStreamHealthRequest=} [properties] Properties to set + * @param {query.IReserveStreamExecuteRequest=} [properties] Properties to set */ - function StreamHealthRequest(properties) { + function ReserveStreamExecuteRequest(properties) { + this.pre_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -91072,63 +91226,164 @@ export const query = $root.query = (() => { } /** - * Creates a new StreamHealthRequest instance using the specified properties. + * ReserveStreamExecuteRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.effective_caller_id = null; + + /** + * ReserveStreamExecuteRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.immediate_caller_id = null; + + /** + * ReserveStreamExecuteRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.target = null; + + /** + * ReserveStreamExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.query = null; + + /** + * ReserveStreamExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.options = null; + + /** + * ReserveStreamExecuteRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ReserveStreamExecuteRequest pre_queries. + * @member {Array.} pre_queries + * @memberof query.ReserveStreamExecuteRequest + * @instance + */ + ReserveStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; + + /** + * Creates a new ReserveStreamExecuteRequest instance using the specified properties. * @function create - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static - * @param {query.IStreamHealthRequest=} [properties] Properties to set - * @returns {query.StreamHealthRequest} StreamHealthRequest instance + * @param {query.IReserveStreamExecuteRequest=} [properties] Properties to set + * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest instance */ - StreamHealthRequest.create = function create(properties) { - return new StreamHealthRequest(properties); + ReserveStreamExecuteRequest.create = function create(properties) { + return new ReserveStreamExecuteRequest(properties); }; /** - * Encodes the specified StreamHealthRequest message. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * Encodes the specified ReserveStreamExecuteRequest message. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static - * @param {query.IStreamHealthRequest} message StreamHealthRequest message or plain object to encode + * @param {query.IReserveStreamExecuteRequest} message ReserveStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamHealthRequest.encode = function encode(message, writer) { + ReserveStreamExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.transaction_id); + if (message.pre_queries != null && message.pre_queries.length) + for (let i = 0; i < message.pre_queries.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.pre_queries[i]); return writer; }; /** - * Encodes the specified StreamHealthRequest message, length delimited. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * Encodes the specified ReserveStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static - * @param {query.IStreamHealthRequest} message StreamHealthRequest message or plain object to encode + * @param {query.IReserveStreamExecuteRequest} message ReserveStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamHealthRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReserveStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamHealthRequest message from the specified reader or buffer. + * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StreamHealthRequest} StreamHealthRequest + * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamHealthRequest.decode = function decode(reader, length) { + ReserveStreamExecuteRequest.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.query.StreamHealthRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveStreamExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } + case 2: { + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + case 4: { + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + break; + } + case 5: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 6: { + message.transaction_id = reader.int64(); + break; + } + case 7: { + if (!(message.pre_queries && message.pre_queries.length)) + message.pre_queries = []; + message.pre_queries.push(reader.string()); + break; + } default: reader.skipType(tag & 7); break; @@ -91138,118 +91393,226 @@ export const query = $root.query = (() => { }; /** - * Decodes a StreamHealthRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveStreamExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamHealthRequest} StreamHealthRequest + * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamHealthRequest.decodeDelimited = function decodeDelimited(reader) { + ReserveStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamHealthRequest message. + * Verifies a ReserveStreamExecuteRequest message. * @function verify - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamHealthRequest.verify = function verify(message) { + ReserveStreamExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { + if (!Array.isArray(message.pre_queries)) + return "pre_queries: array expected"; + for (let i = 0; i < message.pre_queries.length; ++i) + if (!$util.isString(message.pre_queries[i])) + return "pre_queries: string[] expected"; + } return null; }; /** - * Creates a StreamHealthRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.StreamHealthRequest} StreamHealthRequest + * @returns {query.ReserveStreamExecuteRequest} ReserveStreamExecuteRequest */ - StreamHealthRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamHealthRequest) + ReserveStreamExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveStreamExecuteRequest) return object; - return new $root.query.StreamHealthRequest(); + let message = new $root.query.ReserveStreamExecuteRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.ReserveStreamExecuteRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.ReserveStreamExecuteRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.ReserveStreamExecuteRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.ReserveStreamExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.ReserveStreamExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.pre_queries) { + if (!Array.isArray(object.pre_queries)) + throw TypeError(".query.ReserveStreamExecuteRequest.pre_queries: array expected"); + message.pre_queries = []; + for (let i = 0; i < object.pre_queries.length; ++i) + message.pre_queries[i] = String(object.pre_queries[i]); + } + return message; }; /** - * Creates a plain object from a StreamHealthRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReserveStreamExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.StreamHealthRequest + * @memberof query.ReserveStreamExecuteRequest * @static - * @param {query.StreamHealthRequest} message StreamHealthRequest + * @param {query.ReserveStreamExecuteRequest} message ReserveStreamExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamHealthRequest.toObject = function toObject() { - return {}; - }; - - /** - * Converts this StreamHealthRequest to JSON. - * @function toJSON - * @memberof query.StreamHealthRequest - * @instance - * @returns {Object.} JSON object - */ - StreamHealthRequest.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for StreamHealthRequest - * @function getTypeUrl - * @memberof query.StreamHealthRequest - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - StreamHealthRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/query.StreamHealthRequest"; + ReserveStreamExecuteRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.pre_queries = []; + if (options.defaults) { + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = null; + object.options = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; + } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.pre_queries && message.pre_queries.length) { + object.pre_queries = []; + for (let j = 0; j < message.pre_queries.length; ++j) + object.pre_queries[j] = message.pre_queries[j]; + } + return object; }; - return StreamHealthRequest; + /** + * Converts this ReserveStreamExecuteRequest to JSON. + * @function toJSON + * @memberof query.ReserveStreamExecuteRequest + * @instance + * @returns {Object.} JSON object + */ + ReserveStreamExecuteRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReserveStreamExecuteRequest + * @function getTypeUrl + * @memberof query.ReserveStreamExecuteRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReserveStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/query.ReserveStreamExecuteRequest"; + }; + + return ReserveStreamExecuteRequest; })(); - query.RealtimeStats = (function() { + query.ReserveStreamExecuteResponse = (function() { /** - * Properties of a RealtimeStats. + * Properties of a ReserveStreamExecuteResponse. * @memberof query - * @interface IRealtimeStats - * @property {string|null} [health_error] RealtimeStats health_error - * @property {number|null} [replication_lag_seconds] RealtimeStats replication_lag_seconds - * @property {number|null} [binlog_players_count] RealtimeStats binlog_players_count - * @property {number|Long|null} [filtered_replication_lag_seconds] RealtimeStats filtered_replication_lag_seconds - * @property {number|null} [cpu_usage] RealtimeStats cpu_usage - * @property {number|null} [qps] RealtimeStats qps - * @property {Array.|null} [table_schema_changed] RealtimeStats table_schema_changed - * @property {Array.|null} [view_schema_changed] RealtimeStats view_schema_changed + * @interface IReserveStreamExecuteResponse + * @property {vtrpc.IRPCError|null} [error] ReserveStreamExecuteResponse error + * @property {query.IQueryResult|null} [result] ReserveStreamExecuteResponse result + * @property {number|Long|null} [reserved_id] ReserveStreamExecuteResponse reserved_id + * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveStreamExecuteResponse tablet_alias */ /** - * Constructs a new RealtimeStats. + * Constructs a new ReserveStreamExecuteResponse. * @memberof query - * @classdesc Represents a RealtimeStats. - * @implements IRealtimeStats + * @classdesc Represents a ReserveStreamExecuteResponse. + * @implements IReserveStreamExecuteResponse * @constructor - * @param {query.IRealtimeStats=} [properties] Properties to set + * @param {query.IReserveStreamExecuteResponse=} [properties] Properties to set */ - function RealtimeStats(properties) { - this.table_schema_changed = []; - this.view_schema_changed = []; + function ReserveStreamExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -91257,179 +91620,117 @@ export const query = $root.query = (() => { } /** - * RealtimeStats health_error. - * @member {string} health_error - * @memberof query.RealtimeStats - * @instance - */ - RealtimeStats.prototype.health_error = ""; - - /** - * RealtimeStats replication_lag_seconds. - * @member {number} replication_lag_seconds - * @memberof query.RealtimeStats - * @instance - */ - RealtimeStats.prototype.replication_lag_seconds = 0; - - /** - * RealtimeStats binlog_players_count. - * @member {number} binlog_players_count - * @memberof query.RealtimeStats - * @instance - */ - RealtimeStats.prototype.binlog_players_count = 0; - - /** - * RealtimeStats filtered_replication_lag_seconds. - * @member {number|Long} filtered_replication_lag_seconds - * @memberof query.RealtimeStats - * @instance - */ - RealtimeStats.prototype.filtered_replication_lag_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * RealtimeStats cpu_usage. - * @member {number} cpu_usage - * @memberof query.RealtimeStats + * ReserveStreamExecuteResponse error. + * @member {vtrpc.IRPCError|null|undefined} error + * @memberof query.ReserveStreamExecuteResponse * @instance */ - RealtimeStats.prototype.cpu_usage = 0; + ReserveStreamExecuteResponse.prototype.error = null; /** - * RealtimeStats qps. - * @member {number} qps - * @memberof query.RealtimeStats + * ReserveStreamExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ReserveStreamExecuteResponse * @instance */ - RealtimeStats.prototype.qps = 0; + ReserveStreamExecuteResponse.prototype.result = null; /** - * RealtimeStats table_schema_changed. - * @member {Array.} table_schema_changed - * @memberof query.RealtimeStats + * ReserveStreamExecuteResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ReserveStreamExecuteResponse * @instance */ - RealtimeStats.prototype.table_schema_changed = $util.emptyArray; + ReserveStreamExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * RealtimeStats view_schema_changed. - * @member {Array.} view_schema_changed - * @memberof query.RealtimeStats + * ReserveStreamExecuteResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof query.ReserveStreamExecuteResponse * @instance */ - RealtimeStats.prototype.view_schema_changed = $util.emptyArray; + ReserveStreamExecuteResponse.prototype.tablet_alias = null; /** - * Creates a new RealtimeStats instance using the specified properties. + * Creates a new ReserveStreamExecuteResponse instance using the specified properties. * @function create - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static - * @param {query.IRealtimeStats=} [properties] Properties to set - * @returns {query.RealtimeStats} RealtimeStats instance + * @param {query.IReserveStreamExecuteResponse=} [properties] Properties to set + * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse instance */ - RealtimeStats.create = function create(properties) { - return new RealtimeStats(properties); + ReserveStreamExecuteResponse.create = function create(properties) { + return new ReserveStreamExecuteResponse(properties); }; /** - * Encodes the specified RealtimeStats message. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. + * Encodes the specified ReserveStreamExecuteResponse message. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static - * @param {query.IRealtimeStats} message RealtimeStats message or plain object to encode + * @param {query.IReserveStreamExecuteResponse} message ReserveStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RealtimeStats.encode = function encode(message, writer) { + ReserveStreamExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.health_error != null && Object.hasOwnProperty.call(message, "health_error")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.health_error); - if (message.replication_lag_seconds != null && Object.hasOwnProperty.call(message, "replication_lag_seconds")) - writer.uint32(/* id 2, wireType 0 =*/16).uint32(message.replication_lag_seconds); - if (message.binlog_players_count != null && Object.hasOwnProperty.call(message, "binlog_players_count")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.binlog_players_count); - if (message.filtered_replication_lag_seconds != null && Object.hasOwnProperty.call(message, "filtered_replication_lag_seconds")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.filtered_replication_lag_seconds); - if (message.cpu_usage != null && Object.hasOwnProperty.call(message, "cpu_usage")) - writer.uint32(/* id 5, wireType 1 =*/41).double(message.cpu_usage); - if (message.qps != null && Object.hasOwnProperty.call(message, "qps")) - writer.uint32(/* id 6, wireType 1 =*/49).double(message.qps); - if (message.table_schema_changed != null && message.table_schema_changed.length) - for (let i = 0; i < message.table_schema_changed.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.table_schema_changed[i]); - if (message.view_schema_changed != null && message.view_schema_changed.length) - for (let i = 0; i < message.view_schema_changed.length; ++i) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.view_schema_changed[i]); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.reserved_id); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified RealtimeStats message, length delimited. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. + * Encodes the specified ReserveStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveStreamExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static - * @param {query.IRealtimeStats} message RealtimeStats message or plain object to encode + * @param {query.IReserveStreamExecuteResponse} message ReserveStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RealtimeStats.encodeDelimited = function encodeDelimited(message, writer) { + ReserveStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RealtimeStats message from the specified reader or buffer. + * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.RealtimeStats} RealtimeStats + * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RealtimeStats.decode = function decode(reader, length) { + ReserveStreamExecuteResponse.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.query.RealtimeStats(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveStreamExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.health_error = reader.string(); + message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); break; } case 2: { - message.replication_lag_seconds = reader.uint32(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } case 3: { - message.binlog_players_count = reader.int32(); + message.reserved_id = reader.int64(); break; } case 4: { - message.filtered_replication_lag_seconds = reader.int64(); - break; - } - case 5: { - message.cpu_usage = reader.double(); - break; - } - case 6: { - message.qps = reader.double(); - break; - } - case 7: { - if (!(message.table_schema_changed && message.table_schema_changed.length)) - message.table_schema_changed = []; - message.table_schema_changed.push(reader.string()); - break; - } - case 8: { - if (!(message.view_schema_changed && message.view_schema_changed.length)) - message.view_schema_changed = []; - message.view_schema_changed.push(reader.string()); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -91441,222 +91742,184 @@ export const query = $root.query = (() => { }; /** - * Decodes a RealtimeStats message from the specified reader or buffer, length delimited. + * Decodes a ReserveStreamExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.RealtimeStats} RealtimeStats + * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RealtimeStats.decodeDelimited = function decodeDelimited(reader) { + ReserveStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RealtimeStats message. + * Verifies a ReserveStreamExecuteResponse message. * @function verify - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RealtimeStats.verify = function verify(message) { + ReserveStreamExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.health_error != null && message.hasOwnProperty("health_error")) - if (!$util.isString(message.health_error)) - return "health_error: string expected"; - if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) - if (!$util.isInteger(message.replication_lag_seconds)) - return "replication_lag_seconds: integer expected"; - if (message.binlog_players_count != null && message.hasOwnProperty("binlog_players_count")) - if (!$util.isInteger(message.binlog_players_count)) - return "binlog_players_count: integer expected"; - if (message.filtered_replication_lag_seconds != null && message.hasOwnProperty("filtered_replication_lag_seconds")) - if (!$util.isInteger(message.filtered_replication_lag_seconds) && !(message.filtered_replication_lag_seconds && $util.isInteger(message.filtered_replication_lag_seconds.low) && $util.isInteger(message.filtered_replication_lag_seconds.high))) - return "filtered_replication_lag_seconds: integer|Long expected"; - if (message.cpu_usage != null && message.hasOwnProperty("cpu_usage")) - if (typeof message.cpu_usage !== "number") - return "cpu_usage: number expected"; - if (message.qps != null && message.hasOwnProperty("qps")) - if (typeof message.qps !== "number") - return "qps: number expected"; - if (message.table_schema_changed != null && message.hasOwnProperty("table_schema_changed")) { - if (!Array.isArray(message.table_schema_changed)) - return "table_schema_changed: array expected"; - for (let i = 0; i < message.table_schema_changed.length; ++i) - if (!$util.isString(message.table_schema_changed[i])) - return "table_schema_changed: string[] expected"; + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.vtrpc.RPCError.verify(message.error); + if (error) + return "error." + error; } - if (message.view_schema_changed != null && message.hasOwnProperty("view_schema_changed")) { - if (!Array.isArray(message.view_schema_changed)) - return "view_schema_changed: array expected"; - for (let i = 0; i < message.view_schema_changed.length; ++i) - if (!$util.isString(message.view_schema_changed[i])) - return "view_schema_changed: string[] expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; } return null; }; /** - * Creates a RealtimeStats message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.RealtimeStats} RealtimeStats + * @returns {query.ReserveStreamExecuteResponse} ReserveStreamExecuteResponse */ - RealtimeStats.fromObject = function fromObject(object) { - if (object instanceof $root.query.RealtimeStats) + ReserveStreamExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveStreamExecuteResponse) return object; - let message = new $root.query.RealtimeStats(); - if (object.health_error != null) - message.health_error = String(object.health_error); - if (object.replication_lag_seconds != null) - message.replication_lag_seconds = object.replication_lag_seconds >>> 0; - if (object.binlog_players_count != null) - message.binlog_players_count = object.binlog_players_count | 0; - if (object.filtered_replication_lag_seconds != null) - if ($util.Long) - (message.filtered_replication_lag_seconds = $util.Long.fromValue(object.filtered_replication_lag_seconds)).unsigned = false; - else if (typeof object.filtered_replication_lag_seconds === "string") - message.filtered_replication_lag_seconds = parseInt(object.filtered_replication_lag_seconds, 10); - else if (typeof object.filtered_replication_lag_seconds === "number") - message.filtered_replication_lag_seconds = object.filtered_replication_lag_seconds; - else if (typeof object.filtered_replication_lag_seconds === "object") - message.filtered_replication_lag_seconds = new $util.LongBits(object.filtered_replication_lag_seconds.low >>> 0, object.filtered_replication_lag_seconds.high >>> 0).toNumber(); - if (object.cpu_usage != null) - message.cpu_usage = Number(object.cpu_usage); - if (object.qps != null) - message.qps = Number(object.qps); - if (object.table_schema_changed) { - if (!Array.isArray(object.table_schema_changed)) - throw TypeError(".query.RealtimeStats.table_schema_changed: array expected"); - message.table_schema_changed = []; - for (let i = 0; i < object.table_schema_changed.length; ++i) - message.table_schema_changed[i] = String(object.table_schema_changed[i]); + let message = new $root.query.ReserveStreamExecuteResponse(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".query.ReserveStreamExecuteResponse.error: object expected"); + message.error = $root.vtrpc.RPCError.fromObject(object.error); } - if (object.view_schema_changed) { - if (!Array.isArray(object.view_schema_changed)) - throw TypeError(".query.RealtimeStats.view_schema_changed: array expected"); - message.view_schema_changed = []; - for (let i = 0; i < object.view_schema_changed.length; ++i) - message.view_schema_changed[i] = String(object.view_schema_changed[i]); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ReserveStreamExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".query.ReserveStreamExecuteResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } return message; }; /** - * Creates a plain object from a RealtimeStats message. Also converts values to other types if specified. + * Creates a plain object from a ReserveStreamExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static - * @param {query.RealtimeStats} message RealtimeStats + * @param {query.ReserveStreamExecuteResponse} message ReserveStreamExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RealtimeStats.toObject = function toObject(message, options) { + ReserveStreamExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.table_schema_changed = []; - object.view_schema_changed = []; - } if (options.defaults) { - object.health_error = ""; - object.replication_lag_seconds = 0; - object.binlog_players_count = 0; + object.error = null; + object.result = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.filtered_replication_lag_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.filtered_replication_lag_seconds = options.longs === String ? "0" : 0; - object.cpu_usage = 0; - object.qps = 0; + object.reserved_id = options.longs === String ? "0" : 0; + object.tablet_alias = null; } - if (message.health_error != null && message.hasOwnProperty("health_error")) - object.health_error = message.health_error; - if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) - object.replication_lag_seconds = message.replication_lag_seconds; - if (message.binlog_players_count != null && message.hasOwnProperty("binlog_players_count")) - object.binlog_players_count = message.binlog_players_count; - if (message.filtered_replication_lag_seconds != null && message.hasOwnProperty("filtered_replication_lag_seconds")) - if (typeof message.filtered_replication_lag_seconds === "number") - object.filtered_replication_lag_seconds = options.longs === String ? String(message.filtered_replication_lag_seconds) : message.filtered_replication_lag_seconds; + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.vtrpc.RPCError.toObject(message.error, options); + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; else - object.filtered_replication_lag_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.filtered_replication_lag_seconds) : options.longs === Number ? new $util.LongBits(message.filtered_replication_lag_seconds.low >>> 0, message.filtered_replication_lag_seconds.high >>> 0).toNumber() : message.filtered_replication_lag_seconds; - if (message.cpu_usage != null && message.hasOwnProperty("cpu_usage")) - object.cpu_usage = options.json && !isFinite(message.cpu_usage) ? String(message.cpu_usage) : message.cpu_usage; - if (message.qps != null && message.hasOwnProperty("qps")) - object.qps = options.json && !isFinite(message.qps) ? String(message.qps) : message.qps; - if (message.table_schema_changed && message.table_schema_changed.length) { - object.table_schema_changed = []; - for (let j = 0; j < message.table_schema_changed.length; ++j) - object.table_schema_changed[j] = message.table_schema_changed[j]; - } - if (message.view_schema_changed && message.view_schema_changed.length) { - object.view_schema_changed = []; - for (let j = 0; j < message.view_schema_changed.length; ++j) - object.view_schema_changed[j] = message.view_schema_changed[j]; - } + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this RealtimeStats to JSON. + * Converts this ReserveStreamExecuteResponse to JSON. * @function toJSON - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @instance * @returns {Object.} JSON object */ - RealtimeStats.prototype.toJSON = function toJSON() { + ReserveStreamExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RealtimeStats + * Gets the default type url for ReserveStreamExecuteResponse * @function getTypeUrl - * @memberof query.RealtimeStats + * @memberof query.ReserveStreamExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RealtimeStats.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.RealtimeStats"; + return typeUrlPrefix + "/query.ReserveStreamExecuteResponse"; }; - return RealtimeStats; + return ReserveStreamExecuteResponse; })(); - query.AggregateStats = (function() { + query.ReserveBeginExecuteRequest = (function() { /** - * Properties of an AggregateStats. + * Properties of a ReserveBeginExecuteRequest. * @memberof query - * @interface IAggregateStats - * @property {number|null} [healthy_tablet_count] AggregateStats healthy_tablet_count - * @property {number|null} [unhealthy_tablet_count] AggregateStats unhealthy_tablet_count - * @property {number|null} [replication_lag_seconds_min] AggregateStats replication_lag_seconds_min - * @property {number|null} [replication_lag_seconds_max] AggregateStats replication_lag_seconds_max + * @interface IReserveBeginExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveBeginExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveBeginExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReserveBeginExecuteRequest target + * @property {query.IBoundQuery|null} [query] ReserveBeginExecuteRequest query + * @property {query.IExecuteOptions|null} [options] ReserveBeginExecuteRequest options + * @property {Array.|null} [pre_queries] ReserveBeginExecuteRequest pre_queries + * @property {Array.|null} [post_begin_queries] ReserveBeginExecuteRequest post_begin_queries */ /** - * Constructs a new AggregateStats. + * Constructs a new ReserveBeginExecuteRequest. * @memberof query - * @classdesc Represents an AggregateStats. - * @implements IAggregateStats + * @classdesc Represents a ReserveBeginExecuteRequest. + * @implements IReserveBeginExecuteRequest * @constructor - * @param {query.IAggregateStats=} [properties] Properties to set + * @param {query.IReserveBeginExecuteRequest=} [properties] Properties to set */ - function AggregateStats(properties) { + function ReserveBeginExecuteRequest(properties) { + this.pre_queries = []; + this.post_begin_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -91664,117 +91927,165 @@ export const query = $root.query = (() => { } /** - * AggregateStats healthy_tablet_count. - * @member {number} healthy_tablet_count - * @memberof query.AggregateStats + * ReserveBeginExecuteRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.ReserveBeginExecuteRequest * @instance */ - AggregateStats.prototype.healthy_tablet_count = 0; + ReserveBeginExecuteRequest.prototype.effective_caller_id = null; /** - * AggregateStats unhealthy_tablet_count. - * @member {number} unhealthy_tablet_count - * @memberof query.AggregateStats + * ReserveBeginExecuteRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.ReserveBeginExecuteRequest * @instance */ - AggregateStats.prototype.unhealthy_tablet_count = 0; + ReserveBeginExecuteRequest.prototype.immediate_caller_id = null; /** - * AggregateStats replication_lag_seconds_min. - * @member {number} replication_lag_seconds_min - * @memberof query.AggregateStats + * ReserveBeginExecuteRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.ReserveBeginExecuteRequest * @instance */ - AggregateStats.prototype.replication_lag_seconds_min = 0; + ReserveBeginExecuteRequest.prototype.target = null; /** - * AggregateStats replication_lag_seconds_max. - * @member {number} replication_lag_seconds_max - * @memberof query.AggregateStats + * ReserveBeginExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.ReserveBeginExecuteRequest * @instance */ - AggregateStats.prototype.replication_lag_seconds_max = 0; + ReserveBeginExecuteRequest.prototype.query = null; /** - * Creates a new AggregateStats instance using the specified properties. + * ReserveBeginExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.ReserveBeginExecuteRequest + * @instance + */ + ReserveBeginExecuteRequest.prototype.options = null; + + /** + * ReserveBeginExecuteRequest pre_queries. + * @member {Array.} pre_queries + * @memberof query.ReserveBeginExecuteRequest + * @instance + */ + ReserveBeginExecuteRequest.prototype.pre_queries = $util.emptyArray; + + /** + * ReserveBeginExecuteRequest post_begin_queries. + * @member {Array.} post_begin_queries + * @memberof query.ReserveBeginExecuteRequest + * @instance + */ + ReserveBeginExecuteRequest.prototype.post_begin_queries = $util.emptyArray; + + /** + * Creates a new ReserveBeginExecuteRequest instance using the specified properties. * @function create - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static - * @param {query.IAggregateStats=} [properties] Properties to set - * @returns {query.AggregateStats} AggregateStats instance + * @param {query.IReserveBeginExecuteRequest=} [properties] Properties to set + * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest instance */ - AggregateStats.create = function create(properties) { - return new AggregateStats(properties); + ReserveBeginExecuteRequest.create = function create(properties) { + return new ReserveBeginExecuteRequest(properties); }; /** - * Encodes the specified AggregateStats message. Does not implicitly {@link query.AggregateStats.verify|verify} messages. + * Encodes the specified ReserveBeginExecuteRequest message. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static - * @param {query.IAggregateStats} message AggregateStats message or plain object to encode + * @param {query.IReserveBeginExecuteRequest} message ReserveBeginExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AggregateStats.encode = function encode(message, writer) { + ReserveBeginExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.healthy_tablet_count != null && Object.hasOwnProperty.call(message, "healthy_tablet_count")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.healthy_tablet_count); - if (message.unhealthy_tablet_count != null && Object.hasOwnProperty.call(message, "unhealthy_tablet_count")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.unhealthy_tablet_count); - if (message.replication_lag_seconds_min != null && Object.hasOwnProperty.call(message, "replication_lag_seconds_min")) - writer.uint32(/* id 3, wireType 0 =*/24).uint32(message.replication_lag_seconds_min); - if (message.replication_lag_seconds_max != null && Object.hasOwnProperty.call(message, "replication_lag_seconds_max")) - writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.replication_lag_seconds_max); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.pre_queries != null && message.pre_queries.length) + for (let i = 0; i < message.pre_queries.length; ++i) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); + if (message.post_begin_queries != null && message.post_begin_queries.length) + for (let i = 0; i < message.post_begin_queries.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.post_begin_queries[i]); return writer; }; /** - * Encodes the specified AggregateStats message, length delimited. Does not implicitly {@link query.AggregateStats.verify|verify} messages. + * Encodes the specified ReserveBeginExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static - * @param {query.IAggregateStats} message AggregateStats message or plain object to encode + * @param {query.IReserveBeginExecuteRequest} message ReserveBeginExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AggregateStats.encodeDelimited = function encodeDelimited(message, writer) { + ReserveBeginExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an AggregateStats message from the specified reader or buffer. + * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.AggregateStats} AggregateStats + * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AggregateStats.decode = function decode(reader, length) { + ReserveBeginExecuteRequest.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.query.AggregateStats(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveBeginExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.healthy_tablet_count = reader.int32(); + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); break; } case 2: { - message.unhealthy_tablet_count = reader.int32(); + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); break; } case 3: { - message.replication_lag_seconds_min = reader.uint32(); + message.target = $root.query.Target.decode(reader, reader.uint32()); break; } case 4: { - message.replication_lag_seconds_max = reader.uint32(); + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + break; + } + case 5: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 6: { + if (!(message.pre_queries && message.pre_queries.length)) + message.pre_queries = []; + message.pre_queries.push(reader.string()); + break; + } + case 7: { + if (!(message.post_begin_queries && message.post_begin_queries.length)) + message.post_begin_queries = []; + message.post_begin_queries.push(reader.string()); break; } default: @@ -91786,151 +92097,227 @@ export const query = $root.query = (() => { }; /** - * Decodes an AggregateStats message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.AggregateStats} AggregateStats + * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AggregateStats.decodeDelimited = function decodeDelimited(reader) { + ReserveBeginExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an AggregateStats message. + * Verifies a ReserveBeginExecuteRequest message. * @function verify - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - AggregateStats.verify = function verify(message) { + ReserveBeginExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.healthy_tablet_count != null && message.hasOwnProperty("healthy_tablet_count")) - if (!$util.isInteger(message.healthy_tablet_count)) - return "healthy_tablet_count: integer expected"; - if (message.unhealthy_tablet_count != null && message.hasOwnProperty("unhealthy_tablet_count")) - if (!$util.isInteger(message.unhealthy_tablet_count)) - return "unhealthy_tablet_count: integer expected"; - if (message.replication_lag_seconds_min != null && message.hasOwnProperty("replication_lag_seconds_min")) - if (!$util.isInteger(message.replication_lag_seconds_min)) - return "replication_lag_seconds_min: integer expected"; - if (message.replication_lag_seconds_max != null && message.hasOwnProperty("replication_lag_seconds_max")) - if (!$util.isInteger(message.replication_lag_seconds_max)) - return "replication_lag_seconds_max: integer expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { + if (!Array.isArray(message.pre_queries)) + return "pre_queries: array expected"; + for (let i = 0; i < message.pre_queries.length; ++i) + if (!$util.isString(message.pre_queries[i])) + return "pre_queries: string[] expected"; + } + if (message.post_begin_queries != null && message.hasOwnProperty("post_begin_queries")) { + if (!Array.isArray(message.post_begin_queries)) + return "post_begin_queries: array expected"; + for (let i = 0; i < message.post_begin_queries.length; ++i) + if (!$util.isString(message.post_begin_queries[i])) + return "post_begin_queries: string[] expected"; + } return null; }; /** - * Creates an AggregateStats message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.AggregateStats} AggregateStats + * @returns {query.ReserveBeginExecuteRequest} ReserveBeginExecuteRequest */ - AggregateStats.fromObject = function fromObject(object) { - if (object instanceof $root.query.AggregateStats) + ReserveBeginExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveBeginExecuteRequest) return object; - let message = new $root.query.AggregateStats(); - if (object.healthy_tablet_count != null) - message.healthy_tablet_count = object.healthy_tablet_count | 0; - if (object.unhealthy_tablet_count != null) - message.unhealthy_tablet_count = object.unhealthy_tablet_count | 0; - if (object.replication_lag_seconds_min != null) - message.replication_lag_seconds_min = object.replication_lag_seconds_min >>> 0; - if (object.replication_lag_seconds_max != null) - message.replication_lag_seconds_max = object.replication_lag_seconds_max >>> 0; + let message = new $root.query.ReserveBeginExecuteRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.ReserveBeginExecuteRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.ReserveBeginExecuteRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.ReserveBeginExecuteRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.ReserveBeginExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.ReserveBeginExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.pre_queries) { + if (!Array.isArray(object.pre_queries)) + throw TypeError(".query.ReserveBeginExecuteRequest.pre_queries: array expected"); + message.pre_queries = []; + for (let i = 0; i < object.pre_queries.length; ++i) + message.pre_queries[i] = String(object.pre_queries[i]); + } + if (object.post_begin_queries) { + if (!Array.isArray(object.post_begin_queries)) + throw TypeError(".query.ReserveBeginExecuteRequest.post_begin_queries: array expected"); + message.post_begin_queries = []; + for (let i = 0; i < object.post_begin_queries.length; ++i) + message.post_begin_queries[i] = String(object.post_begin_queries[i]); + } return message; }; /** - * Creates a plain object from an AggregateStats message. Also converts values to other types if specified. + * Creates a plain object from a ReserveBeginExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static - * @param {query.AggregateStats} message AggregateStats + * @param {query.ReserveBeginExecuteRequest} message ReserveBeginExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - AggregateStats.toObject = function toObject(message, options) { + ReserveBeginExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.pre_queries = []; + object.post_begin_queries = []; + } if (options.defaults) { - object.healthy_tablet_count = 0; - object.unhealthy_tablet_count = 0; - object.replication_lag_seconds_min = 0; - object.replication_lag_seconds_max = 0; + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = null; + object.options = null; + } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.pre_queries && message.pre_queries.length) { + object.pre_queries = []; + for (let j = 0; j < message.pre_queries.length; ++j) + object.pre_queries[j] = message.pre_queries[j]; + } + if (message.post_begin_queries && message.post_begin_queries.length) { + object.post_begin_queries = []; + for (let j = 0; j < message.post_begin_queries.length; ++j) + object.post_begin_queries[j] = message.post_begin_queries[j]; } - if (message.healthy_tablet_count != null && message.hasOwnProperty("healthy_tablet_count")) - object.healthy_tablet_count = message.healthy_tablet_count; - if (message.unhealthy_tablet_count != null && message.hasOwnProperty("unhealthy_tablet_count")) - object.unhealthy_tablet_count = message.unhealthy_tablet_count; - if (message.replication_lag_seconds_min != null && message.hasOwnProperty("replication_lag_seconds_min")) - object.replication_lag_seconds_min = message.replication_lag_seconds_min; - if (message.replication_lag_seconds_max != null && message.hasOwnProperty("replication_lag_seconds_max")) - object.replication_lag_seconds_max = message.replication_lag_seconds_max; return object; }; /** - * Converts this AggregateStats to JSON. + * Converts this ReserveBeginExecuteRequest to JSON. * @function toJSON - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @instance * @returns {Object.} JSON object */ - AggregateStats.prototype.toJSON = function toJSON() { + ReserveBeginExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for AggregateStats + * Gets the default type url for ReserveBeginExecuteRequest * @function getTypeUrl - * @memberof query.AggregateStats + * @memberof query.ReserveBeginExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - AggregateStats.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveBeginExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.AggregateStats"; + return typeUrlPrefix + "/query.ReserveBeginExecuteRequest"; }; - return AggregateStats; + return ReserveBeginExecuteRequest; })(); - query.StreamHealthResponse = (function() { + query.ReserveBeginExecuteResponse = (function() { /** - * Properties of a StreamHealthResponse. + * Properties of a ReserveBeginExecuteResponse. * @memberof query - * @interface IStreamHealthResponse - * @property {query.ITarget|null} [target] StreamHealthResponse target - * @property {boolean|null} [serving] StreamHealthResponse serving - * @property {number|Long|null} [tablet_externally_reparented_timestamp] StreamHealthResponse tablet_externally_reparented_timestamp - * @property {query.IRealtimeStats|null} [realtime_stats] StreamHealthResponse realtime_stats - * @property {topodata.ITabletAlias|null} [tablet_alias] StreamHealthResponse tablet_alias + * @interface IReserveBeginExecuteResponse + * @property {vtrpc.IRPCError|null} [error] ReserveBeginExecuteResponse error + * @property {query.IQueryResult|null} [result] ReserveBeginExecuteResponse result + * @property {number|Long|null} [transaction_id] ReserveBeginExecuteResponse transaction_id + * @property {number|Long|null} [reserved_id] ReserveBeginExecuteResponse reserved_id + * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveBeginExecuteResponse tablet_alias + * @property {string|null} [session_state_changes] ReserveBeginExecuteResponse session_state_changes */ /** - * Constructs a new StreamHealthResponse. + * Constructs a new ReserveBeginExecuteResponse. * @memberof query - * @classdesc Represents a StreamHealthResponse. - * @implements IStreamHealthResponse + * @classdesc Represents a ReserveBeginExecuteResponse. + * @implements IReserveBeginExecuteResponse * @constructor - * @param {query.IStreamHealthResponse=} [properties] Properties to set + * @param {query.IReserveBeginExecuteResponse=} [properties] Properties to set */ - function StreamHealthResponse(properties) { + function ReserveBeginExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -91938,133 +92325,147 @@ export const query = $root.query = (() => { } /** - * StreamHealthResponse target. - * @member {query.ITarget|null|undefined} target - * @memberof query.StreamHealthResponse + * ReserveBeginExecuteResponse error. + * @member {vtrpc.IRPCError|null|undefined} error + * @memberof query.ReserveBeginExecuteResponse * @instance */ - StreamHealthResponse.prototype.target = null; + ReserveBeginExecuteResponse.prototype.error = null; /** - * StreamHealthResponse serving. - * @member {boolean} serving - * @memberof query.StreamHealthResponse + * ReserveBeginExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ReserveBeginExecuteResponse * @instance */ - StreamHealthResponse.prototype.serving = false; + ReserveBeginExecuteResponse.prototype.result = null; /** - * StreamHealthResponse tablet_externally_reparented_timestamp. - * @member {number|Long} tablet_externally_reparented_timestamp - * @memberof query.StreamHealthResponse + * ReserveBeginExecuteResponse transaction_id. + * @member {number|Long} transaction_id + * @memberof query.ReserveBeginExecuteResponse * @instance */ - StreamHealthResponse.prototype.tablet_externally_reparented_timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ReserveBeginExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * StreamHealthResponse realtime_stats. - * @member {query.IRealtimeStats|null|undefined} realtime_stats - * @memberof query.StreamHealthResponse + * ReserveBeginExecuteResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ReserveBeginExecuteResponse * @instance */ - StreamHealthResponse.prototype.realtime_stats = null; + ReserveBeginExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * StreamHealthResponse tablet_alias. + * ReserveBeginExecuteResponse tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @instance */ - StreamHealthResponse.prototype.tablet_alias = null; + ReserveBeginExecuteResponse.prototype.tablet_alias = null; /** - * Creates a new StreamHealthResponse instance using the specified properties. + * ReserveBeginExecuteResponse session_state_changes. + * @member {string} session_state_changes + * @memberof query.ReserveBeginExecuteResponse + * @instance + */ + ReserveBeginExecuteResponse.prototype.session_state_changes = ""; + + /** + * Creates a new ReserveBeginExecuteResponse instance using the specified properties. * @function create - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static - * @param {query.IStreamHealthResponse=} [properties] Properties to set - * @returns {query.StreamHealthResponse} StreamHealthResponse instance + * @param {query.IReserveBeginExecuteResponse=} [properties] Properties to set + * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse instance */ - StreamHealthResponse.create = function create(properties) { - return new StreamHealthResponse(properties); + ReserveBeginExecuteResponse.create = function create(properties) { + return new ReserveBeginExecuteResponse(properties); }; /** - * Encodes the specified StreamHealthResponse message. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. + * Encodes the specified ReserveBeginExecuteResponse message. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static - * @param {query.IStreamHealthResponse} message StreamHealthResponse message or plain object to encode + * @param {query.IReserveBeginExecuteResponse} message ReserveBeginExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamHealthResponse.encode = function encode(message, writer) { + ReserveBeginExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.serving != null && Object.hasOwnProperty.call(message, "serving")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.serving); - if (message.tablet_externally_reparented_timestamp != null && Object.hasOwnProperty.call(message, "tablet_externally_reparented_timestamp")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.tablet_externally_reparented_timestamp); - if (message.realtime_stats != null && Object.hasOwnProperty.call(message, "realtime_stats")) - $root.query.RealtimeStats.encode(message.realtime_stats, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.reserved_id); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.session_state_changes); return writer; }; /** - * Encodes the specified StreamHealthResponse message, length delimited. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. + * Encodes the specified ReserveBeginExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static - * @param {query.IStreamHealthResponse} message StreamHealthResponse message or plain object to encode + * @param {query.IReserveBeginExecuteResponse} message ReserveBeginExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StreamHealthResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReserveBeginExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StreamHealthResponse message from the specified reader or buffer. + * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.StreamHealthResponse} StreamHealthResponse + * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamHealthResponse.decode = function decode(reader, length) { + ReserveBeginExecuteResponse.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.query.StreamHealthResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveBeginExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.target = $root.query.Target.decode(reader, reader.uint32()); + message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); break; } case 2: { - message.serving = reader.bool(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } case 3: { - message.tablet_externally_reparented_timestamp = reader.int64(); + message.transaction_id = reader.int64(); break; } case 4: { - message.realtime_stats = $root.query.RealtimeStats.decode(reader, reader.uint32()); + message.reserved_id = reader.int64(); break; } case 5: { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } + case 6: { + message.session_state_changes = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -92074,206 +92475,214 @@ export const query = $root.query = (() => { }; /** - * Decodes a StreamHealthResponse message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.StreamHealthResponse} StreamHealthResponse + * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StreamHealthResponse.decodeDelimited = function decodeDelimited(reader) { + ReserveBeginExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StreamHealthResponse message. + * Verifies a ReserveBeginExecuteResponse message. * @function verify - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StreamHealthResponse.verify = function verify(message) { + ReserveBeginExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.vtrpc.RPCError.verify(message.error); if (error) - return "target." + error; + return "error." + error; } - if (message.serving != null && message.hasOwnProperty("serving")) - if (typeof message.serving !== "boolean") - return "serving: boolean expected"; - if (message.tablet_externally_reparented_timestamp != null && message.hasOwnProperty("tablet_externally_reparented_timestamp")) - if (!$util.isInteger(message.tablet_externally_reparented_timestamp) && !(message.tablet_externally_reparented_timestamp && $util.isInteger(message.tablet_externally_reparented_timestamp.low) && $util.isInteger(message.tablet_externally_reparented_timestamp.high))) - return "tablet_externally_reparented_timestamp: integer|Long expected"; - if (message.realtime_stats != null && message.hasOwnProperty("realtime_stats")) { - let error = $root.query.RealtimeStats.verify(message.realtime_stats); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "realtime_stats." + error; + return "result." + error; } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) return "tablet_alias." + error; } + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a StreamHealthResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.StreamHealthResponse} StreamHealthResponse + * @returns {query.ReserveBeginExecuteResponse} ReserveBeginExecuteResponse */ - StreamHealthResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.StreamHealthResponse) + ReserveBeginExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveBeginExecuteResponse) return object; - let message = new $root.query.StreamHealthResponse(); - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.StreamHealthResponse.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + let message = new $root.query.ReserveBeginExecuteResponse(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".query.ReserveBeginExecuteResponse.error: object expected"); + message.error = $root.vtrpc.RPCError.fromObject(object.error); } - if (object.serving != null) - message.serving = Boolean(object.serving); - if (object.tablet_externally_reparented_timestamp != null) + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ReserveBeginExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } + if (object.transaction_id != null) if ($util.Long) - (message.tablet_externally_reparented_timestamp = $util.Long.fromValue(object.tablet_externally_reparented_timestamp)).unsigned = false; - else if (typeof object.tablet_externally_reparented_timestamp === "string") - message.tablet_externally_reparented_timestamp = parseInt(object.tablet_externally_reparented_timestamp, 10); - else if (typeof object.tablet_externally_reparented_timestamp === "number") - message.tablet_externally_reparented_timestamp = object.tablet_externally_reparented_timestamp; - else if (typeof object.tablet_externally_reparented_timestamp === "object") - message.tablet_externally_reparented_timestamp = new $util.LongBits(object.tablet_externally_reparented_timestamp.low >>> 0, object.tablet_externally_reparented_timestamp.high >>> 0).toNumber(); - if (object.realtime_stats != null) { - if (typeof object.realtime_stats !== "object") - throw TypeError(".query.StreamHealthResponse.realtime_stats: object expected"); - message.realtime_stats = $root.query.RealtimeStats.fromObject(object.realtime_stats); - } + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".query.StreamHealthResponse.tablet_alias: object expected"); + throw TypeError(".query.ReserveBeginExecuteResponse.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a StreamHealthResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReserveBeginExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static - * @param {query.StreamHealthResponse} message StreamHealthResponse + * @param {query.ReserveBeginExecuteResponse} message ReserveBeginExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StreamHealthResponse.toObject = function toObject(message, options) { + ReserveBeginExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.target = null; - object.serving = false; + object.error = null; + object.result = null; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.tablet_externally_reparented_timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.tablet_externally_reparented_timestamp = options.longs === String ? "0" : 0; - object.realtime_stats = null; + object.transaction_id = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; object.tablet_alias = null; + object.session_state_changes = ""; } - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.serving != null && message.hasOwnProperty("serving")) - object.serving = message.serving; - if (message.tablet_externally_reparented_timestamp != null && message.hasOwnProperty("tablet_externally_reparented_timestamp")) - if (typeof message.tablet_externally_reparented_timestamp === "number") - object.tablet_externally_reparented_timestamp = options.longs === String ? String(message.tablet_externally_reparented_timestamp) : message.tablet_externally_reparented_timestamp; + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.vtrpc.RPCError.toObject(message.error, options); + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; else - object.tablet_externally_reparented_timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.tablet_externally_reparented_timestamp) : options.longs === Number ? new $util.LongBits(message.tablet_externally_reparented_timestamp.low >>> 0, message.tablet_externally_reparented_timestamp.high >>> 0).toNumber() : message.tablet_externally_reparented_timestamp; - if (message.realtime_stats != null && message.hasOwnProperty("realtime_stats")) - object.realtime_stats = $root.query.RealtimeStats.toObject(message.realtime_stats, options); + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this StreamHealthResponse to JSON. + * Converts this ReserveBeginExecuteResponse to JSON. * @function toJSON - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @instance * @returns {Object.} JSON object */ - StreamHealthResponse.prototype.toJSON = function toJSON() { + ReserveBeginExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StreamHealthResponse + * Gets the default type url for ReserveBeginExecuteResponse * @function getTypeUrl - * @memberof query.StreamHealthResponse + * @memberof query.ReserveBeginExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StreamHealthResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveBeginExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.StreamHealthResponse"; + return typeUrlPrefix + "/query.ReserveBeginExecuteResponse"; }; - return StreamHealthResponse; - })(); - - /** - * TransactionState enum. - * @name query.TransactionState - * @enum {number} - * @property {number} UNKNOWN=0 UNKNOWN value - * @property {number} PREPARE=1 PREPARE value - * @property {number} COMMIT=2 COMMIT value - * @property {number} ROLLBACK=3 ROLLBACK value - */ - query.TransactionState = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "UNKNOWN"] = 0; - values[valuesById[1] = "PREPARE"] = 1; - values[valuesById[2] = "COMMIT"] = 2; - values[valuesById[3] = "ROLLBACK"] = 3; - return values; + return ReserveBeginExecuteResponse; })(); - query.TransactionMetadata = (function() { + query.ReserveBeginStreamExecuteRequest = (function() { /** - * Properties of a TransactionMetadata. + * Properties of a ReserveBeginStreamExecuteRequest. * @memberof query - * @interface ITransactionMetadata - * @property {string|null} [dtid] TransactionMetadata dtid - * @property {query.TransactionState|null} [state] TransactionMetadata state - * @property {number|Long|null} [time_created] TransactionMetadata time_created - * @property {Array.|null} [participants] TransactionMetadata participants + * @interface IReserveBeginStreamExecuteRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReserveBeginStreamExecuteRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReserveBeginStreamExecuteRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReserveBeginStreamExecuteRequest target + * @property {query.IBoundQuery|null} [query] ReserveBeginStreamExecuteRequest query + * @property {query.IExecuteOptions|null} [options] ReserveBeginStreamExecuteRequest options + * @property {Array.|null} [pre_queries] ReserveBeginStreamExecuteRequest pre_queries + * @property {Array.|null} [post_begin_queries] ReserveBeginStreamExecuteRequest post_begin_queries */ /** - * Constructs a new TransactionMetadata. + * Constructs a new ReserveBeginStreamExecuteRequest. * @memberof query - * @classdesc Represents a TransactionMetadata. - * @implements ITransactionMetadata + * @classdesc Represents a ReserveBeginStreamExecuteRequest. + * @implements IReserveBeginStreamExecuteRequest * @constructor - * @param {query.ITransactionMetadata=} [properties] Properties to set + * @param {query.IReserveBeginStreamExecuteRequest=} [properties] Properties to set */ - function TransactionMetadata(properties) { - this.participants = []; + function ReserveBeginStreamExecuteRequest(properties) { + this.pre_queries = []; + this.post_begin_queries = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -92281,120 +92690,165 @@ export const query = $root.query = (() => { } /** - * TransactionMetadata dtid. - * @member {string} dtid - * @memberof query.TransactionMetadata + * ReserveBeginStreamExecuteRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.ReserveBeginStreamExecuteRequest * @instance */ - TransactionMetadata.prototype.dtid = ""; + ReserveBeginStreamExecuteRequest.prototype.effective_caller_id = null; /** - * TransactionMetadata state. - * @member {query.TransactionState} state - * @memberof query.TransactionMetadata + * ReserveBeginStreamExecuteRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.ReserveBeginStreamExecuteRequest * @instance */ - TransactionMetadata.prototype.state = 0; + ReserveBeginStreamExecuteRequest.prototype.immediate_caller_id = null; /** - * TransactionMetadata time_created. - * @member {number|Long} time_created - * @memberof query.TransactionMetadata + * ReserveBeginStreamExecuteRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.ReserveBeginStreamExecuteRequest * @instance */ - TransactionMetadata.prototype.time_created = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + ReserveBeginStreamExecuteRequest.prototype.target = null; /** - * TransactionMetadata participants. - * @member {Array.} participants - * @memberof query.TransactionMetadata + * ReserveBeginStreamExecuteRequest query. + * @member {query.IBoundQuery|null|undefined} query + * @memberof query.ReserveBeginStreamExecuteRequest * @instance */ - TransactionMetadata.prototype.participants = $util.emptyArray; + ReserveBeginStreamExecuteRequest.prototype.query = null; /** - * Creates a new TransactionMetadata instance using the specified properties. + * ReserveBeginStreamExecuteRequest options. + * @member {query.IExecuteOptions|null|undefined} options + * @memberof query.ReserveBeginStreamExecuteRequest + * @instance + */ + ReserveBeginStreamExecuteRequest.prototype.options = null; + + /** + * ReserveBeginStreamExecuteRequest pre_queries. + * @member {Array.} pre_queries + * @memberof query.ReserveBeginStreamExecuteRequest + * @instance + */ + ReserveBeginStreamExecuteRequest.prototype.pre_queries = $util.emptyArray; + + /** + * ReserveBeginStreamExecuteRequest post_begin_queries. + * @member {Array.} post_begin_queries + * @memberof query.ReserveBeginStreamExecuteRequest + * @instance + */ + ReserveBeginStreamExecuteRequest.prototype.post_begin_queries = $util.emptyArray; + + /** + * Creates a new ReserveBeginStreamExecuteRequest instance using the specified properties. * @function create - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static - * @param {query.ITransactionMetadata=} [properties] Properties to set - * @returns {query.TransactionMetadata} TransactionMetadata instance + * @param {query.IReserveBeginStreamExecuteRequest=} [properties] Properties to set + * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest instance */ - TransactionMetadata.create = function create(properties) { - return new TransactionMetadata(properties); + ReserveBeginStreamExecuteRequest.create = function create(properties) { + return new ReserveBeginStreamExecuteRequest(properties); }; /** - * Encodes the specified TransactionMetadata message. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. + * Encodes the specified ReserveBeginStreamExecuteRequest message. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. * @function encode - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static - * @param {query.ITransactionMetadata} message TransactionMetadata message or plain object to encode + * @param {query.IReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TransactionMetadata.encode = function encode(message, writer) { + ReserveBeginStreamExecuteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.dtid); - if (message.state != null && Object.hasOwnProperty.call(message, "state")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.state); - if (message.time_created != null && Object.hasOwnProperty.call(message, "time_created")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.time_created); - if (message.participants != null && message.participants.length) - for (let i = 0; i < message.participants.length; ++i) - $root.query.Target.encode(message.participants[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + $root.query.BoundQuery.encode(message.query, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.query.ExecuteOptions.encode(message.options, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.pre_queries != null && message.pre_queries.length) + for (let i = 0; i < message.pre_queries.length; ++i) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.pre_queries[i]); + if (message.post_begin_queries != null && message.post_begin_queries.length) + for (let i = 0; i < message.post_begin_queries.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.post_begin_queries[i]); return writer; }; /** - * Encodes the specified TransactionMetadata message, length delimited. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. + * Encodes the specified ReserveBeginStreamExecuteRequest message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static - * @param {query.ITransactionMetadata} message TransactionMetadata message or plain object to encode + * @param {query.IReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TransactionMetadata.encodeDelimited = function encodeDelimited(message, writer) { + ReserveBeginStreamExecuteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TransactionMetadata message from the specified reader or buffer. + * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer. * @function decode - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.TransactionMetadata} TransactionMetadata + * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TransactionMetadata.decode = function decode(reader, length) { + ReserveBeginStreamExecuteRequest.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.query.TransactionMetadata(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveBeginStreamExecuteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.dtid = reader.string(); + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); break; } case 2: { - message.state = reader.int32(); + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); break; } case 3: { - message.time_created = reader.int64(); + message.target = $root.query.Target.decode(reader, reader.uint32()); break; } case 4: { - if (!(message.participants && message.participants.length)) - message.participants = []; - message.participants.push($root.query.Target.decode(reader, reader.uint32())); + message.query = $root.query.BoundQuery.decode(reader, reader.uint32()); + break; + } + case 5: { + message.options = $root.query.ExecuteOptions.decode(reader, reader.uint32()); + break; + } + case 6: { + if (!(message.pre_queries && message.pre_queries.length)) + message.pre_queries = []; + message.pre_queries.push(reader.string()); + break; + } + case 7: { + if (!(message.post_begin_queries && message.post_begin_queries.length)) + message.post_begin_queries = []; + message.post_begin_queries.push(reader.string()); break; } default: @@ -92406,227 +92860,227 @@ export const query = $root.query = (() => { }; /** - * Decodes a TransactionMetadata message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginStreamExecuteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.TransactionMetadata} TransactionMetadata + * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TransactionMetadata.decodeDelimited = function decodeDelimited(reader) { + ReserveBeginStreamExecuteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TransactionMetadata message. + * Verifies a ReserveBeginStreamExecuteRequest message. * @function verify - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TransactionMetadata.verify = function verify(message) { + ReserveBeginStreamExecuteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.dtid != null && message.hasOwnProperty("dtid")) - if (!$util.isString(message.dtid)) - return "dtid: string expected"; - if (message.state != null && message.hasOwnProperty("state")) - switch (message.state) { - default: - return "state: enum value expected"; - case 0: - case 1: - case 2: - case 3: - break; - } - if (message.time_created != null && message.hasOwnProperty("time_created")) - if (!$util.isInteger(message.time_created) && !(message.time_created && $util.isInteger(message.time_created.low) && $util.isInteger(message.time_created.high))) - return "time_created: integer|Long expected"; - if (message.participants != null && message.hasOwnProperty("participants")) { - if (!Array.isArray(message.participants)) - return "participants: array expected"; - for (let i = 0; i < message.participants.length; ++i) { - let error = $root.query.Target.verify(message.participants[i]); - if (error) - return "participants." + error; - } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.query != null && message.hasOwnProperty("query")) { + let error = $root.query.BoundQuery.verify(message.query); + if (error) + return "query." + error; + } + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.query.ExecuteOptions.verify(message.options); + if (error) + return "options." + error; + } + if (message.pre_queries != null && message.hasOwnProperty("pre_queries")) { + if (!Array.isArray(message.pre_queries)) + return "pre_queries: array expected"; + for (let i = 0; i < message.pre_queries.length; ++i) + if (!$util.isString(message.pre_queries[i])) + return "pre_queries: string[] expected"; + } + if (message.post_begin_queries != null && message.hasOwnProperty("post_begin_queries")) { + if (!Array.isArray(message.post_begin_queries)) + return "post_begin_queries: array expected"; + for (let i = 0; i < message.post_begin_queries.length; ++i) + if (!$util.isString(message.post_begin_queries[i])) + return "post_begin_queries: string[] expected"; } return null; }; /** - * Creates a TransactionMetadata message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginStreamExecuteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static * @param {Object.} object Plain object - * @returns {query.TransactionMetadata} TransactionMetadata + * @returns {query.ReserveBeginStreamExecuteRequest} ReserveBeginStreamExecuteRequest */ - TransactionMetadata.fromObject = function fromObject(object) { - if (object instanceof $root.query.TransactionMetadata) + ReserveBeginStreamExecuteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveBeginStreamExecuteRequest) return object; - let message = new $root.query.TransactionMetadata(); - if (object.dtid != null) - message.dtid = String(object.dtid); - switch (object.state) { - default: - if (typeof object.state === "number") { - message.state = object.state; - break; - } - break; - case "UNKNOWN": - case 0: - message.state = 0; - break; - case "PREPARE": - case 1: - message.state = 1; - break; - case "COMMIT": - case 2: - message.state = 2; - break; - case "ROLLBACK": - case 3: - message.state = 3; - break; + let message = new $root.query.ReserveBeginStreamExecuteRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); } - if (object.time_created != null) - if ($util.Long) - (message.time_created = $util.Long.fromValue(object.time_created)).unsigned = false; - else if (typeof object.time_created === "string") - message.time_created = parseInt(object.time_created, 10); - else if (typeof object.time_created === "number") - message.time_created = object.time_created; - else if (typeof object.time_created === "object") - message.time_created = new $util.LongBits(object.time_created.low >>> 0, object.time_created.high >>> 0).toNumber(); - if (object.participants) { - if (!Array.isArray(object.participants)) - throw TypeError(".query.TransactionMetadata.participants: array expected"); - message.participants = []; - for (let i = 0; i < object.participants.length; ++i) { - if (typeof object.participants[i] !== "object") - throw TypeError(".query.TransactionMetadata.participants: object expected"); - message.participants[i] = $root.query.Target.fromObject(object.participants[i]); - } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.query != null) { + if (typeof object.query !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteRequest.query: object expected"); + message.query = $root.query.BoundQuery.fromObject(object.query); + } + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteRequest.options: object expected"); + message.options = $root.query.ExecuteOptions.fromObject(object.options); + } + if (object.pre_queries) { + if (!Array.isArray(object.pre_queries)) + throw TypeError(".query.ReserveBeginStreamExecuteRequest.pre_queries: array expected"); + message.pre_queries = []; + for (let i = 0; i < object.pre_queries.length; ++i) + message.pre_queries[i] = String(object.pre_queries[i]); + } + if (object.post_begin_queries) { + if (!Array.isArray(object.post_begin_queries)) + throw TypeError(".query.ReserveBeginStreamExecuteRequest.post_begin_queries: array expected"); + message.post_begin_queries = []; + for (let i = 0; i < object.post_begin_queries.length; ++i) + message.post_begin_queries[i] = String(object.post_begin_queries[i]); } return message; }; /** - * Creates a plain object from a TransactionMetadata message. Also converts values to other types if specified. + * Creates a plain object from a ReserveBeginStreamExecuteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static - * @param {query.TransactionMetadata} message TransactionMetadata + * @param {query.ReserveBeginStreamExecuteRequest} message ReserveBeginStreamExecuteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TransactionMetadata.toObject = function toObject(message, options) { + ReserveBeginStreamExecuteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.participants = []; + if (options.arrays || options.defaults) { + object.pre_queries = []; + object.post_begin_queries = []; + } if (options.defaults) { - object.dtid = ""; - object.state = options.enums === String ? "UNKNOWN" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.time_created = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.time_created = options.longs === String ? "0" : 0; + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + object.query = null; + object.options = null; } - if (message.dtid != null && message.hasOwnProperty("dtid")) - object.dtid = message.dtid; - if (message.state != null && message.hasOwnProperty("state")) - object.state = options.enums === String ? $root.query.TransactionState[message.state] === undefined ? message.state : $root.query.TransactionState[message.state] : message.state; - if (message.time_created != null && message.hasOwnProperty("time_created")) - if (typeof message.time_created === "number") - object.time_created = options.longs === String ? String(message.time_created) : message.time_created; - else - object.time_created = options.longs === String ? $util.Long.prototype.toString.call(message.time_created) : options.longs === Number ? new $util.LongBits(message.time_created.low >>> 0, message.time_created.high >>> 0).toNumber() : message.time_created; - if (message.participants && message.participants.length) { - object.participants = []; - for (let j = 0; j < message.participants.length; ++j) - object.participants[j] = $root.query.Target.toObject(message.participants[j], options); + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = $root.query.BoundQuery.toObject(message.query, options); + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.query.ExecuteOptions.toObject(message.options, options); + if (message.pre_queries && message.pre_queries.length) { + object.pre_queries = []; + for (let j = 0; j < message.pre_queries.length; ++j) + object.pre_queries[j] = message.pre_queries[j]; } - return object; - }; - + if (message.post_begin_queries && message.post_begin_queries.length) { + object.post_begin_queries = []; + for (let j = 0; j < message.post_begin_queries.length; ++j) + object.post_begin_queries[j] = message.post_begin_queries[j]; + } + return object; + }; + /** - * Converts this TransactionMetadata to JSON. + * Converts this ReserveBeginStreamExecuteRequest to JSON. * @function toJSON - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @instance * @returns {Object.} JSON object */ - TransactionMetadata.prototype.toJSON = function toJSON() { + ReserveBeginStreamExecuteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TransactionMetadata + * Gets the default type url for ReserveBeginStreamExecuteRequest * @function getTypeUrl - * @memberof query.TransactionMetadata + * @memberof query.ReserveBeginStreamExecuteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TransactionMetadata.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveBeginStreamExecuteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.TransactionMetadata"; + return typeUrlPrefix + "/query.ReserveBeginStreamExecuteRequest"; }; - return TransactionMetadata; - })(); - - /** - * SchemaTableType enum. - * @name query.SchemaTableType - * @enum {number} - * @property {number} VIEWS=0 VIEWS value - * @property {number} TABLES=1 TABLES value - * @property {number} ALL=2 ALL value - */ - query.SchemaTableType = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "VIEWS"] = 0; - values[valuesById[1] = "TABLES"] = 1; - values[valuesById[2] = "ALL"] = 2; - return values; + return ReserveBeginStreamExecuteRequest; })(); - query.GetSchemaRequest = (function() { + query.ReserveBeginStreamExecuteResponse = (function() { /** - * Properties of a GetSchemaRequest. + * Properties of a ReserveBeginStreamExecuteResponse. * @memberof query - * @interface IGetSchemaRequest - * @property {query.ITarget|null} [target] GetSchemaRequest target - * @property {query.SchemaTableType|null} [table_type] GetSchemaRequest table_type - * @property {Array.|null} [table_names] GetSchemaRequest table_names + * @interface IReserveBeginStreamExecuteResponse + * @property {vtrpc.IRPCError|null} [error] ReserveBeginStreamExecuteResponse error + * @property {query.IQueryResult|null} [result] ReserveBeginStreamExecuteResponse result + * @property {number|Long|null} [transaction_id] ReserveBeginStreamExecuteResponse transaction_id + * @property {number|Long|null} [reserved_id] ReserveBeginStreamExecuteResponse reserved_id + * @property {topodata.ITabletAlias|null} [tablet_alias] ReserveBeginStreamExecuteResponse tablet_alias + * @property {string|null} [session_state_changes] ReserveBeginStreamExecuteResponse session_state_changes */ /** - * Constructs a new GetSchemaRequest. + * Constructs a new ReserveBeginStreamExecuteResponse. * @memberof query - * @classdesc Represents a GetSchemaRequest. - * @implements IGetSchemaRequest + * @classdesc Represents a ReserveBeginStreamExecuteResponse. + * @implements IReserveBeginStreamExecuteResponse * @constructor - * @param {query.IGetSchemaRequest=} [properties] Properties to set + * @param {query.IReserveBeginStreamExecuteResponse=} [properties] Properties to set */ - function GetSchemaRequest(properties) { - this.table_names = []; + function ReserveBeginStreamExecuteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -92634,106 +93088,145 @@ export const query = $root.query = (() => { } /** - * GetSchemaRequest target. - * @member {query.ITarget|null|undefined} target - * @memberof query.GetSchemaRequest + * ReserveBeginStreamExecuteResponse error. + * @member {vtrpc.IRPCError|null|undefined} error + * @memberof query.ReserveBeginStreamExecuteResponse * @instance */ - GetSchemaRequest.prototype.target = null; + ReserveBeginStreamExecuteResponse.prototype.error = null; /** - * GetSchemaRequest table_type. - * @member {query.SchemaTableType} table_type - * @memberof query.GetSchemaRequest + * ReserveBeginStreamExecuteResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof query.ReserveBeginStreamExecuteResponse * @instance */ - GetSchemaRequest.prototype.table_type = 0; + ReserveBeginStreamExecuteResponse.prototype.result = null; /** - * GetSchemaRequest table_names. - * @member {Array.} table_names - * @memberof query.GetSchemaRequest + * ReserveBeginStreamExecuteResponse transaction_id. + * @member {number|Long} transaction_id + * @memberof query.ReserveBeginStreamExecuteResponse * @instance */ - GetSchemaRequest.prototype.table_names = $util.emptyArray; + ReserveBeginStreamExecuteResponse.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new GetSchemaRequest instance using the specified properties. + * ReserveBeginStreamExecuteResponse reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ReserveBeginStreamExecuteResponse + * @instance + */ + ReserveBeginStreamExecuteResponse.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ReserveBeginStreamExecuteResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof query.ReserveBeginStreamExecuteResponse + * @instance + */ + ReserveBeginStreamExecuteResponse.prototype.tablet_alias = null; + + /** + * ReserveBeginStreamExecuteResponse session_state_changes. + * @member {string} session_state_changes + * @memberof query.ReserveBeginStreamExecuteResponse + * @instance + */ + ReserveBeginStreamExecuteResponse.prototype.session_state_changes = ""; + + /** + * Creates a new ReserveBeginStreamExecuteResponse instance using the specified properties. * @function create - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static - * @param {query.IGetSchemaRequest=} [properties] Properties to set - * @returns {query.GetSchemaRequest} GetSchemaRequest instance + * @param {query.IReserveBeginStreamExecuteResponse=} [properties] Properties to set + * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse instance */ - GetSchemaRequest.create = function create(properties) { - return new GetSchemaRequest(properties); + ReserveBeginStreamExecuteResponse.create = function create(properties) { + return new ReserveBeginStreamExecuteResponse(properties); }; /** - * Encodes the specified GetSchemaRequest message. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. + * Encodes the specified ReserveBeginStreamExecuteResponse message. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. * @function encode - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static - * @param {query.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode + * @param {query.IReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaRequest.encode = function encode(message, writer) { + ReserveBeginStreamExecuteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.query.Target.encode(message.target, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.table_type != null && Object.hasOwnProperty.call(message, "table_type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.table_type); - if (message.table_names != null && message.table_names.length) - for (let i = 0; i < message.table_names.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.table_names[i]); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.vtrpc.RPCError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.transaction_id); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.reserved_id); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.session_state_changes != null && Object.hasOwnProperty.call(message, "session_state_changes")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.session_state_changes); return writer; }; /** - * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. + * Encodes the specified ReserveBeginStreamExecuteResponse message, length delimited. Does not implicitly {@link query.ReserveBeginStreamExecuteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static - * @param {query.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode + * @param {query.IReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReserveBeginStreamExecuteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer. + * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer. * @function decode - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.GetSchemaRequest} GetSchemaRequest + * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaRequest.decode = function decode(reader, length) { + ReserveBeginStreamExecuteResponse.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.query.GetSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReserveBeginStreamExecuteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.target = $root.query.Target.decode(reader, reader.uint32()); + message.error = $root.vtrpc.RPCError.decode(reader, reader.uint32()); break; } case 2: { - message.table_type = reader.int32(); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } case 3: { - if (!(message.table_names && message.table_names.length)) - message.table_names = []; - message.table_names.push(reader.string()); + message.transaction_id = reader.int64(); + break; + } + case 4: { + message.reserved_id = reader.int64(); + break; + } + case 5: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 6: { + message.session_state_changes = reader.string(); break; } default: @@ -92745,182 +93238,210 @@ export const query = $root.query = (() => { }; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a ReserveBeginStreamExecuteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.GetSchemaRequest} GetSchemaRequest + * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + ReserveBeginStreamExecuteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSchemaRequest message. + * Verifies a ReserveBeginStreamExecuteResponse message. * @function verify - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSchemaRequest.verify = function verify(message) { + ReserveBeginStreamExecuteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.query.Target.verify(message.target); + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.vtrpc.RPCError.verify(message.error); if (error) - return "target." + error; + return "error." + error; } - if (message.table_type != null && message.hasOwnProperty("table_type")) - switch (message.table_type) { - default: - return "table_type: enum value expected"; - case 0: - case 1: - case 2: - break; - } - if (message.table_names != null && message.hasOwnProperty("table_names")) { - if (!Array.isArray(message.table_names)) - return "table_names: array expected"; - for (let i = 0; i < message.table_names.length; ++i) - if (!$util.isString(message.table_names[i])) - return "table_names: string[] expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; } + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + if (!$util.isString(message.session_state_changes)) + return "session_state_changes: string expected"; return null; }; /** - * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReserveBeginStreamExecuteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static * @param {Object.} object Plain object - * @returns {query.GetSchemaRequest} GetSchemaRequest + * @returns {query.ReserveBeginStreamExecuteResponse} ReserveBeginStreamExecuteResponse */ - GetSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.query.GetSchemaRequest) + ReserveBeginStreamExecuteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReserveBeginStreamExecuteResponse) return object; - let message = new $root.query.GetSchemaRequest(); - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".query.GetSchemaRequest.target: object expected"); - message.target = $root.query.Target.fromObject(object.target); + let message = new $root.query.ReserveBeginStreamExecuteResponse(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteResponse.error: object expected"); + message.error = $root.vtrpc.RPCError.fromObject(object.error); } - switch (object.table_type) { - default: - if (typeof object.table_type === "number") { - message.table_type = object.table_type; - break; - } - break; - case "VIEWS": - case 0: - message.table_type = 0; - break; - case "TABLES": - case 1: - message.table_type = 1; - break; - case "ALL": - case 2: - message.table_type = 2; - break; + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } - if (object.table_names) { - if (!Array.isArray(object.table_names)) - throw TypeError(".query.GetSchemaRequest.table_names: array expected"); - message.table_names = []; - for (let i = 0; i < object.table_names.length; ++i) - message.table_names[i] = String(object.table_names[i]); + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".query.ReserveBeginStreamExecuteResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.session_state_changes != null) + message.session_state_changes = String(object.session_state_changes); return message; }; /** - * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReserveBeginStreamExecuteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static - * @param {query.GetSchemaRequest} message GetSchemaRequest + * @param {query.ReserveBeginStreamExecuteResponse} message ReserveBeginStreamExecuteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSchemaRequest.toObject = function toObject(message, options) { + ReserveBeginStreamExecuteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.table_names = []; if (options.defaults) { - object.target = null; - object.table_type = options.enums === String ? "VIEWS" : 0; - } - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.query.Target.toObject(message.target, options); - if (message.table_type != null && message.hasOwnProperty("table_type")) - object.table_type = options.enums === String ? $root.query.SchemaTableType[message.table_type] === undefined ? message.table_type : $root.query.SchemaTableType[message.table_type] : message.table_type; - if (message.table_names && message.table_names.length) { - object.table_names = []; - for (let j = 0; j < message.table_names.length; ++j) - object.table_names[j] = message.table_names[j]; + object.error = null; + object.result = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; + object.tablet_alias = null; + object.session_state_changes = ""; } + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.vtrpc.RPCError.toObject(message.error, options); + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.session_state_changes != null && message.hasOwnProperty("session_state_changes")) + object.session_state_changes = message.session_state_changes; return object; }; /** - * Converts this GetSchemaRequest to JSON. + * Converts this ReserveBeginStreamExecuteResponse to JSON. * @function toJSON - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @instance * @returns {Object.} JSON object */ - GetSchemaRequest.prototype.toJSON = function toJSON() { + ReserveBeginStreamExecuteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSchemaRequest + * Gets the default type url for ReserveBeginStreamExecuteResponse * @function getTypeUrl - * @memberof query.GetSchemaRequest + * @memberof query.ReserveBeginStreamExecuteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReserveBeginStreamExecuteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.GetSchemaRequest"; + return typeUrlPrefix + "/query.ReserveBeginStreamExecuteResponse"; }; - return GetSchemaRequest; + return ReserveBeginStreamExecuteResponse; })(); - query.GetSchemaResponse = (function() { + query.ReleaseRequest = (function() { /** - * Properties of a GetSchemaResponse. + * Properties of a ReleaseRequest. * @memberof query - * @interface IGetSchemaResponse - * @property {Object.|null} [table_definition] GetSchemaResponse table_definition + * @interface IReleaseRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] ReleaseRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] ReleaseRequest immediate_caller_id + * @property {query.ITarget|null} [target] ReleaseRequest target + * @property {number|Long|null} [transaction_id] ReleaseRequest transaction_id + * @property {number|Long|null} [reserved_id] ReleaseRequest reserved_id */ /** - * Constructs a new GetSchemaResponse. + * Constructs a new ReleaseRequest. * @memberof query - * @classdesc Represents a GetSchemaResponse. - * @implements IGetSchemaResponse + * @classdesc Represents a ReleaseRequest. + * @implements IReleaseRequest * @constructor - * @param {query.IGetSchemaResponse=} [properties] Properties to set + * @param {query.IReleaseRequest=} [properties] Properties to set */ - function GetSchemaResponse(properties) { - this.table_definition = {}; + function ReleaseRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -92928,95 +93449,131 @@ export const query = $root.query = (() => { } /** - * GetSchemaResponse table_definition. - * @member {Object.} table_definition - * @memberof query.GetSchemaResponse + * ReleaseRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof query.ReleaseRequest * @instance */ - GetSchemaResponse.prototype.table_definition = $util.emptyObject; + ReleaseRequest.prototype.effective_caller_id = null; /** - * Creates a new GetSchemaResponse instance using the specified properties. + * ReleaseRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof query.ReleaseRequest + * @instance + */ + ReleaseRequest.prototype.immediate_caller_id = null; + + /** + * ReleaseRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.ReleaseRequest + * @instance + */ + ReleaseRequest.prototype.target = null; + + /** + * ReleaseRequest transaction_id. + * @member {number|Long} transaction_id + * @memberof query.ReleaseRequest + * @instance + */ + ReleaseRequest.prototype.transaction_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ReleaseRequest reserved_id. + * @member {number|Long} reserved_id + * @memberof query.ReleaseRequest + * @instance + */ + ReleaseRequest.prototype.reserved_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Creates a new ReleaseRequest instance using the specified properties. * @function create - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static - * @param {query.IGetSchemaResponse=} [properties] Properties to set - * @returns {query.GetSchemaResponse} GetSchemaResponse instance + * @param {query.IReleaseRequest=} [properties] Properties to set + * @returns {query.ReleaseRequest} ReleaseRequest instance */ - GetSchemaResponse.create = function create(properties) { - return new GetSchemaResponse(properties); + ReleaseRequest.create = function create(properties) { + return new ReleaseRequest(properties); }; /** - * Encodes the specified GetSchemaResponse message. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. + * Encodes the specified ReleaseRequest message. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. * @function encode - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static - * @param {query.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode + * @param {query.IReleaseRequest} message ReleaseRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaResponse.encode = function encode(message, writer) { + ReleaseRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.table_definition != null && Object.hasOwnProperty.call(message, "table_definition")) - for (let keys = Object.keys(message.table_definition), i = 0; i < keys.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.table_definition[keys[i]]).ldelim(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.transaction_id != null && Object.hasOwnProperty.call(message, "transaction_id")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.transaction_id); + if (message.reserved_id != null && Object.hasOwnProperty.call(message, "reserved_id")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.reserved_id); return writer; }; /** - * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. + * Encodes the specified ReleaseRequest message, length delimited. Does not implicitly {@link query.ReleaseRequest.verify|verify} messages. * @function encodeDelimited - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static - * @param {query.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode + * @param {query.IReleaseRequest} message ReleaseRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReleaseRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer. + * Decodes a ReleaseRequest message from the specified reader or buffer. * @function decode - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {query.GetSchemaResponse} GetSchemaResponse + * @returns {query.ReleaseRequest} ReleaseRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaResponse.decode = function decode(reader, length) { + ReleaseRequest.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.query.GetSchemaResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.ReleaseRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } case 2: { - if (message.table_definition === $util.emptyObject) - message.table_definition = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = ""; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = reader.string(); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.table_definition[key] = value; + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + case 4: { + message.transaction_id = reader.int64(); + break; + } + case 5: { + message.reserved_id = reader.int64(); break; } default: @@ -93028,169 +93585,197 @@ export const query = $root.query = (() => { }; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a ReleaseRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {query.GetSchemaResponse} GetSchemaResponse + * @returns {query.ReleaseRequest} ReleaseRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + ReleaseRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSchemaResponse message. + * Verifies a ReleaseRequest message. * @function verify - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSchemaResponse.verify = function verify(message) { + ReleaseRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.table_definition != null && message.hasOwnProperty("table_definition")) { - if (!$util.isObject(message.table_definition)) - return "table_definition: object expected"; - let key = Object.keys(message.table_definition); - for (let i = 0; i < key.length; ++i) - if (!$util.isString(message.table_definition[key[i]])) - return "table_definition: string{k:string} expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; } + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (!$util.isInteger(message.transaction_id) && !(message.transaction_id && $util.isInteger(message.transaction_id.low) && $util.isInteger(message.transaction_id.high))) + return "transaction_id: integer|Long expected"; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (!$util.isInteger(message.reserved_id) && !(message.reserved_id && $util.isInteger(message.reserved_id.low) && $util.isInteger(message.reserved_id.high))) + return "reserved_id: integer|Long expected"; return null; }; /** - * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReleaseRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static * @param {Object.} object Plain object - * @returns {query.GetSchemaResponse} GetSchemaResponse + * @returns {query.ReleaseRequest} ReleaseRequest */ - GetSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.query.GetSchemaResponse) + ReleaseRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReleaseRequest) return object; - let message = new $root.query.GetSchemaResponse(); - if (object.table_definition) { - if (typeof object.table_definition !== "object") - throw TypeError(".query.GetSchemaResponse.table_definition: object expected"); - message.table_definition = {}; - for (let keys = Object.keys(object.table_definition), i = 0; i < keys.length; ++i) - message.table_definition[keys[i]] = String(object.table_definition[keys[i]]); + let message = new $root.query.ReleaseRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".query.ReleaseRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".query.ReleaseRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.ReleaseRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); } + if (object.transaction_id != null) + if ($util.Long) + (message.transaction_id = $util.Long.fromValue(object.transaction_id)).unsigned = false; + else if (typeof object.transaction_id === "string") + message.transaction_id = parseInt(object.transaction_id, 10); + else if (typeof object.transaction_id === "number") + message.transaction_id = object.transaction_id; + else if (typeof object.transaction_id === "object") + message.transaction_id = new $util.LongBits(object.transaction_id.low >>> 0, object.transaction_id.high >>> 0).toNumber(); + if (object.reserved_id != null) + if ($util.Long) + (message.reserved_id = $util.Long.fromValue(object.reserved_id)).unsigned = false; + else if (typeof object.reserved_id === "string") + message.reserved_id = parseInt(object.reserved_id, 10); + else if (typeof object.reserved_id === "number") + message.reserved_id = object.reserved_id; + else if (typeof object.reserved_id === "object") + message.reserved_id = new $util.LongBits(object.reserved_id.low >>> 0, object.reserved_id.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReleaseRequest message. Also converts values to other types if specified. * @function toObject - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static - * @param {query.GetSchemaResponse} message GetSchemaResponse + * @param {query.ReleaseRequest} message ReleaseRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSchemaResponse.toObject = function toObject(message, options) { + ReleaseRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.table_definition = {}; - let keys2; - if (message.table_definition && (keys2 = Object.keys(message.table_definition)).length) { - object.table_definition = {}; - for (let j = 0; j < keys2.length; ++j) - object.table_definition[keys2[j]] = message.table_definition[keys2[j]]; + if (options.defaults) { + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.transaction_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.transaction_id = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.reserved_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.reserved_id = options.longs === String ? "0" : 0; } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.transaction_id != null && message.hasOwnProperty("transaction_id")) + if (typeof message.transaction_id === "number") + object.transaction_id = options.longs === String ? String(message.transaction_id) : message.transaction_id; + else + object.transaction_id = options.longs === String ? $util.Long.prototype.toString.call(message.transaction_id) : options.longs === Number ? new $util.LongBits(message.transaction_id.low >>> 0, message.transaction_id.high >>> 0).toNumber() : message.transaction_id; + if (message.reserved_id != null && message.hasOwnProperty("reserved_id")) + if (typeof message.reserved_id === "number") + object.reserved_id = options.longs === String ? String(message.reserved_id) : message.reserved_id; + else + object.reserved_id = options.longs === String ? $util.Long.prototype.toString.call(message.reserved_id) : options.longs === Number ? new $util.LongBits(message.reserved_id.low >>> 0, message.reserved_id.high >>> 0).toNumber() : message.reserved_id; return object; }; /** - * Converts this GetSchemaResponse to JSON. + * Converts this ReleaseRequest to JSON. * @function toJSON - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @instance * @returns {Object.} JSON object */ - GetSchemaResponse.prototype.toJSON = function toJSON() { + ReleaseRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSchemaResponse + * Gets the default type url for ReleaseRequest * @function getTypeUrl - * @memberof query.GetSchemaResponse + * @memberof query.ReleaseRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReleaseRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/query.GetSchemaResponse"; + return typeUrlPrefix + "/query.ReleaseRequest"; }; - return GetSchemaResponse; + return ReleaseRequest; })(); - return query; -})(); - -export const replicationdata = $root.replicationdata = (() => { - - /** - * Namespace replicationdata. - * @exports replicationdata - * @namespace - */ - const replicationdata = {}; - - replicationdata.Status = (function() { + query.ReleaseResponse = (function() { /** - * Properties of a Status. - * @memberof replicationdata - * @interface IStatus - * @property {string|null} [position] Status position - * @property {number|null} [replication_lag_seconds] Status replication_lag_seconds - * @property {string|null} [source_host] Status source_host - * @property {number|null} [source_port] Status source_port - * @property {number|null} [connect_retry] Status connect_retry - * @property {string|null} [relay_log_position] Status relay_log_position - * @property {string|null} [file_position] Status file_position - * @property {string|null} [relay_log_source_binlog_equivalent_position] Status relay_log_source_binlog_equivalent_position - * @property {number|null} [source_server_id] Status source_server_id - * @property {string|null} [source_uuid] Status source_uuid - * @property {number|null} [io_state] Status io_state - * @property {string|null} [last_io_error] Status last_io_error - * @property {number|null} [sql_state] Status sql_state - * @property {string|null} [last_sql_error] Status last_sql_error - * @property {string|null} [relay_log_file_position] Status relay_log_file_position - * @property {string|null} [source_user] Status source_user - * @property {number|null} [sql_delay] Status sql_delay - * @property {boolean|null} [auto_position] Status auto_position - * @property {boolean|null} [using_gtid] Status using_gtid - * @property {boolean|null} [has_replication_filters] Status has_replication_filters - * @property {boolean|null} [ssl_allowed] Status ssl_allowed - * @property {boolean|null} [replication_lag_unknown] Status replication_lag_unknown + * Properties of a ReleaseResponse. + * @memberof query + * @interface IReleaseResponse */ /** - * Constructs a new Status. - * @memberof replicationdata - * @classdesc Represents a Status. - * @implements IStatus + * Constructs a new ReleaseResponse. + * @memberof query + * @classdesc Represents a ReleaseResponse. + * @implements IReleaseResponse * @constructor - * @param {replicationdata.IStatus=} [properties] Properties to set + * @param {query.IReleaseResponse=} [properties] Properties to set */ - function Status(properties) { + function ReleaseResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -93198,369 +93783,539 @@ export const replicationdata = $root.replicationdata = (() => { } /** - * Status position. - * @member {string} position - * @memberof replicationdata.Status - * @instance + * Creates a new ReleaseResponse instance using the specified properties. + * @function create + * @memberof query.ReleaseResponse + * @static + * @param {query.IReleaseResponse=} [properties] Properties to set + * @returns {query.ReleaseResponse} ReleaseResponse instance */ - Status.prototype.position = ""; + ReleaseResponse.create = function create(properties) { + return new ReleaseResponse(properties); + }; /** - * Status replication_lag_seconds. - * @member {number} replication_lag_seconds - * @memberof replicationdata.Status - * @instance + * Encodes the specified ReleaseResponse message. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * @function encode + * @memberof query.ReleaseResponse + * @static + * @param {query.IReleaseResponse} message ReleaseResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer */ - Status.prototype.replication_lag_seconds = 0; + ReleaseResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; /** - * Status source_host. - * @member {string} source_host - * @memberof replicationdata.Status - * @instance + * Encodes the specified ReleaseResponse message, length delimited. Does not implicitly {@link query.ReleaseResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof query.ReleaseResponse + * @static + * @param {query.IReleaseResponse} message ReleaseResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer */ - Status.prototype.source_host = ""; + ReleaseResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; /** - * Status source_port. - * @member {number} source_port - * @memberof replicationdata.Status - * @instance + * Decodes a ReleaseResponse message from the specified reader or buffer. + * @function decode + * @memberof query.ReleaseResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {query.ReleaseResponse} ReleaseResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.prototype.source_port = 0; + ReleaseResponse.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.query.ReleaseResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; /** - * Status connect_retry. - * @member {number} connect_retry - * @memberof replicationdata.Status - * @instance + * Decodes a ReleaseResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof query.ReleaseResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {query.ReleaseResponse} ReleaseResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.prototype.connect_retry = 0; + ReleaseResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; /** - * Status relay_log_position. - * @member {string} relay_log_position - * @memberof replicationdata.Status - * @instance + * Verifies a ReleaseResponse message. + * @function verify + * @memberof query.ReleaseResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Status.prototype.relay_log_position = ""; + ReleaseResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; /** - * Status file_position. - * @member {string} file_position - * @memberof replicationdata.Status - * @instance + * Creates a ReleaseResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof query.ReleaseResponse + * @static + * @param {Object.} object Plain object + * @returns {query.ReleaseResponse} ReleaseResponse */ - Status.prototype.file_position = ""; + ReleaseResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.ReleaseResponse) + return object; + return new $root.query.ReleaseResponse(); + }; /** - * Status relay_log_source_binlog_equivalent_position. - * @member {string} relay_log_source_binlog_equivalent_position - * @memberof replicationdata.Status - * @instance + * Creates a plain object from a ReleaseResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof query.ReleaseResponse + * @static + * @param {query.ReleaseResponse} message ReleaseResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object */ - Status.prototype.relay_log_source_binlog_equivalent_position = ""; + ReleaseResponse.toObject = function toObject() { + return {}; + }; /** - * Status source_server_id. - * @member {number} source_server_id - * @memberof replicationdata.Status + * Converts this ReleaseResponse to JSON. + * @function toJSON + * @memberof query.ReleaseResponse * @instance + * @returns {Object.} JSON object */ - Status.prototype.source_server_id = 0; + ReleaseResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; /** - * Status source_uuid. - * @member {string} source_uuid - * @memberof replicationdata.Status - * @instance + * Gets the default type url for ReleaseResponse + * @function getTypeUrl + * @memberof query.ReleaseResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url */ - Status.prototype.source_uuid = ""; + ReleaseResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/query.ReleaseResponse"; + }; - /** - * Status io_state. - * @member {number} io_state - * @memberof replicationdata.Status - * @instance - */ - Status.prototype.io_state = 0; + return ReleaseResponse; + })(); + + query.StreamHealthRequest = (function() { /** - * Status last_io_error. - * @member {string} last_io_error - * @memberof replicationdata.Status - * @instance + * Properties of a StreamHealthRequest. + * @memberof query + * @interface IStreamHealthRequest */ - Status.prototype.last_io_error = ""; /** - * Status sql_state. - * @member {number} sql_state - * @memberof replicationdata.Status - * @instance + * Constructs a new StreamHealthRequest. + * @memberof query + * @classdesc Represents a StreamHealthRequest. + * @implements IStreamHealthRequest + * @constructor + * @param {query.IStreamHealthRequest=} [properties] Properties to set */ - Status.prototype.sql_state = 0; + function StreamHealthRequest(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]]; + } /** - * Status last_sql_error. - * @member {string} last_sql_error - * @memberof replicationdata.Status - * @instance + * Creates a new StreamHealthRequest instance using the specified properties. + * @function create + * @memberof query.StreamHealthRequest + * @static + * @param {query.IStreamHealthRequest=} [properties] Properties to set + * @returns {query.StreamHealthRequest} StreamHealthRequest instance */ - Status.prototype.last_sql_error = ""; + StreamHealthRequest.create = function create(properties) { + return new StreamHealthRequest(properties); + }; /** - * Status relay_log_file_position. - * @member {string} relay_log_file_position - * @memberof replicationdata.Status - * @instance + * Encodes the specified StreamHealthRequest message. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * @function encode + * @memberof query.StreamHealthRequest + * @static + * @param {query.IStreamHealthRequest} message StreamHealthRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer */ - Status.prototype.relay_log_file_position = ""; + StreamHealthRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; /** - * Status source_user. - * @member {string} source_user - * @memberof replicationdata.Status - * @instance + * Encodes the specified StreamHealthRequest message, length delimited. Does not implicitly {@link query.StreamHealthRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof query.StreamHealthRequest + * @static + * @param {query.IStreamHealthRequest} message StreamHealthRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer */ - Status.prototype.source_user = ""; + StreamHealthRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; /** - * Status sql_delay. - * @member {number} sql_delay - * @memberof replicationdata.Status - * @instance + * Decodes a StreamHealthRequest message from the specified reader or buffer. + * @function decode + * @memberof query.StreamHealthRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {query.StreamHealthRequest} StreamHealthRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.prototype.sql_delay = 0; + StreamHealthRequest.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.query.StreamHealthRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; /** - * Status auto_position. - * @member {boolean} auto_position - * @memberof replicationdata.Status - * @instance + * Decodes a StreamHealthRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof query.StreamHealthRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {query.StreamHealthRequest} StreamHealthRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.prototype.auto_position = false; + StreamHealthRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; /** - * Status using_gtid. - * @member {boolean} using_gtid - * @memberof replicationdata.Status + * Verifies a StreamHealthRequest message. + * @function verify + * @memberof query.StreamHealthRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + StreamHealthRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates a StreamHealthRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof query.StreamHealthRequest + * @static + * @param {Object.} object Plain object + * @returns {query.StreamHealthRequest} StreamHealthRequest + */ + StreamHealthRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamHealthRequest) + return object; + return new $root.query.StreamHealthRequest(); + }; + + /** + * Creates a plain object from a StreamHealthRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof query.StreamHealthRequest + * @static + * @param {query.StreamHealthRequest} message StreamHealthRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + StreamHealthRequest.toObject = function toObject() { + return {}; + }; + + /** + * Converts this StreamHealthRequest to JSON. + * @function toJSON + * @memberof query.StreamHealthRequest * @instance + * @returns {Object.} JSON object */ - Status.prototype.using_gtid = false; + StreamHealthRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; /** - * Status has_replication_filters. - * @member {boolean} has_replication_filters - * @memberof replicationdata.Status + * Gets the default type url for StreamHealthRequest + * @function getTypeUrl + * @memberof query.StreamHealthRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + StreamHealthRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/query.StreamHealthRequest"; + }; + + return StreamHealthRequest; + })(); + + query.RealtimeStats = (function() { + + /** + * Properties of a RealtimeStats. + * @memberof query + * @interface IRealtimeStats + * @property {string|null} [health_error] RealtimeStats health_error + * @property {number|null} [replication_lag_seconds] RealtimeStats replication_lag_seconds + * @property {number|null} [binlog_players_count] RealtimeStats binlog_players_count + * @property {number|Long|null} [filtered_replication_lag_seconds] RealtimeStats filtered_replication_lag_seconds + * @property {number|null} [cpu_usage] RealtimeStats cpu_usage + * @property {number|null} [qps] RealtimeStats qps + * @property {Array.|null} [table_schema_changed] RealtimeStats table_schema_changed + * @property {Array.|null} [view_schema_changed] RealtimeStats view_schema_changed + */ + + /** + * Constructs a new RealtimeStats. + * @memberof query + * @classdesc Represents a RealtimeStats. + * @implements IRealtimeStats + * @constructor + * @param {query.IRealtimeStats=} [properties] Properties to set + */ + function RealtimeStats(properties) { + this.table_schema_changed = []; + this.view_schema_changed = []; + 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]]; + } + + /** + * RealtimeStats health_error. + * @member {string} health_error + * @memberof query.RealtimeStats * @instance */ - Status.prototype.has_replication_filters = false; + RealtimeStats.prototype.health_error = ""; /** - * Status ssl_allowed. - * @member {boolean} ssl_allowed - * @memberof replicationdata.Status + * RealtimeStats replication_lag_seconds. + * @member {number} replication_lag_seconds + * @memberof query.RealtimeStats * @instance */ - Status.prototype.ssl_allowed = false; + RealtimeStats.prototype.replication_lag_seconds = 0; /** - * Status replication_lag_unknown. - * @member {boolean} replication_lag_unknown - * @memberof replicationdata.Status + * RealtimeStats binlog_players_count. + * @member {number} binlog_players_count + * @memberof query.RealtimeStats * @instance */ - Status.prototype.replication_lag_unknown = false; + RealtimeStats.prototype.binlog_players_count = 0; /** - * Creates a new Status instance using the specified properties. + * RealtimeStats filtered_replication_lag_seconds. + * @member {number|Long} filtered_replication_lag_seconds + * @memberof query.RealtimeStats + * @instance + */ + RealtimeStats.prototype.filtered_replication_lag_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * RealtimeStats cpu_usage. + * @member {number} cpu_usage + * @memberof query.RealtimeStats + * @instance + */ + RealtimeStats.prototype.cpu_usage = 0; + + /** + * RealtimeStats qps. + * @member {number} qps + * @memberof query.RealtimeStats + * @instance + */ + RealtimeStats.prototype.qps = 0; + + /** + * RealtimeStats table_schema_changed. + * @member {Array.} table_schema_changed + * @memberof query.RealtimeStats + * @instance + */ + RealtimeStats.prototype.table_schema_changed = $util.emptyArray; + + /** + * RealtimeStats view_schema_changed. + * @member {Array.} view_schema_changed + * @memberof query.RealtimeStats + * @instance + */ + RealtimeStats.prototype.view_schema_changed = $util.emptyArray; + + /** + * Creates a new RealtimeStats instance using the specified properties. * @function create - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static - * @param {replicationdata.IStatus=} [properties] Properties to set - * @returns {replicationdata.Status} Status instance + * @param {query.IRealtimeStats=} [properties] Properties to set + * @returns {query.RealtimeStats} RealtimeStats instance */ - Status.create = function create(properties) { - return new Status(properties); + RealtimeStats.create = function create(properties) { + return new RealtimeStats(properties); }; /** - * Encodes the specified Status message. Does not implicitly {@link replicationdata.Status.verify|verify} messages. + * Encodes the specified RealtimeStats message. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. * @function encode - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static - * @param {replicationdata.IStatus} message Status message or plain object to encode + * @param {query.IRealtimeStats} message RealtimeStats message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Status.encode = function encode(message, writer) { + RealtimeStats.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); + if (message.health_error != null && Object.hasOwnProperty.call(message, "health_error")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.health_error); if (message.replication_lag_seconds != null && Object.hasOwnProperty.call(message, "replication_lag_seconds")) - writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.replication_lag_seconds); - if (message.source_host != null && Object.hasOwnProperty.call(message, "source_host")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_host); - if (message.source_port != null && Object.hasOwnProperty.call(message, "source_port")) - writer.uint32(/* id 6, wireType 0 =*/48).int32(message.source_port); - if (message.connect_retry != null && Object.hasOwnProperty.call(message, "connect_retry")) - writer.uint32(/* id 7, wireType 0 =*/56).int32(message.connect_retry); - if (message.relay_log_position != null && Object.hasOwnProperty.call(message, "relay_log_position")) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.relay_log_position); - if (message.file_position != null && Object.hasOwnProperty.call(message, "file_position")) - writer.uint32(/* id 9, wireType 2 =*/74).string(message.file_position); - if (message.relay_log_source_binlog_equivalent_position != null && Object.hasOwnProperty.call(message, "relay_log_source_binlog_equivalent_position")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.relay_log_source_binlog_equivalent_position); - if (message.source_server_id != null && Object.hasOwnProperty.call(message, "source_server_id")) - writer.uint32(/* id 11, wireType 0 =*/88).uint32(message.source_server_id); - if (message.source_uuid != null && Object.hasOwnProperty.call(message, "source_uuid")) - writer.uint32(/* id 12, wireType 2 =*/98).string(message.source_uuid); - if (message.io_state != null && Object.hasOwnProperty.call(message, "io_state")) - writer.uint32(/* id 13, wireType 0 =*/104).int32(message.io_state); - if (message.last_io_error != null && Object.hasOwnProperty.call(message, "last_io_error")) - writer.uint32(/* id 14, wireType 2 =*/114).string(message.last_io_error); - if (message.sql_state != null && Object.hasOwnProperty.call(message, "sql_state")) - writer.uint32(/* id 15, wireType 0 =*/120).int32(message.sql_state); - if (message.last_sql_error != null && Object.hasOwnProperty.call(message, "last_sql_error")) - writer.uint32(/* id 16, wireType 2 =*/130).string(message.last_sql_error); - if (message.relay_log_file_position != null && Object.hasOwnProperty.call(message, "relay_log_file_position")) - writer.uint32(/* id 17, wireType 2 =*/138).string(message.relay_log_file_position); - if (message.source_user != null && Object.hasOwnProperty.call(message, "source_user")) - writer.uint32(/* id 18, wireType 2 =*/146).string(message.source_user); - if (message.sql_delay != null && Object.hasOwnProperty.call(message, "sql_delay")) - writer.uint32(/* id 19, wireType 0 =*/152).uint32(message.sql_delay); - if (message.auto_position != null && Object.hasOwnProperty.call(message, "auto_position")) - writer.uint32(/* id 20, wireType 0 =*/160).bool(message.auto_position); - if (message.using_gtid != null && Object.hasOwnProperty.call(message, "using_gtid")) - writer.uint32(/* id 21, wireType 0 =*/168).bool(message.using_gtid); - if (message.has_replication_filters != null && Object.hasOwnProperty.call(message, "has_replication_filters")) - writer.uint32(/* id 22, wireType 0 =*/176).bool(message.has_replication_filters); - if (message.ssl_allowed != null && Object.hasOwnProperty.call(message, "ssl_allowed")) - writer.uint32(/* id 23, wireType 0 =*/184).bool(message.ssl_allowed); - if (message.replication_lag_unknown != null && Object.hasOwnProperty.call(message, "replication_lag_unknown")) - writer.uint32(/* id 24, wireType 0 =*/192).bool(message.replication_lag_unknown); + writer.uint32(/* id 2, wireType 0 =*/16).uint32(message.replication_lag_seconds); + if (message.binlog_players_count != null && Object.hasOwnProperty.call(message, "binlog_players_count")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.binlog_players_count); + if (message.filtered_replication_lag_seconds != null && Object.hasOwnProperty.call(message, "filtered_replication_lag_seconds")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.filtered_replication_lag_seconds); + if (message.cpu_usage != null && Object.hasOwnProperty.call(message, "cpu_usage")) + writer.uint32(/* id 5, wireType 1 =*/41).double(message.cpu_usage); + if (message.qps != null && Object.hasOwnProperty.call(message, "qps")) + writer.uint32(/* id 6, wireType 1 =*/49).double(message.qps); + if (message.table_schema_changed != null && message.table_schema_changed.length) + for (let i = 0; i < message.table_schema_changed.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.table_schema_changed[i]); + if (message.view_schema_changed != null && message.view_schema_changed.length) + for (let i = 0; i < message.view_schema_changed.length; ++i) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.view_schema_changed[i]); return writer; }; /** - * Encodes the specified Status message, length delimited. Does not implicitly {@link replicationdata.Status.verify|verify} messages. + * Encodes the specified RealtimeStats message, length delimited. Does not implicitly {@link query.RealtimeStats.verify|verify} messages. * @function encodeDelimited - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static - * @param {replicationdata.IStatus} message Status message or plain object to encode + * @param {query.IRealtimeStats} message RealtimeStats message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Status.encodeDelimited = function encodeDelimited(message, writer) { + RealtimeStats.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Status message from the specified reader or buffer. + * Decodes a RealtimeStats message from the specified reader or buffer. * @function decode - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {replicationdata.Status} Status + * @returns {query.RealtimeStats} RealtimeStats * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.decode = function decode(reader, length) { + RealtimeStats.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.replicationdata.Status(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.RealtimeStats(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.position = reader.string(); + message.health_error = reader.string(); break; } - case 4: { + case 2: { message.replication_lag_seconds = reader.uint32(); break; } - case 5: { - message.source_host = reader.string(); - break; - } - case 6: { - message.source_port = reader.int32(); - break; - } - case 7: { - message.connect_retry = reader.int32(); - break; - } - case 8: { - message.relay_log_position = reader.string(); - break; - } - case 9: { - message.file_position = reader.string(); - break; - } - case 10: { - message.relay_log_source_binlog_equivalent_position = reader.string(); - break; - } - case 11: { - message.source_server_id = reader.uint32(); - break; - } - case 12: { - message.source_uuid = reader.string(); - break; - } - case 13: { - message.io_state = reader.int32(); - break; - } - case 14: { - message.last_io_error = reader.string(); - break; - } - case 15: { - message.sql_state = reader.int32(); - break; - } - case 16: { - message.last_sql_error = reader.string(); - break; - } - case 17: { - message.relay_log_file_position = reader.string(); - break; - } - case 18: { - message.source_user = reader.string(); - break; - } - case 19: { - message.sql_delay = reader.uint32(); + case 3: { + message.binlog_players_count = reader.int32(); break; } - case 20: { - message.auto_position = reader.bool(); + case 4: { + message.filtered_replication_lag_seconds = reader.int64(); break; } - case 21: { - message.using_gtid = reader.bool(); + case 5: { + message.cpu_usage = reader.double(); break; } - case 22: { - message.has_replication_filters = reader.bool(); + case 6: { + message.qps = reader.double(); break; } - case 23: { - message.ssl_allowed = reader.bool(); + case 7: { + if (!(message.table_schema_changed && message.table_schema_changed.length)) + message.table_schema_changed = []; + message.table_schema_changed.push(reader.string()); break; } - case 24: { - message.replication_lag_unknown = reader.bool(); + case 8: { + if (!(message.view_schema_changed && message.view_schema_changed.length)) + message.view_schema_changed = []; + message.view_schema_changed.push(reader.string()); break; } default: @@ -93572,292 +94327,222 @@ export const replicationdata = $root.replicationdata = (() => { }; /** - * Decodes a Status message from the specified reader or buffer, length delimited. + * Decodes a RealtimeStats message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {replicationdata.Status} Status + * @returns {query.RealtimeStats} RealtimeStats * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Status.decodeDelimited = function decodeDelimited(reader) { + RealtimeStats.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Status message. + * Verifies a RealtimeStats message. * @function verify - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Status.verify = function verify(message) { + RealtimeStats.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; + if (message.health_error != null && message.hasOwnProperty("health_error")) + if (!$util.isString(message.health_error)) + return "health_error: string expected"; if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) if (!$util.isInteger(message.replication_lag_seconds)) return "replication_lag_seconds: integer expected"; - if (message.source_host != null && message.hasOwnProperty("source_host")) - if (!$util.isString(message.source_host)) - return "source_host: string expected"; - if (message.source_port != null && message.hasOwnProperty("source_port")) - if (!$util.isInteger(message.source_port)) - return "source_port: integer expected"; - if (message.connect_retry != null && message.hasOwnProperty("connect_retry")) - if (!$util.isInteger(message.connect_retry)) - return "connect_retry: integer expected"; - if (message.relay_log_position != null && message.hasOwnProperty("relay_log_position")) - if (!$util.isString(message.relay_log_position)) - return "relay_log_position: string expected"; - if (message.file_position != null && message.hasOwnProperty("file_position")) - if (!$util.isString(message.file_position)) - return "file_position: string expected"; - if (message.relay_log_source_binlog_equivalent_position != null && message.hasOwnProperty("relay_log_source_binlog_equivalent_position")) - if (!$util.isString(message.relay_log_source_binlog_equivalent_position)) - return "relay_log_source_binlog_equivalent_position: string expected"; - if (message.source_server_id != null && message.hasOwnProperty("source_server_id")) - if (!$util.isInteger(message.source_server_id)) - return "source_server_id: integer expected"; - if (message.source_uuid != null && message.hasOwnProperty("source_uuid")) - if (!$util.isString(message.source_uuid)) - return "source_uuid: string expected"; - if (message.io_state != null && message.hasOwnProperty("io_state")) - if (!$util.isInteger(message.io_state)) - return "io_state: integer expected"; - if (message.last_io_error != null && message.hasOwnProperty("last_io_error")) - if (!$util.isString(message.last_io_error)) - return "last_io_error: string expected"; - if (message.sql_state != null && message.hasOwnProperty("sql_state")) - if (!$util.isInteger(message.sql_state)) - return "sql_state: integer expected"; - if (message.last_sql_error != null && message.hasOwnProperty("last_sql_error")) - if (!$util.isString(message.last_sql_error)) - return "last_sql_error: string expected"; - if (message.relay_log_file_position != null && message.hasOwnProperty("relay_log_file_position")) - if (!$util.isString(message.relay_log_file_position)) - return "relay_log_file_position: string expected"; - if (message.source_user != null && message.hasOwnProperty("source_user")) - if (!$util.isString(message.source_user)) - return "source_user: string expected"; - if (message.sql_delay != null && message.hasOwnProperty("sql_delay")) - if (!$util.isInteger(message.sql_delay)) - return "sql_delay: integer expected"; - if (message.auto_position != null && message.hasOwnProperty("auto_position")) - if (typeof message.auto_position !== "boolean") - return "auto_position: boolean expected"; - if (message.using_gtid != null && message.hasOwnProperty("using_gtid")) - if (typeof message.using_gtid !== "boolean") - return "using_gtid: boolean expected"; - if (message.has_replication_filters != null && message.hasOwnProperty("has_replication_filters")) - if (typeof message.has_replication_filters !== "boolean") - return "has_replication_filters: boolean expected"; - if (message.ssl_allowed != null && message.hasOwnProperty("ssl_allowed")) - if (typeof message.ssl_allowed !== "boolean") - return "ssl_allowed: boolean expected"; - if (message.replication_lag_unknown != null && message.hasOwnProperty("replication_lag_unknown")) - if (typeof message.replication_lag_unknown !== "boolean") - return "replication_lag_unknown: boolean expected"; + if (message.binlog_players_count != null && message.hasOwnProperty("binlog_players_count")) + if (!$util.isInteger(message.binlog_players_count)) + return "binlog_players_count: integer expected"; + if (message.filtered_replication_lag_seconds != null && message.hasOwnProperty("filtered_replication_lag_seconds")) + if (!$util.isInteger(message.filtered_replication_lag_seconds) && !(message.filtered_replication_lag_seconds && $util.isInteger(message.filtered_replication_lag_seconds.low) && $util.isInteger(message.filtered_replication_lag_seconds.high))) + return "filtered_replication_lag_seconds: integer|Long expected"; + if (message.cpu_usage != null && message.hasOwnProperty("cpu_usage")) + if (typeof message.cpu_usage !== "number") + return "cpu_usage: number expected"; + if (message.qps != null && message.hasOwnProperty("qps")) + if (typeof message.qps !== "number") + return "qps: number expected"; + if (message.table_schema_changed != null && message.hasOwnProperty("table_schema_changed")) { + if (!Array.isArray(message.table_schema_changed)) + return "table_schema_changed: array expected"; + for (let i = 0; i < message.table_schema_changed.length; ++i) + if (!$util.isString(message.table_schema_changed[i])) + return "table_schema_changed: string[] expected"; + } + if (message.view_schema_changed != null && message.hasOwnProperty("view_schema_changed")) { + if (!Array.isArray(message.view_schema_changed)) + return "view_schema_changed: array expected"; + for (let i = 0; i < message.view_schema_changed.length; ++i) + if (!$util.isString(message.view_schema_changed[i])) + return "view_schema_changed: string[] expected"; + } return null; }; /** - * Creates a Status message from a plain object. Also converts values to their respective internal types. + * Creates a RealtimeStats message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static * @param {Object.} object Plain object - * @returns {replicationdata.Status} Status + * @returns {query.RealtimeStats} RealtimeStats */ - Status.fromObject = function fromObject(object) { - if (object instanceof $root.replicationdata.Status) + RealtimeStats.fromObject = function fromObject(object) { + if (object instanceof $root.query.RealtimeStats) return object; - let message = new $root.replicationdata.Status(); - if (object.position != null) - message.position = String(object.position); + let message = new $root.query.RealtimeStats(); + if (object.health_error != null) + message.health_error = String(object.health_error); if (object.replication_lag_seconds != null) message.replication_lag_seconds = object.replication_lag_seconds >>> 0; - if (object.source_host != null) - message.source_host = String(object.source_host); - if (object.source_port != null) - message.source_port = object.source_port | 0; - if (object.connect_retry != null) - message.connect_retry = object.connect_retry | 0; - if (object.relay_log_position != null) - message.relay_log_position = String(object.relay_log_position); - if (object.file_position != null) - message.file_position = String(object.file_position); - if (object.relay_log_source_binlog_equivalent_position != null) - message.relay_log_source_binlog_equivalent_position = String(object.relay_log_source_binlog_equivalent_position); - if (object.source_server_id != null) - message.source_server_id = object.source_server_id >>> 0; - if (object.source_uuid != null) - message.source_uuid = String(object.source_uuid); - if (object.io_state != null) - message.io_state = object.io_state | 0; - if (object.last_io_error != null) - message.last_io_error = String(object.last_io_error); - if (object.sql_state != null) - message.sql_state = object.sql_state | 0; - if (object.last_sql_error != null) - message.last_sql_error = String(object.last_sql_error); - if (object.relay_log_file_position != null) - message.relay_log_file_position = String(object.relay_log_file_position); - if (object.source_user != null) - message.source_user = String(object.source_user); - if (object.sql_delay != null) - message.sql_delay = object.sql_delay >>> 0; - if (object.auto_position != null) - message.auto_position = Boolean(object.auto_position); - if (object.using_gtid != null) - message.using_gtid = Boolean(object.using_gtid); - if (object.has_replication_filters != null) - message.has_replication_filters = Boolean(object.has_replication_filters); - if (object.ssl_allowed != null) - message.ssl_allowed = Boolean(object.ssl_allowed); - if (object.replication_lag_unknown != null) - message.replication_lag_unknown = Boolean(object.replication_lag_unknown); + if (object.binlog_players_count != null) + message.binlog_players_count = object.binlog_players_count | 0; + if (object.filtered_replication_lag_seconds != null) + if ($util.Long) + (message.filtered_replication_lag_seconds = $util.Long.fromValue(object.filtered_replication_lag_seconds)).unsigned = false; + else if (typeof object.filtered_replication_lag_seconds === "string") + message.filtered_replication_lag_seconds = parseInt(object.filtered_replication_lag_seconds, 10); + else if (typeof object.filtered_replication_lag_seconds === "number") + message.filtered_replication_lag_seconds = object.filtered_replication_lag_seconds; + else if (typeof object.filtered_replication_lag_seconds === "object") + message.filtered_replication_lag_seconds = new $util.LongBits(object.filtered_replication_lag_seconds.low >>> 0, object.filtered_replication_lag_seconds.high >>> 0).toNumber(); + if (object.cpu_usage != null) + message.cpu_usage = Number(object.cpu_usage); + if (object.qps != null) + message.qps = Number(object.qps); + if (object.table_schema_changed) { + if (!Array.isArray(object.table_schema_changed)) + throw TypeError(".query.RealtimeStats.table_schema_changed: array expected"); + message.table_schema_changed = []; + for (let i = 0; i < object.table_schema_changed.length; ++i) + message.table_schema_changed[i] = String(object.table_schema_changed[i]); + } + if (object.view_schema_changed) { + if (!Array.isArray(object.view_schema_changed)) + throw TypeError(".query.RealtimeStats.view_schema_changed: array expected"); + message.view_schema_changed = []; + for (let i = 0; i < object.view_schema_changed.length; ++i) + message.view_schema_changed[i] = String(object.view_schema_changed[i]); + } return message; }; /** - * Creates a plain object from a Status message. Also converts values to other types if specified. + * Creates a plain object from a RealtimeStats message. Also converts values to other types if specified. * @function toObject - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static - * @param {replicationdata.Status} message Status + * @param {query.RealtimeStats} message RealtimeStats * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Status.toObject = function toObject(message, options) { + RealtimeStats.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.table_schema_changed = []; + object.view_schema_changed = []; + } if (options.defaults) { - object.position = ""; + object.health_error = ""; object.replication_lag_seconds = 0; - object.source_host = ""; - object.source_port = 0; - object.connect_retry = 0; - object.relay_log_position = ""; - object.file_position = ""; - object.relay_log_source_binlog_equivalent_position = ""; - object.source_server_id = 0; - object.source_uuid = ""; - object.io_state = 0; - object.last_io_error = ""; - object.sql_state = 0; - object.last_sql_error = ""; - object.relay_log_file_position = ""; - object.source_user = ""; - object.sql_delay = 0; - object.auto_position = false; - object.using_gtid = false; - object.has_replication_filters = false; - object.ssl_allowed = false; - object.replication_lag_unknown = false; + object.binlog_players_count = 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.filtered_replication_lag_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.filtered_replication_lag_seconds = options.longs === String ? "0" : 0; + object.cpu_usage = 0; + object.qps = 0; } - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; + if (message.health_error != null && message.hasOwnProperty("health_error")) + object.health_error = message.health_error; if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) object.replication_lag_seconds = message.replication_lag_seconds; - if (message.source_host != null && message.hasOwnProperty("source_host")) - object.source_host = message.source_host; - if (message.source_port != null && message.hasOwnProperty("source_port")) - object.source_port = message.source_port; - if (message.connect_retry != null && message.hasOwnProperty("connect_retry")) - object.connect_retry = message.connect_retry; - if (message.relay_log_position != null && message.hasOwnProperty("relay_log_position")) - object.relay_log_position = message.relay_log_position; - if (message.file_position != null && message.hasOwnProperty("file_position")) - object.file_position = message.file_position; - if (message.relay_log_source_binlog_equivalent_position != null && message.hasOwnProperty("relay_log_source_binlog_equivalent_position")) - object.relay_log_source_binlog_equivalent_position = message.relay_log_source_binlog_equivalent_position; - if (message.source_server_id != null && message.hasOwnProperty("source_server_id")) - object.source_server_id = message.source_server_id; - if (message.source_uuid != null && message.hasOwnProperty("source_uuid")) - object.source_uuid = message.source_uuid; - if (message.io_state != null && message.hasOwnProperty("io_state")) - object.io_state = message.io_state; - if (message.last_io_error != null && message.hasOwnProperty("last_io_error")) - object.last_io_error = message.last_io_error; - if (message.sql_state != null && message.hasOwnProperty("sql_state")) - object.sql_state = message.sql_state; - if (message.last_sql_error != null && message.hasOwnProperty("last_sql_error")) - object.last_sql_error = message.last_sql_error; - if (message.relay_log_file_position != null && message.hasOwnProperty("relay_log_file_position")) - object.relay_log_file_position = message.relay_log_file_position; - if (message.source_user != null && message.hasOwnProperty("source_user")) - object.source_user = message.source_user; - if (message.sql_delay != null && message.hasOwnProperty("sql_delay")) - object.sql_delay = message.sql_delay; - if (message.auto_position != null && message.hasOwnProperty("auto_position")) - object.auto_position = message.auto_position; - if (message.using_gtid != null && message.hasOwnProperty("using_gtid")) - object.using_gtid = message.using_gtid; - if (message.has_replication_filters != null && message.hasOwnProperty("has_replication_filters")) - object.has_replication_filters = message.has_replication_filters; - if (message.ssl_allowed != null && message.hasOwnProperty("ssl_allowed")) - object.ssl_allowed = message.ssl_allowed; - if (message.replication_lag_unknown != null && message.hasOwnProperty("replication_lag_unknown")) - object.replication_lag_unknown = message.replication_lag_unknown; + if (message.binlog_players_count != null && message.hasOwnProperty("binlog_players_count")) + object.binlog_players_count = message.binlog_players_count; + if (message.filtered_replication_lag_seconds != null && message.hasOwnProperty("filtered_replication_lag_seconds")) + if (typeof message.filtered_replication_lag_seconds === "number") + object.filtered_replication_lag_seconds = options.longs === String ? String(message.filtered_replication_lag_seconds) : message.filtered_replication_lag_seconds; + else + object.filtered_replication_lag_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.filtered_replication_lag_seconds) : options.longs === Number ? new $util.LongBits(message.filtered_replication_lag_seconds.low >>> 0, message.filtered_replication_lag_seconds.high >>> 0).toNumber() : message.filtered_replication_lag_seconds; + if (message.cpu_usage != null && message.hasOwnProperty("cpu_usage")) + object.cpu_usage = options.json && !isFinite(message.cpu_usage) ? String(message.cpu_usage) : message.cpu_usage; + if (message.qps != null && message.hasOwnProperty("qps")) + object.qps = options.json && !isFinite(message.qps) ? String(message.qps) : message.qps; + if (message.table_schema_changed && message.table_schema_changed.length) { + object.table_schema_changed = []; + for (let j = 0; j < message.table_schema_changed.length; ++j) + object.table_schema_changed[j] = message.table_schema_changed[j]; + } + if (message.view_schema_changed && message.view_schema_changed.length) { + object.view_schema_changed = []; + for (let j = 0; j < message.view_schema_changed.length; ++j) + object.view_schema_changed[j] = message.view_schema_changed[j]; + } return object; }; /** - * Converts this Status to JSON. + * Converts this RealtimeStats to JSON. * @function toJSON - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @instance * @returns {Object.} JSON object */ - Status.prototype.toJSON = function toJSON() { + RealtimeStats.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Status + * Gets the default type url for RealtimeStats * @function getTypeUrl - * @memberof replicationdata.Status + * @memberof query.RealtimeStats * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Status.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RealtimeStats.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/replicationdata.Status"; + return typeUrlPrefix + "/query.RealtimeStats"; }; - return Status; + return RealtimeStats; })(); - replicationdata.StopReplicationStatus = (function() { + query.AggregateStats = (function() { /** - * Properties of a StopReplicationStatus. - * @memberof replicationdata - * @interface IStopReplicationStatus - * @property {replicationdata.IStatus|null} [before] StopReplicationStatus before - * @property {replicationdata.IStatus|null} [after] StopReplicationStatus after + * Properties of an AggregateStats. + * @memberof query + * @interface IAggregateStats + * @property {number|null} [healthy_tablet_count] AggregateStats healthy_tablet_count + * @property {number|null} [unhealthy_tablet_count] AggregateStats unhealthy_tablet_count + * @property {number|null} [replication_lag_seconds_min] AggregateStats replication_lag_seconds_min + * @property {number|null} [replication_lag_seconds_max] AggregateStats replication_lag_seconds_max */ /** - * Constructs a new StopReplicationStatus. - * @memberof replicationdata - * @classdesc Represents a StopReplicationStatus. - * @implements IStopReplicationStatus + * Constructs a new AggregateStats. + * @memberof query + * @classdesc Represents an AggregateStats. + * @implements IAggregateStats * @constructor - * @param {replicationdata.IStopReplicationStatus=} [properties] Properties to set + * @param {query.IAggregateStats=} [properties] Properties to set */ - function StopReplicationStatus(properties) { + function AggregateStats(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -93865,89 +94550,117 @@ export const replicationdata = $root.replicationdata = (() => { } /** - * StopReplicationStatus before. - * @member {replicationdata.IStatus|null|undefined} before - * @memberof replicationdata.StopReplicationStatus + * AggregateStats healthy_tablet_count. + * @member {number} healthy_tablet_count + * @memberof query.AggregateStats * @instance */ - StopReplicationStatus.prototype.before = null; + AggregateStats.prototype.healthy_tablet_count = 0; /** - * StopReplicationStatus after. - * @member {replicationdata.IStatus|null|undefined} after - * @memberof replicationdata.StopReplicationStatus + * AggregateStats unhealthy_tablet_count. + * @member {number} unhealthy_tablet_count + * @memberof query.AggregateStats * @instance */ - StopReplicationStatus.prototype.after = null; + AggregateStats.prototype.unhealthy_tablet_count = 0; /** - * Creates a new StopReplicationStatus instance using the specified properties. + * AggregateStats replication_lag_seconds_min. + * @member {number} replication_lag_seconds_min + * @memberof query.AggregateStats + * @instance + */ + AggregateStats.prototype.replication_lag_seconds_min = 0; + + /** + * AggregateStats replication_lag_seconds_max. + * @member {number} replication_lag_seconds_max + * @memberof query.AggregateStats + * @instance + */ + AggregateStats.prototype.replication_lag_seconds_max = 0; + + /** + * Creates a new AggregateStats instance using the specified properties. * @function create - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static - * @param {replicationdata.IStopReplicationStatus=} [properties] Properties to set - * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus instance + * @param {query.IAggregateStats=} [properties] Properties to set + * @returns {query.AggregateStats} AggregateStats instance */ - StopReplicationStatus.create = function create(properties) { - return new StopReplicationStatus(properties); + AggregateStats.create = function create(properties) { + return new AggregateStats(properties); }; /** - * Encodes the specified StopReplicationStatus message. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. + * Encodes the specified AggregateStats message. Does not implicitly {@link query.AggregateStats.verify|verify} messages. * @function encode - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static - * @param {replicationdata.IStopReplicationStatus} message StopReplicationStatus message or plain object to encode + * @param {query.IAggregateStats} message AggregateStats message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationStatus.encode = function encode(message, writer) { + AggregateStats.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.before != null && Object.hasOwnProperty.call(message, "before")) - $root.replicationdata.Status.encode(message.before, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.after != null && Object.hasOwnProperty.call(message, "after")) - $root.replicationdata.Status.encode(message.after, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.healthy_tablet_count != null && Object.hasOwnProperty.call(message, "healthy_tablet_count")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.healthy_tablet_count); + if (message.unhealthy_tablet_count != null && Object.hasOwnProperty.call(message, "unhealthy_tablet_count")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.unhealthy_tablet_count); + if (message.replication_lag_seconds_min != null && Object.hasOwnProperty.call(message, "replication_lag_seconds_min")) + writer.uint32(/* id 3, wireType 0 =*/24).uint32(message.replication_lag_seconds_min); + if (message.replication_lag_seconds_max != null && Object.hasOwnProperty.call(message, "replication_lag_seconds_max")) + writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.replication_lag_seconds_max); return writer; }; /** - * Encodes the specified StopReplicationStatus message, length delimited. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. + * Encodes the specified AggregateStats message, length delimited. Does not implicitly {@link query.AggregateStats.verify|verify} messages. * @function encodeDelimited - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static - * @param {replicationdata.IStopReplicationStatus} message StopReplicationStatus message or plain object to encode + * @param {query.IAggregateStats} message AggregateStats message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationStatus.encodeDelimited = function encodeDelimited(message, writer) { + AggregateStats.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StopReplicationStatus message from the specified reader or buffer. + * Decodes an AggregateStats message from the specified reader or buffer. * @function decode - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus + * @returns {query.AggregateStats} AggregateStats * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationStatus.decode = function decode(reader, length) { + AggregateStats.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.replicationdata.StopReplicationStatus(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.AggregateStats(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.before = $root.replicationdata.Status.decode(reader, reader.uint32()); + message.healthy_tablet_count = reader.int32(); break; } case 2: { - message.after = $root.replicationdata.Status.decode(reader, reader.uint32()); + message.unhealthy_tablet_count = reader.int32(); + break; + } + case 3: { + message.replication_lag_seconds_min = reader.uint32(); + break; + } + case 4: { + message.replication_lag_seconds_max = reader.uint32(); break; } default: @@ -93959,156 +94672,151 @@ export const replicationdata = $root.replicationdata = (() => { }; /** - * Decodes a StopReplicationStatus message from the specified reader or buffer, length delimited. + * Decodes an AggregateStats message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus + * @returns {query.AggregateStats} AggregateStats * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationStatus.decodeDelimited = function decodeDelimited(reader) { + AggregateStats.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StopReplicationStatus message. + * Verifies an AggregateStats message. * @function verify - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StopReplicationStatus.verify = function verify(message) { + AggregateStats.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.before != null && message.hasOwnProperty("before")) { - let error = $root.replicationdata.Status.verify(message.before); - if (error) - return "before." + error; - } - if (message.after != null && message.hasOwnProperty("after")) { - let error = $root.replicationdata.Status.verify(message.after); - if (error) - return "after." + error; - } + if (message.healthy_tablet_count != null && message.hasOwnProperty("healthy_tablet_count")) + if (!$util.isInteger(message.healthy_tablet_count)) + return "healthy_tablet_count: integer expected"; + if (message.unhealthy_tablet_count != null && message.hasOwnProperty("unhealthy_tablet_count")) + if (!$util.isInteger(message.unhealthy_tablet_count)) + return "unhealthy_tablet_count: integer expected"; + if (message.replication_lag_seconds_min != null && message.hasOwnProperty("replication_lag_seconds_min")) + if (!$util.isInteger(message.replication_lag_seconds_min)) + return "replication_lag_seconds_min: integer expected"; + if (message.replication_lag_seconds_max != null && message.hasOwnProperty("replication_lag_seconds_max")) + if (!$util.isInteger(message.replication_lag_seconds_max)) + return "replication_lag_seconds_max: integer expected"; return null; }; /** - * Creates a StopReplicationStatus message from a plain object. Also converts values to their respective internal types. + * Creates an AggregateStats message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static * @param {Object.} object Plain object - * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus + * @returns {query.AggregateStats} AggregateStats */ - StopReplicationStatus.fromObject = function fromObject(object) { - if (object instanceof $root.replicationdata.StopReplicationStatus) + AggregateStats.fromObject = function fromObject(object) { + if (object instanceof $root.query.AggregateStats) return object; - let message = new $root.replicationdata.StopReplicationStatus(); - if (object.before != null) { - if (typeof object.before !== "object") - throw TypeError(".replicationdata.StopReplicationStatus.before: object expected"); - message.before = $root.replicationdata.Status.fromObject(object.before); - } - if (object.after != null) { - if (typeof object.after !== "object") - throw TypeError(".replicationdata.StopReplicationStatus.after: object expected"); - message.after = $root.replicationdata.Status.fromObject(object.after); - } + let message = new $root.query.AggregateStats(); + if (object.healthy_tablet_count != null) + message.healthy_tablet_count = object.healthy_tablet_count | 0; + if (object.unhealthy_tablet_count != null) + message.unhealthy_tablet_count = object.unhealthy_tablet_count | 0; + if (object.replication_lag_seconds_min != null) + message.replication_lag_seconds_min = object.replication_lag_seconds_min >>> 0; + if (object.replication_lag_seconds_max != null) + message.replication_lag_seconds_max = object.replication_lag_seconds_max >>> 0; return message; }; /** - * Creates a plain object from a StopReplicationStatus message. Also converts values to other types if specified. + * Creates a plain object from an AggregateStats message. Also converts values to other types if specified. * @function toObject - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static - * @param {replicationdata.StopReplicationStatus} message StopReplicationStatus + * @param {query.AggregateStats} message AggregateStats * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StopReplicationStatus.toObject = function toObject(message, options) { + AggregateStats.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.before = null; - object.after = null; + object.healthy_tablet_count = 0; + object.unhealthy_tablet_count = 0; + object.replication_lag_seconds_min = 0; + object.replication_lag_seconds_max = 0; } - if (message.before != null && message.hasOwnProperty("before")) - object.before = $root.replicationdata.Status.toObject(message.before, options); - if (message.after != null && message.hasOwnProperty("after")) - object.after = $root.replicationdata.Status.toObject(message.after, options); + if (message.healthy_tablet_count != null && message.hasOwnProperty("healthy_tablet_count")) + object.healthy_tablet_count = message.healthy_tablet_count; + if (message.unhealthy_tablet_count != null && message.hasOwnProperty("unhealthy_tablet_count")) + object.unhealthy_tablet_count = message.unhealthy_tablet_count; + if (message.replication_lag_seconds_min != null && message.hasOwnProperty("replication_lag_seconds_min")) + object.replication_lag_seconds_min = message.replication_lag_seconds_min; + if (message.replication_lag_seconds_max != null && message.hasOwnProperty("replication_lag_seconds_max")) + object.replication_lag_seconds_max = message.replication_lag_seconds_max; return object; }; /** - * Converts this StopReplicationStatus to JSON. + * Converts this AggregateStats to JSON. * @function toJSON - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @instance * @returns {Object.} JSON object */ - StopReplicationStatus.prototype.toJSON = function toJSON() { + AggregateStats.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StopReplicationStatus + * Gets the default type url for AggregateStats * @function getTypeUrl - * @memberof replicationdata.StopReplicationStatus + * @memberof query.AggregateStats * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StopReplicationStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + AggregateStats.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/replicationdata.StopReplicationStatus"; + return typeUrlPrefix + "/query.AggregateStats"; }; - return StopReplicationStatus; - })(); - - /** - * StopReplicationMode enum. - * @name replicationdata.StopReplicationMode - * @enum {number} - * @property {number} IOANDSQLTHREAD=0 IOANDSQLTHREAD value - * @property {number} IOTHREADONLY=1 IOTHREADONLY value - */ - replicationdata.StopReplicationMode = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "IOANDSQLTHREAD"] = 0; - values[valuesById[1] = "IOTHREADONLY"] = 1; - return values; + return AggregateStats; })(); - replicationdata.PrimaryStatus = (function() { + query.StreamHealthResponse = (function() { /** - * Properties of a PrimaryStatus. - * @memberof replicationdata - * @interface IPrimaryStatus - * @property {string|null} [position] PrimaryStatus position - * @property {string|null} [file_position] PrimaryStatus file_position + * Properties of a StreamHealthResponse. + * @memberof query + * @interface IStreamHealthResponse + * @property {query.ITarget|null} [target] StreamHealthResponse target + * @property {boolean|null} [serving] StreamHealthResponse serving + * @property {number|Long|null} [tablet_externally_reparented_timestamp] StreamHealthResponse tablet_externally_reparented_timestamp + * @property {query.IRealtimeStats|null} [realtime_stats] StreamHealthResponse realtime_stats + * @property {topodata.ITabletAlias|null} [tablet_alias] StreamHealthResponse tablet_alias */ /** - * Constructs a new PrimaryStatus. - * @memberof replicationdata - * @classdesc Represents a PrimaryStatus. - * @implements IPrimaryStatus + * Constructs a new StreamHealthResponse. + * @memberof query + * @classdesc Represents a StreamHealthResponse. + * @implements IStreamHealthResponse * @constructor - * @param {replicationdata.IPrimaryStatus=} [properties] Properties to set + * @param {query.IStreamHealthResponse=} [properties] Properties to set */ - function PrimaryStatus(properties) { + function StreamHealthResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -94116,89 +94824,131 @@ export const replicationdata = $root.replicationdata = (() => { } /** - * PrimaryStatus position. - * @member {string} position - * @memberof replicationdata.PrimaryStatus + * StreamHealthResponse target. + * @member {query.ITarget|null|undefined} target + * @memberof query.StreamHealthResponse * @instance */ - PrimaryStatus.prototype.position = ""; + StreamHealthResponse.prototype.target = null; /** - * PrimaryStatus file_position. - * @member {string} file_position - * @memberof replicationdata.PrimaryStatus + * StreamHealthResponse serving. + * @member {boolean} serving + * @memberof query.StreamHealthResponse * @instance */ - PrimaryStatus.prototype.file_position = ""; + StreamHealthResponse.prototype.serving = false; /** - * Creates a new PrimaryStatus instance using the specified properties. + * StreamHealthResponse tablet_externally_reparented_timestamp. + * @member {number|Long} tablet_externally_reparented_timestamp + * @memberof query.StreamHealthResponse + * @instance + */ + StreamHealthResponse.prototype.tablet_externally_reparented_timestamp = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * StreamHealthResponse realtime_stats. + * @member {query.IRealtimeStats|null|undefined} realtime_stats + * @memberof query.StreamHealthResponse + * @instance + */ + StreamHealthResponse.prototype.realtime_stats = null; + + /** + * StreamHealthResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof query.StreamHealthResponse + * @instance + */ + StreamHealthResponse.prototype.tablet_alias = null; + + /** + * Creates a new StreamHealthResponse instance using the specified properties. * @function create - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static - * @param {replicationdata.IPrimaryStatus=} [properties] Properties to set - * @returns {replicationdata.PrimaryStatus} PrimaryStatus instance + * @param {query.IStreamHealthResponse=} [properties] Properties to set + * @returns {query.StreamHealthResponse} StreamHealthResponse instance */ - PrimaryStatus.create = function create(properties) { - return new PrimaryStatus(properties); + StreamHealthResponse.create = function create(properties) { + return new StreamHealthResponse(properties); }; /** - * Encodes the specified PrimaryStatus message. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. + * Encodes the specified StreamHealthResponse message. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. * @function encode - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static - * @param {replicationdata.IPrimaryStatus} message PrimaryStatus message or plain object to encode + * @param {query.IStreamHealthResponse} message StreamHealthResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrimaryStatus.encode = function encode(message, writer) { + StreamHealthResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); - if (message.file_position != null && Object.hasOwnProperty.call(message, "file_position")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.file_position); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.serving != null && Object.hasOwnProperty.call(message, "serving")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.serving); + if (message.tablet_externally_reparented_timestamp != null && Object.hasOwnProperty.call(message, "tablet_externally_reparented_timestamp")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.tablet_externally_reparented_timestamp); + if (message.realtime_stats != null && Object.hasOwnProperty.call(message, "realtime_stats")) + $root.query.RealtimeStats.encode(message.realtime_stats, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified PrimaryStatus message, length delimited. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. + * Encodes the specified StreamHealthResponse message, length delimited. Does not implicitly {@link query.StreamHealthResponse.verify|verify} messages. * @function encodeDelimited - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static - * @param {replicationdata.IPrimaryStatus} message PrimaryStatus message or plain object to encode + * @param {query.IStreamHealthResponse} message StreamHealthResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PrimaryStatus.encodeDelimited = function encodeDelimited(message, writer) { + StreamHealthResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PrimaryStatus message from the specified reader or buffer. + * Decodes a StreamHealthResponse message from the specified reader or buffer. * @function decode - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {replicationdata.PrimaryStatus} PrimaryStatus + * @returns {query.StreamHealthResponse} StreamHealthResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrimaryStatus.decode = function decode(reader, length) { + StreamHealthResponse.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.replicationdata.PrimaryStatus(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.StreamHealthResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.position = reader.string(); + message.target = $root.query.Target.decode(reader, reader.uint32()); break; } case 2: { - message.file_position = reader.string(); + message.serving = reader.bool(); + break; + } + case 3: { + message.tablet_externally_reparented_timestamp = reader.int64(); + break; + } + case 4: { + message.realtime_stats = $root.query.RealtimeStats.decode(reader, reader.uint32()); + break; + } + case 5: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -94210,151 +94960,206 @@ export const replicationdata = $root.replicationdata = (() => { }; /** - * Decodes a PrimaryStatus message from the specified reader or buffer, length delimited. + * Decodes a StreamHealthResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {replicationdata.PrimaryStatus} PrimaryStatus + * @returns {query.StreamHealthResponse} StreamHealthResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PrimaryStatus.decodeDelimited = function decodeDelimited(reader) { + StreamHealthResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PrimaryStatus message. + * Verifies a StreamHealthResponse message. * @function verify - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PrimaryStatus.verify = function verify(message) { + StreamHealthResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; - if (message.file_position != null && message.hasOwnProperty("file_position")) - if (!$util.isString(message.file_position)) - return "file_position: string expected"; + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.serving != null && message.hasOwnProperty("serving")) + if (typeof message.serving !== "boolean") + return "serving: boolean expected"; + if (message.tablet_externally_reparented_timestamp != null && message.hasOwnProperty("tablet_externally_reparented_timestamp")) + if (!$util.isInteger(message.tablet_externally_reparented_timestamp) && !(message.tablet_externally_reparented_timestamp && $util.isInteger(message.tablet_externally_reparented_timestamp.low) && $util.isInteger(message.tablet_externally_reparented_timestamp.high))) + return "tablet_externally_reparented_timestamp: integer|Long expected"; + if (message.realtime_stats != null && message.hasOwnProperty("realtime_stats")) { + let error = $root.query.RealtimeStats.verify(message.realtime_stats); + if (error) + return "realtime_stats." + error; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a PrimaryStatus message from a plain object. Also converts values to their respective internal types. + * Creates a StreamHealthResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static * @param {Object.} object Plain object - * @returns {replicationdata.PrimaryStatus} PrimaryStatus + * @returns {query.StreamHealthResponse} StreamHealthResponse */ - PrimaryStatus.fromObject = function fromObject(object) { - if (object instanceof $root.replicationdata.PrimaryStatus) + StreamHealthResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.StreamHealthResponse) return object; - let message = new $root.replicationdata.PrimaryStatus(); - if (object.position != null) - message.position = String(object.position); - if (object.file_position != null) - message.file_position = String(object.file_position); + let message = new $root.query.StreamHealthResponse(); + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.StreamHealthResponse.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + if (object.serving != null) + message.serving = Boolean(object.serving); + if (object.tablet_externally_reparented_timestamp != null) + if ($util.Long) + (message.tablet_externally_reparented_timestamp = $util.Long.fromValue(object.tablet_externally_reparented_timestamp)).unsigned = false; + else if (typeof object.tablet_externally_reparented_timestamp === "string") + message.tablet_externally_reparented_timestamp = parseInt(object.tablet_externally_reparented_timestamp, 10); + else if (typeof object.tablet_externally_reparented_timestamp === "number") + message.tablet_externally_reparented_timestamp = object.tablet_externally_reparented_timestamp; + else if (typeof object.tablet_externally_reparented_timestamp === "object") + message.tablet_externally_reparented_timestamp = new $util.LongBits(object.tablet_externally_reparented_timestamp.low >>> 0, object.tablet_externally_reparented_timestamp.high >>> 0).toNumber(); + if (object.realtime_stats != null) { + if (typeof object.realtime_stats !== "object") + throw TypeError(".query.StreamHealthResponse.realtime_stats: object expected"); + message.realtime_stats = $root.query.RealtimeStats.fromObject(object.realtime_stats); + } + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".query.StreamHealthResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a PrimaryStatus message. Also converts values to other types if specified. + * Creates a plain object from a StreamHealthResponse message. Also converts values to other types if specified. * @function toObject - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static - * @param {replicationdata.PrimaryStatus} message PrimaryStatus + * @param {query.StreamHealthResponse} message StreamHealthResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PrimaryStatus.toObject = function toObject(message, options) { + StreamHealthResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.position = ""; - object.file_position = ""; + object.target = null; + object.serving = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.tablet_externally_reparented_timestamp = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.tablet_externally_reparented_timestamp = options.longs === String ? "0" : 0; + object.realtime_stats = null; + object.tablet_alias = null; } - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; - if (message.file_position != null && message.hasOwnProperty("file_position")) - object.file_position = message.file_position; + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.serving != null && message.hasOwnProperty("serving")) + object.serving = message.serving; + if (message.tablet_externally_reparented_timestamp != null && message.hasOwnProperty("tablet_externally_reparented_timestamp")) + if (typeof message.tablet_externally_reparented_timestamp === "number") + object.tablet_externally_reparented_timestamp = options.longs === String ? String(message.tablet_externally_reparented_timestamp) : message.tablet_externally_reparented_timestamp; + else + object.tablet_externally_reparented_timestamp = options.longs === String ? $util.Long.prototype.toString.call(message.tablet_externally_reparented_timestamp) : options.longs === Number ? new $util.LongBits(message.tablet_externally_reparented_timestamp.low >>> 0, message.tablet_externally_reparented_timestamp.high >>> 0).toNumber() : message.tablet_externally_reparented_timestamp; + if (message.realtime_stats != null && message.hasOwnProperty("realtime_stats")) + object.realtime_stats = $root.query.RealtimeStats.toObject(message.realtime_stats, options); + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this PrimaryStatus to JSON. + * Converts this StreamHealthResponse to JSON. * @function toJSON - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @instance * @returns {Object.} JSON object */ - PrimaryStatus.prototype.toJSON = function toJSON() { + StreamHealthResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PrimaryStatus + * Gets the default type url for StreamHealthResponse * @function getTypeUrl - * @memberof replicationdata.PrimaryStatus + * @memberof query.StreamHealthResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PrimaryStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StreamHealthResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/replicationdata.PrimaryStatus"; + return typeUrlPrefix + "/query.StreamHealthResponse"; }; - return PrimaryStatus; + return StreamHealthResponse; })(); - replicationdata.FullStatus = (function() { + /** + * TransactionState enum. + * @name query.TransactionState + * @enum {number} + * @property {number} UNKNOWN=0 UNKNOWN value + * @property {number} PREPARE=1 PREPARE value + * @property {number} COMMIT=2 COMMIT value + * @property {number} ROLLBACK=3 ROLLBACK value + */ + query.TransactionState = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "UNKNOWN"] = 0; + values[valuesById[1] = "PREPARE"] = 1; + values[valuesById[2] = "COMMIT"] = 2; + values[valuesById[3] = "ROLLBACK"] = 3; + return values; + })(); + + query.TransactionMetadata = (function() { /** - * Properties of a FullStatus. - * @memberof replicationdata - * @interface IFullStatus - * @property {number|null} [server_id] FullStatus server_id - * @property {string|null} [server_uuid] FullStatus server_uuid - * @property {replicationdata.IStatus|null} [replication_status] FullStatus replication_status - * @property {replicationdata.IPrimaryStatus|null} [primary_status] FullStatus primary_status - * @property {string|null} [gtid_purged] FullStatus gtid_purged - * @property {string|null} [version] FullStatus version - * @property {string|null} [version_comment] FullStatus version_comment - * @property {boolean|null} [read_only] FullStatus read_only - * @property {string|null} [gtid_mode] FullStatus gtid_mode - * @property {string|null} [binlog_format] FullStatus binlog_format - * @property {string|null} [binlog_row_image] FullStatus binlog_row_image - * @property {boolean|null} [log_bin_enabled] FullStatus log_bin_enabled - * @property {boolean|null} [log_replica_updates] FullStatus log_replica_updates - * @property {boolean|null} [semi_sync_primary_enabled] FullStatus semi_sync_primary_enabled - * @property {boolean|null} [semi_sync_replica_enabled] FullStatus semi_sync_replica_enabled - * @property {boolean|null} [semi_sync_primary_status] FullStatus semi_sync_primary_status - * @property {boolean|null} [semi_sync_replica_status] FullStatus semi_sync_replica_status - * @property {number|null} [semi_sync_primary_clients] FullStatus semi_sync_primary_clients - * @property {number|Long|null} [semi_sync_primary_timeout] FullStatus semi_sync_primary_timeout - * @property {number|null} [semi_sync_wait_for_replica_count] FullStatus semi_sync_wait_for_replica_count - * @property {boolean|null} [super_read_only] FullStatus super_read_only + * Properties of a TransactionMetadata. + * @memberof query + * @interface ITransactionMetadata + * @property {string|null} [dtid] TransactionMetadata dtid + * @property {query.TransactionState|null} [state] TransactionMetadata state + * @property {number|Long|null} [time_created] TransactionMetadata time_created + * @property {Array.|null} [participants] TransactionMetadata participants */ /** - * Constructs a new FullStatus. - * @memberof replicationdata - * @classdesc Represents a FullStatus. - * @implements IFullStatus + * Constructs a new TransactionMetadata. + * @memberof query + * @classdesc Represents a TransactionMetadata. + * @implements ITransactionMetadata * @constructor - * @param {replicationdata.IFullStatus=} [properties] Properties to set + * @param {query.ITransactionMetadata=} [properties] Properties to set */ - function FullStatus(properties) { + function TransactionMetadata(properties) { + this.participants = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -94362,355 +95167,120 @@ export const replicationdata = $root.replicationdata = (() => { } /** - * FullStatus server_id. - * @member {number} server_id - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.server_id = 0; - - /** - * FullStatus server_uuid. - * @member {string} server_uuid - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.server_uuid = ""; - - /** - * FullStatus replication_status. - * @member {replicationdata.IStatus|null|undefined} replication_status - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.replication_status = null; - - /** - * FullStatus primary_status. - * @member {replicationdata.IPrimaryStatus|null|undefined} primary_status - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.primary_status = null; - - /** - * FullStatus gtid_purged. - * @member {string} gtid_purged - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.gtid_purged = ""; - - /** - * FullStatus version. - * @member {string} version - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.version = ""; - - /** - * FullStatus version_comment. - * @member {string} version_comment - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.version_comment = ""; - - /** - * FullStatus read_only. - * @member {boolean} read_only - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.read_only = false; - - /** - * FullStatus gtid_mode. - * @member {string} gtid_mode - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.gtid_mode = ""; - - /** - * FullStatus binlog_format. - * @member {string} binlog_format - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.binlog_format = ""; - - /** - * FullStatus binlog_row_image. - * @member {string} binlog_row_image - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.binlog_row_image = ""; - - /** - * FullStatus log_bin_enabled. - * @member {boolean} log_bin_enabled - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.log_bin_enabled = false; - - /** - * FullStatus log_replica_updates. - * @member {boolean} log_replica_updates - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.log_replica_updates = false; - - /** - * FullStatus semi_sync_primary_enabled. - * @member {boolean} semi_sync_primary_enabled - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.semi_sync_primary_enabled = false; - - /** - * FullStatus semi_sync_replica_enabled. - * @member {boolean} semi_sync_replica_enabled - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.semi_sync_replica_enabled = false; - - /** - * FullStatus semi_sync_primary_status. - * @member {boolean} semi_sync_primary_status - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.semi_sync_primary_status = false; - - /** - * FullStatus semi_sync_replica_status. - * @member {boolean} semi_sync_replica_status - * @memberof replicationdata.FullStatus - * @instance - */ - FullStatus.prototype.semi_sync_replica_status = false; - - /** - * FullStatus semi_sync_primary_clients. - * @member {number} semi_sync_primary_clients - * @memberof replicationdata.FullStatus + * TransactionMetadata dtid. + * @member {string} dtid + * @memberof query.TransactionMetadata * @instance */ - FullStatus.prototype.semi_sync_primary_clients = 0; + TransactionMetadata.prototype.dtid = ""; /** - * FullStatus semi_sync_primary_timeout. - * @member {number|Long} semi_sync_primary_timeout - * @memberof replicationdata.FullStatus + * TransactionMetadata state. + * @member {query.TransactionState} state + * @memberof query.TransactionMetadata * @instance */ - FullStatus.prototype.semi_sync_primary_timeout = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + TransactionMetadata.prototype.state = 0; /** - * FullStatus semi_sync_wait_for_replica_count. - * @member {number} semi_sync_wait_for_replica_count - * @memberof replicationdata.FullStatus + * TransactionMetadata time_created. + * @member {number|Long} time_created + * @memberof query.TransactionMetadata * @instance */ - FullStatus.prototype.semi_sync_wait_for_replica_count = 0; + TransactionMetadata.prototype.time_created = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * FullStatus super_read_only. - * @member {boolean} super_read_only - * @memberof replicationdata.FullStatus + * TransactionMetadata participants. + * @member {Array.} participants + * @memberof query.TransactionMetadata * @instance */ - FullStatus.prototype.super_read_only = false; + TransactionMetadata.prototype.participants = $util.emptyArray; /** - * Creates a new FullStatus instance using the specified properties. + * Creates a new TransactionMetadata instance using the specified properties. * @function create - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static - * @param {replicationdata.IFullStatus=} [properties] Properties to set - * @returns {replicationdata.FullStatus} FullStatus instance + * @param {query.ITransactionMetadata=} [properties] Properties to set + * @returns {query.TransactionMetadata} TransactionMetadata instance */ - FullStatus.create = function create(properties) { - return new FullStatus(properties); + TransactionMetadata.create = function create(properties) { + return new TransactionMetadata(properties); }; /** - * Encodes the specified FullStatus message. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. + * Encodes the specified TransactionMetadata message. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. * @function encode - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static - * @param {replicationdata.IFullStatus} message FullStatus message or plain object to encode + * @param {query.ITransactionMetadata} message TransactionMetadata message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FullStatus.encode = function encode(message, writer) { + TransactionMetadata.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.server_id != null && Object.hasOwnProperty.call(message, "server_id")) - writer.uint32(/* id 1, wireType 0 =*/8).uint32(message.server_id); - if (message.server_uuid != null && Object.hasOwnProperty.call(message, "server_uuid")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.server_uuid); - if (message.replication_status != null && Object.hasOwnProperty.call(message, "replication_status")) - $root.replicationdata.Status.encode(message.replication_status, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.primary_status != null && Object.hasOwnProperty.call(message, "primary_status")) - $root.replicationdata.PrimaryStatus.encode(message.primary_status, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.gtid_purged != null && Object.hasOwnProperty.call(message, "gtid_purged")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.gtid_purged); - if (message.version != null && Object.hasOwnProperty.call(message, "version")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.version); - if (message.version_comment != null && Object.hasOwnProperty.call(message, "version_comment")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.version_comment); - if (message.read_only != null && Object.hasOwnProperty.call(message, "read_only")) - writer.uint32(/* id 8, wireType 0 =*/64).bool(message.read_only); - if (message.gtid_mode != null && Object.hasOwnProperty.call(message, "gtid_mode")) - writer.uint32(/* id 9, wireType 2 =*/74).string(message.gtid_mode); - if (message.binlog_format != null && Object.hasOwnProperty.call(message, "binlog_format")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.binlog_format); - if (message.binlog_row_image != null && Object.hasOwnProperty.call(message, "binlog_row_image")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.binlog_row_image); - if (message.log_bin_enabled != null && Object.hasOwnProperty.call(message, "log_bin_enabled")) - writer.uint32(/* id 12, wireType 0 =*/96).bool(message.log_bin_enabled); - if (message.log_replica_updates != null && Object.hasOwnProperty.call(message, "log_replica_updates")) - writer.uint32(/* id 13, wireType 0 =*/104).bool(message.log_replica_updates); - if (message.semi_sync_primary_enabled != null && Object.hasOwnProperty.call(message, "semi_sync_primary_enabled")) - writer.uint32(/* id 14, wireType 0 =*/112).bool(message.semi_sync_primary_enabled); - if (message.semi_sync_replica_enabled != null && Object.hasOwnProperty.call(message, "semi_sync_replica_enabled")) - writer.uint32(/* id 15, wireType 0 =*/120).bool(message.semi_sync_replica_enabled); - if (message.semi_sync_primary_status != null && Object.hasOwnProperty.call(message, "semi_sync_primary_status")) - writer.uint32(/* id 16, wireType 0 =*/128).bool(message.semi_sync_primary_status); - if (message.semi_sync_replica_status != null && Object.hasOwnProperty.call(message, "semi_sync_replica_status")) - writer.uint32(/* id 17, wireType 0 =*/136).bool(message.semi_sync_replica_status); - if (message.semi_sync_primary_clients != null && Object.hasOwnProperty.call(message, "semi_sync_primary_clients")) - writer.uint32(/* id 18, wireType 0 =*/144).uint32(message.semi_sync_primary_clients); - if (message.semi_sync_primary_timeout != null && Object.hasOwnProperty.call(message, "semi_sync_primary_timeout")) - writer.uint32(/* id 19, wireType 0 =*/152).uint64(message.semi_sync_primary_timeout); - if (message.semi_sync_wait_for_replica_count != null && Object.hasOwnProperty.call(message, "semi_sync_wait_for_replica_count")) - writer.uint32(/* id 20, wireType 0 =*/160).uint32(message.semi_sync_wait_for_replica_count); - if (message.super_read_only != null && Object.hasOwnProperty.call(message, "super_read_only")) - writer.uint32(/* id 21, wireType 0 =*/168).bool(message.super_read_only); + if (message.dtid != null && Object.hasOwnProperty.call(message, "dtid")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.dtid); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.state); + if (message.time_created != null && Object.hasOwnProperty.call(message, "time_created")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.time_created); + if (message.participants != null && message.participants.length) + for (let i = 0; i < message.participants.length; ++i) + $root.query.Target.encode(message.participants[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified FullStatus message, length delimited. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. + * Encodes the specified TransactionMetadata message, length delimited. Does not implicitly {@link query.TransactionMetadata.verify|verify} messages. * @function encodeDelimited - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static - * @param {replicationdata.IFullStatus} message FullStatus message or plain object to encode + * @param {query.ITransactionMetadata} message TransactionMetadata message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FullStatus.encodeDelimited = function encodeDelimited(message, writer) { + TransactionMetadata.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a FullStatus message from the specified reader or buffer. + * Decodes a TransactionMetadata message from the specified reader or buffer. * @function decode - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {replicationdata.FullStatus} FullStatus + * @returns {query.TransactionMetadata} TransactionMetadata * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FullStatus.decode = function decode(reader, length) { + TransactionMetadata.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.replicationdata.FullStatus(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.TransactionMetadata(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.server_id = reader.uint32(); + message.dtid = reader.string(); break; } case 2: { - message.server_uuid = reader.string(); + message.state = reader.int32(); break; } case 3: { - message.replication_status = $root.replicationdata.Status.decode(reader, reader.uint32()); + message.time_created = reader.int64(); break; } case 4: { - message.primary_status = $root.replicationdata.PrimaryStatus.decode(reader, reader.uint32()); - break; - } - case 5: { - message.gtid_purged = reader.string(); - break; - } - case 6: { - message.version = reader.string(); - break; - } - case 7: { - message.version_comment = reader.string(); - break; - } - case 8: { - message.read_only = reader.bool(); - break; - } - case 9: { - message.gtid_mode = reader.string(); - break; - } - case 10: { - message.binlog_format = reader.string(); - break; - } - case 11: { - message.binlog_row_image = reader.string(); - break; - } - case 12: { - message.log_bin_enabled = reader.bool(); - break; - } - case 13: { - message.log_replica_updates = reader.bool(); - break; - } - case 14: { - message.semi_sync_primary_enabled = reader.bool(); - break; - } - case 15: { - message.semi_sync_replica_enabled = reader.bool(); - break; - } - case 16: { - message.semi_sync_primary_status = reader.bool(); - break; - } - case 17: { - message.semi_sync_replica_status = reader.bool(); - break; - } - case 18: { - message.semi_sync_primary_clients = reader.uint32(); - break; - } - case 19: { - message.semi_sync_primary_timeout = reader.uint64(); - break; - } - case 20: { - message.semi_sync_wait_for_replica_count = reader.uint32(); - break; - } - case 21: { - message.super_read_only = reader.bool(); + if (!(message.participants && message.participants.length)) + message.participants = []; + message.participants.push($root.query.Target.decode(reader, reader.uint32())); break; } default: @@ -94722,320 +95292,227 @@ export const replicationdata = $root.replicationdata = (() => { }; /** - * Decodes a FullStatus message from the specified reader or buffer, length delimited. + * Decodes a TransactionMetadata message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {replicationdata.FullStatus} FullStatus + * @returns {query.TransactionMetadata} TransactionMetadata * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FullStatus.decodeDelimited = function decodeDelimited(reader) { + TransactionMetadata.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a FullStatus message. + * Verifies a TransactionMetadata message. * @function verify - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - FullStatus.verify = function verify(message) { + TransactionMetadata.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.server_id != null && message.hasOwnProperty("server_id")) - if (!$util.isInteger(message.server_id)) - return "server_id: integer expected"; - if (message.server_uuid != null && message.hasOwnProperty("server_uuid")) - if (!$util.isString(message.server_uuid)) - return "server_uuid: string expected"; - if (message.replication_status != null && message.hasOwnProperty("replication_status")) { - let error = $root.replicationdata.Status.verify(message.replication_status); - if (error) - return "replication_status." + error; - } - if (message.primary_status != null && message.hasOwnProperty("primary_status")) { - let error = $root.replicationdata.PrimaryStatus.verify(message.primary_status); - if (error) - return "primary_status." + error; + if (message.dtid != null && message.hasOwnProperty("dtid")) + if (!$util.isString(message.dtid)) + return "dtid: string expected"; + if (message.state != null && message.hasOwnProperty("state")) + switch (message.state) { + default: + return "state: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.time_created != null && message.hasOwnProperty("time_created")) + if (!$util.isInteger(message.time_created) && !(message.time_created && $util.isInteger(message.time_created.low) && $util.isInteger(message.time_created.high))) + return "time_created: integer|Long expected"; + if (message.participants != null && message.hasOwnProperty("participants")) { + if (!Array.isArray(message.participants)) + return "participants: array expected"; + for (let i = 0; i < message.participants.length; ++i) { + let error = $root.query.Target.verify(message.participants[i]); + if (error) + return "participants." + error; + } } - if (message.gtid_purged != null && message.hasOwnProperty("gtid_purged")) - if (!$util.isString(message.gtid_purged)) - return "gtid_purged: string expected"; - if (message.version != null && message.hasOwnProperty("version")) - if (!$util.isString(message.version)) - return "version: string expected"; - if (message.version_comment != null && message.hasOwnProperty("version_comment")) - if (!$util.isString(message.version_comment)) - return "version_comment: string expected"; - if (message.read_only != null && message.hasOwnProperty("read_only")) - if (typeof message.read_only !== "boolean") - return "read_only: boolean expected"; - if (message.gtid_mode != null && message.hasOwnProperty("gtid_mode")) - if (!$util.isString(message.gtid_mode)) - return "gtid_mode: string expected"; - if (message.binlog_format != null && message.hasOwnProperty("binlog_format")) - if (!$util.isString(message.binlog_format)) - return "binlog_format: string expected"; - if (message.binlog_row_image != null && message.hasOwnProperty("binlog_row_image")) - if (!$util.isString(message.binlog_row_image)) - return "binlog_row_image: string expected"; - if (message.log_bin_enabled != null && message.hasOwnProperty("log_bin_enabled")) - if (typeof message.log_bin_enabled !== "boolean") - return "log_bin_enabled: boolean expected"; - if (message.log_replica_updates != null && message.hasOwnProperty("log_replica_updates")) - if (typeof message.log_replica_updates !== "boolean") - return "log_replica_updates: boolean expected"; - if (message.semi_sync_primary_enabled != null && message.hasOwnProperty("semi_sync_primary_enabled")) - if (typeof message.semi_sync_primary_enabled !== "boolean") - return "semi_sync_primary_enabled: boolean expected"; - if (message.semi_sync_replica_enabled != null && message.hasOwnProperty("semi_sync_replica_enabled")) - if (typeof message.semi_sync_replica_enabled !== "boolean") - return "semi_sync_replica_enabled: boolean expected"; - if (message.semi_sync_primary_status != null && message.hasOwnProperty("semi_sync_primary_status")) - if (typeof message.semi_sync_primary_status !== "boolean") - return "semi_sync_primary_status: boolean expected"; - if (message.semi_sync_replica_status != null && message.hasOwnProperty("semi_sync_replica_status")) - if (typeof message.semi_sync_replica_status !== "boolean") - return "semi_sync_replica_status: boolean expected"; - if (message.semi_sync_primary_clients != null && message.hasOwnProperty("semi_sync_primary_clients")) - if (!$util.isInteger(message.semi_sync_primary_clients)) - return "semi_sync_primary_clients: integer expected"; - if (message.semi_sync_primary_timeout != null && message.hasOwnProperty("semi_sync_primary_timeout")) - if (!$util.isInteger(message.semi_sync_primary_timeout) && !(message.semi_sync_primary_timeout && $util.isInteger(message.semi_sync_primary_timeout.low) && $util.isInteger(message.semi_sync_primary_timeout.high))) - return "semi_sync_primary_timeout: integer|Long expected"; - if (message.semi_sync_wait_for_replica_count != null && message.hasOwnProperty("semi_sync_wait_for_replica_count")) - if (!$util.isInteger(message.semi_sync_wait_for_replica_count)) - return "semi_sync_wait_for_replica_count: integer expected"; - if (message.super_read_only != null && message.hasOwnProperty("super_read_only")) - if (typeof message.super_read_only !== "boolean") - return "super_read_only: boolean expected"; return null; }; /** - * Creates a FullStatus message from a plain object. Also converts values to their respective internal types. + * Creates a TransactionMetadata message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static * @param {Object.} object Plain object - * @returns {replicationdata.FullStatus} FullStatus + * @returns {query.TransactionMetadata} TransactionMetadata */ - FullStatus.fromObject = function fromObject(object) { - if (object instanceof $root.replicationdata.FullStatus) + TransactionMetadata.fromObject = function fromObject(object) { + if (object instanceof $root.query.TransactionMetadata) return object; - let message = new $root.replicationdata.FullStatus(); - if (object.server_id != null) - message.server_id = object.server_id >>> 0; - if (object.server_uuid != null) - message.server_uuid = String(object.server_uuid); - if (object.replication_status != null) { - if (typeof object.replication_status !== "object") - throw TypeError(".replicationdata.FullStatus.replication_status: object expected"); - message.replication_status = $root.replicationdata.Status.fromObject(object.replication_status); - } - if (object.primary_status != null) { - if (typeof object.primary_status !== "object") - throw TypeError(".replicationdata.FullStatus.primary_status: object expected"); - message.primary_status = $root.replicationdata.PrimaryStatus.fromObject(object.primary_status); + let message = new $root.query.TransactionMetadata(); + if (object.dtid != null) + message.dtid = String(object.dtid); + switch (object.state) { + default: + if (typeof object.state === "number") { + message.state = object.state; + break; + } + break; + case "UNKNOWN": + case 0: + message.state = 0; + break; + case "PREPARE": + case 1: + message.state = 1; + break; + case "COMMIT": + case 2: + message.state = 2; + break; + case "ROLLBACK": + case 3: + message.state = 3; + break; } - if (object.gtid_purged != null) - message.gtid_purged = String(object.gtid_purged); - if (object.version != null) - message.version = String(object.version); - if (object.version_comment != null) - message.version_comment = String(object.version_comment); - if (object.read_only != null) - message.read_only = Boolean(object.read_only); - if (object.gtid_mode != null) - message.gtid_mode = String(object.gtid_mode); - if (object.binlog_format != null) - message.binlog_format = String(object.binlog_format); - if (object.binlog_row_image != null) - message.binlog_row_image = String(object.binlog_row_image); - if (object.log_bin_enabled != null) - message.log_bin_enabled = Boolean(object.log_bin_enabled); - if (object.log_replica_updates != null) - message.log_replica_updates = Boolean(object.log_replica_updates); - if (object.semi_sync_primary_enabled != null) - message.semi_sync_primary_enabled = Boolean(object.semi_sync_primary_enabled); - if (object.semi_sync_replica_enabled != null) - message.semi_sync_replica_enabled = Boolean(object.semi_sync_replica_enabled); - if (object.semi_sync_primary_status != null) - message.semi_sync_primary_status = Boolean(object.semi_sync_primary_status); - if (object.semi_sync_replica_status != null) - message.semi_sync_replica_status = Boolean(object.semi_sync_replica_status); - if (object.semi_sync_primary_clients != null) - message.semi_sync_primary_clients = object.semi_sync_primary_clients >>> 0; - if (object.semi_sync_primary_timeout != null) + if (object.time_created != null) if ($util.Long) - (message.semi_sync_primary_timeout = $util.Long.fromValue(object.semi_sync_primary_timeout)).unsigned = true; - else if (typeof object.semi_sync_primary_timeout === "string") - message.semi_sync_primary_timeout = parseInt(object.semi_sync_primary_timeout, 10); - else if (typeof object.semi_sync_primary_timeout === "number") - message.semi_sync_primary_timeout = object.semi_sync_primary_timeout; - else if (typeof object.semi_sync_primary_timeout === "object") - message.semi_sync_primary_timeout = new $util.LongBits(object.semi_sync_primary_timeout.low >>> 0, object.semi_sync_primary_timeout.high >>> 0).toNumber(true); - if (object.semi_sync_wait_for_replica_count != null) - message.semi_sync_wait_for_replica_count = object.semi_sync_wait_for_replica_count >>> 0; - if (object.super_read_only != null) - message.super_read_only = Boolean(object.super_read_only); + (message.time_created = $util.Long.fromValue(object.time_created)).unsigned = false; + else if (typeof object.time_created === "string") + message.time_created = parseInt(object.time_created, 10); + else if (typeof object.time_created === "number") + message.time_created = object.time_created; + else if (typeof object.time_created === "object") + message.time_created = new $util.LongBits(object.time_created.low >>> 0, object.time_created.high >>> 0).toNumber(); + if (object.participants) { + if (!Array.isArray(object.participants)) + throw TypeError(".query.TransactionMetadata.participants: array expected"); + message.participants = []; + for (let i = 0; i < object.participants.length; ++i) { + if (typeof object.participants[i] !== "object") + throw TypeError(".query.TransactionMetadata.participants: object expected"); + message.participants[i] = $root.query.Target.fromObject(object.participants[i]); + } + } return message; }; /** - * Creates a plain object from a FullStatus message. Also converts values to other types if specified. + * Creates a plain object from a TransactionMetadata message. Also converts values to other types if specified. * @function toObject - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static - * @param {replicationdata.FullStatus} message FullStatus + * @param {query.TransactionMetadata} message TransactionMetadata * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - FullStatus.toObject = function toObject(message, options) { + TransactionMetadata.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.participants = []; if (options.defaults) { - object.server_id = 0; - object.server_uuid = ""; - object.replication_status = null; - object.primary_status = null; - object.gtid_purged = ""; - object.version = ""; - object.version_comment = ""; - object.read_only = false; - object.gtid_mode = ""; - object.binlog_format = ""; - object.binlog_row_image = ""; - object.log_bin_enabled = false; - object.log_replica_updates = false; - object.semi_sync_primary_enabled = false; - object.semi_sync_replica_enabled = false; - object.semi_sync_primary_status = false; - object.semi_sync_replica_status = false; - object.semi_sync_primary_clients = 0; + object.dtid = ""; + object.state = options.enums === String ? "UNKNOWN" : 0; if ($util.Long) { - let long = new $util.Long(0, 0, true); - object.semi_sync_primary_timeout = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + let long = new $util.Long(0, 0, false); + object.time_created = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.semi_sync_primary_timeout = options.longs === String ? "0" : 0; - object.semi_sync_wait_for_replica_count = 0; - object.super_read_only = false; + object.time_created = options.longs === String ? "0" : 0; } - if (message.server_id != null && message.hasOwnProperty("server_id")) - object.server_id = message.server_id; - if (message.server_uuid != null && message.hasOwnProperty("server_uuid")) - object.server_uuid = message.server_uuid; - if (message.replication_status != null && message.hasOwnProperty("replication_status")) - object.replication_status = $root.replicationdata.Status.toObject(message.replication_status, options); - if (message.primary_status != null && message.hasOwnProperty("primary_status")) - object.primary_status = $root.replicationdata.PrimaryStatus.toObject(message.primary_status, options); - if (message.gtid_purged != null && message.hasOwnProperty("gtid_purged")) - object.gtid_purged = message.gtid_purged; - if (message.version != null && message.hasOwnProperty("version")) - object.version = message.version; - if (message.version_comment != null && message.hasOwnProperty("version_comment")) - object.version_comment = message.version_comment; - if (message.read_only != null && message.hasOwnProperty("read_only")) - object.read_only = message.read_only; - if (message.gtid_mode != null && message.hasOwnProperty("gtid_mode")) - object.gtid_mode = message.gtid_mode; - if (message.binlog_format != null && message.hasOwnProperty("binlog_format")) - object.binlog_format = message.binlog_format; - if (message.binlog_row_image != null && message.hasOwnProperty("binlog_row_image")) - object.binlog_row_image = message.binlog_row_image; - if (message.log_bin_enabled != null && message.hasOwnProperty("log_bin_enabled")) - object.log_bin_enabled = message.log_bin_enabled; - if (message.log_replica_updates != null && message.hasOwnProperty("log_replica_updates")) - object.log_replica_updates = message.log_replica_updates; - if (message.semi_sync_primary_enabled != null && message.hasOwnProperty("semi_sync_primary_enabled")) - object.semi_sync_primary_enabled = message.semi_sync_primary_enabled; - if (message.semi_sync_replica_enabled != null && message.hasOwnProperty("semi_sync_replica_enabled")) - object.semi_sync_replica_enabled = message.semi_sync_replica_enabled; - if (message.semi_sync_primary_status != null && message.hasOwnProperty("semi_sync_primary_status")) - object.semi_sync_primary_status = message.semi_sync_primary_status; - if (message.semi_sync_replica_status != null && message.hasOwnProperty("semi_sync_replica_status")) - object.semi_sync_replica_status = message.semi_sync_replica_status; - if (message.semi_sync_primary_clients != null && message.hasOwnProperty("semi_sync_primary_clients")) - object.semi_sync_primary_clients = message.semi_sync_primary_clients; - if (message.semi_sync_primary_timeout != null && message.hasOwnProperty("semi_sync_primary_timeout")) - if (typeof message.semi_sync_primary_timeout === "number") - object.semi_sync_primary_timeout = options.longs === String ? String(message.semi_sync_primary_timeout) : message.semi_sync_primary_timeout; + if (message.dtid != null && message.hasOwnProperty("dtid")) + object.dtid = message.dtid; + if (message.state != null && message.hasOwnProperty("state")) + object.state = options.enums === String ? $root.query.TransactionState[message.state] === undefined ? message.state : $root.query.TransactionState[message.state] : message.state; + if (message.time_created != null && message.hasOwnProperty("time_created")) + if (typeof message.time_created === "number") + object.time_created = options.longs === String ? String(message.time_created) : message.time_created; else - object.semi_sync_primary_timeout = options.longs === String ? $util.Long.prototype.toString.call(message.semi_sync_primary_timeout) : options.longs === Number ? new $util.LongBits(message.semi_sync_primary_timeout.low >>> 0, message.semi_sync_primary_timeout.high >>> 0).toNumber(true) : message.semi_sync_primary_timeout; - if (message.semi_sync_wait_for_replica_count != null && message.hasOwnProperty("semi_sync_wait_for_replica_count")) - object.semi_sync_wait_for_replica_count = message.semi_sync_wait_for_replica_count; - if (message.super_read_only != null && message.hasOwnProperty("super_read_only")) - object.super_read_only = message.super_read_only; + object.time_created = options.longs === String ? $util.Long.prototype.toString.call(message.time_created) : options.longs === Number ? new $util.LongBits(message.time_created.low >>> 0, message.time_created.high >>> 0).toNumber() : message.time_created; + if (message.participants && message.participants.length) { + object.participants = []; + for (let j = 0; j < message.participants.length; ++j) + object.participants[j] = $root.query.Target.toObject(message.participants[j], options); + } return object; }; /** - * Converts this FullStatus to JSON. + * Converts this TransactionMetadata to JSON. * @function toJSON - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @instance * @returns {Object.} JSON object */ - FullStatus.prototype.toJSON = function toJSON() { + TransactionMetadata.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for FullStatus + * Gets the default type url for TransactionMetadata * @function getTypeUrl - * @memberof replicationdata.FullStatus + * @memberof query.TransactionMetadata * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - FullStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + TransactionMetadata.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/replicationdata.FullStatus"; + return typeUrlPrefix + "/query.TransactionMetadata"; }; - return FullStatus; + return TransactionMetadata; })(); - return replicationdata; -})(); - -export const vschema = $root.vschema = (() => { - /** - * Namespace vschema. - * @exports vschema - * @namespace + * SchemaTableType enum. + * @name query.SchemaTableType + * @enum {number} + * @property {number} VIEWS=0 VIEWS value + * @property {number} TABLES=1 TABLES value + * @property {number} ALL=2 ALL value */ - const vschema = {}; + query.SchemaTableType = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "VIEWS"] = 0; + values[valuesById[1] = "TABLES"] = 1; + values[valuesById[2] = "ALL"] = 2; + return values; + })(); - vschema.RoutingRules = (function() { + query.GetSchemaRequest = (function() { /** - * Properties of a RoutingRules. - * @memberof vschema - * @interface IRoutingRules - * @property {Array.|null} [rules] RoutingRules rules + * Properties of a GetSchemaRequest. + * @memberof query + * @interface IGetSchemaRequest + * @property {query.ITarget|null} [target] GetSchemaRequest target + * @property {query.SchemaTableType|null} [table_type] GetSchemaRequest table_type + * @property {Array.|null} [table_names] GetSchemaRequest table_names */ /** - * Constructs a new RoutingRules. - * @memberof vschema - * @classdesc Represents a RoutingRules. - * @implements IRoutingRules + * Constructs a new GetSchemaRequest. + * @memberof query + * @classdesc Represents a GetSchemaRequest. + * @implements IGetSchemaRequest * @constructor - * @param {vschema.IRoutingRules=} [properties] Properties to set + * @param {query.IGetSchemaRequest=} [properties] Properties to set */ - function RoutingRules(properties) { - this.rules = []; + function GetSchemaRequest(properties) { + this.table_names = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -95043,224 +95520,293 @@ export const vschema = $root.vschema = (() => { } /** - * RoutingRules rules. - * @member {Array.} rules - * @memberof vschema.RoutingRules + * GetSchemaRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof query.GetSchemaRequest * @instance */ - RoutingRules.prototype.rules = $util.emptyArray; + GetSchemaRequest.prototype.target = null; /** - * Creates a new RoutingRules instance using the specified properties. + * GetSchemaRequest table_type. + * @member {query.SchemaTableType} table_type + * @memberof query.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.table_type = 0; + + /** + * GetSchemaRequest table_names. + * @member {Array.} table_names + * @memberof query.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.table_names = $util.emptyArray; + + /** + * Creates a new GetSchemaRequest instance using the specified properties. * @function create - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static - * @param {vschema.IRoutingRules=} [properties] Properties to set - * @returns {vschema.RoutingRules} RoutingRules instance + * @param {query.IGetSchemaRequest=} [properties] Properties to set + * @returns {query.GetSchemaRequest} GetSchemaRequest instance */ - RoutingRules.create = function create(properties) { - return new RoutingRules(properties); + GetSchemaRequest.create = function create(properties) { + return new GetSchemaRequest(properties); }; /** - * Encodes the specified RoutingRules message. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. + * Encodes the specified GetSchemaRequest message. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. * @function encode - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static - * @param {vschema.IRoutingRules} message RoutingRules message or plain object to encode + * @param {query.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RoutingRules.encode = function encode(message, writer) { + GetSchemaRequest.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.RoutingRule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.table_type != null && Object.hasOwnProperty.call(message, "table_type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.table_type); + if (message.table_names != null && message.table_names.length) + for (let i = 0; i < message.table_names.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.table_names[i]); return writer; }; /** - * Encodes the specified RoutingRules message, length delimited. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. + * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link query.GetSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static - * @param {vschema.IRoutingRules} message RoutingRules message or plain object to encode + * @param {query.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RoutingRules.encodeDelimited = function encodeDelimited(message, writer) { + GetSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RoutingRules message from the specified reader or buffer. + * Decodes a GetSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.RoutingRules} RoutingRules + * @returns {query.GetSchemaRequest} GetSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RoutingRules.decode = function decode(reader, length) { + GetSchemaRequest.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.RoutingRules(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.GetSchemaRequest(); 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.RoutingRule.decode(reader, reader.uint32())); + message.target = $root.query.Target.decode(reader, reader.uint32()); break; } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a RoutingRules message from the specified reader or buffer, length delimited. + case 2: { + message.table_type = reader.int32(); + break; + } + case 3: { + if (!(message.table_names && message.table_names.length)) + message.table_names = []; + message.table_names.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.RoutingRules} RoutingRules + * @returns {query.GetSchemaRequest} GetSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RoutingRules.decodeDelimited = function decodeDelimited(reader) { + GetSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RoutingRules message. + * Verifies a GetSchemaRequest message. * @function verify - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RoutingRules.verify = function verify(message) { + GetSchemaRequest.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.RoutingRule.verify(message.rules[i]); - if (error) - return "rules." + error; + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + if (message.table_type != null && message.hasOwnProperty("table_type")) + switch (message.table_type) { + default: + return "table_type: enum value expected"; + case 0: + case 1: + case 2: + break; } + if (message.table_names != null && message.hasOwnProperty("table_names")) { + if (!Array.isArray(message.table_names)) + return "table_names: array expected"; + for (let i = 0; i < message.table_names.length; ++i) + if (!$util.isString(message.table_names[i])) + return "table_names: string[] expected"; } return null; }; /** - * Creates a RoutingRules message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vschema.RoutingRules} RoutingRules + * @returns {query.GetSchemaRequest} GetSchemaRequest */ - RoutingRules.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.RoutingRules) + GetSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.query.GetSchemaRequest) return object; - let message = new $root.vschema.RoutingRules(); - if (object.rules) { - if (!Array.isArray(object.rules)) - throw TypeError(".vschema.RoutingRules.rules: array expected"); - message.rules = []; - for (let i = 0; i < object.rules.length; ++i) { - if (typeof object.rules[i] !== "object") - throw TypeError(".vschema.RoutingRules.rules: object expected"); - message.rules[i] = $root.vschema.RoutingRule.fromObject(object.rules[i]); + let message = new $root.query.GetSchemaRequest(); + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".query.GetSchemaRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + switch (object.table_type) { + default: + if (typeof object.table_type === "number") { + message.table_type = object.table_type; + break; } + break; + case "VIEWS": + case 0: + message.table_type = 0; + break; + case "TABLES": + case 1: + message.table_type = 1; + break; + case "ALL": + case 2: + message.table_type = 2; + break; + } + if (object.table_names) { + if (!Array.isArray(object.table_names)) + throw TypeError(".query.GetSchemaRequest.table_names: array expected"); + message.table_names = []; + for (let i = 0; i < object.table_names.length; ++i) + message.table_names[i] = String(object.table_names[i]); } return message; }; /** - * Creates a plain object from a RoutingRules message. Also converts values to other types if specified. + * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static - * @param {vschema.RoutingRules} message RoutingRules + * @param {query.GetSchemaRequest} message GetSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RoutingRules.toObject = function toObject(message, options) { + GetSchemaRequest.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.RoutingRule.toObject(message.rules[j], options); + object.table_names = []; + if (options.defaults) { + object.target = null; + object.table_type = options.enums === String ? "VIEWS" : 0; + } + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + if (message.table_type != null && message.hasOwnProperty("table_type")) + object.table_type = options.enums === String ? $root.query.SchemaTableType[message.table_type] === undefined ? message.table_type : $root.query.SchemaTableType[message.table_type] : message.table_type; + if (message.table_names && message.table_names.length) { + object.table_names = []; + for (let j = 0; j < message.table_names.length; ++j) + object.table_names[j] = message.table_names[j]; } return object; }; /** - * Converts this RoutingRules to JSON. + * Converts this GetSchemaRequest to JSON. * @function toJSON - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @instance * @returns {Object.} JSON object */ - RoutingRules.prototype.toJSON = function toJSON() { + GetSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RoutingRules + * Gets the default type url for GetSchemaRequest * @function getTypeUrl - * @memberof vschema.RoutingRules + * @memberof query.GetSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RoutingRules.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.RoutingRules"; + return typeUrlPrefix + "/query.GetSchemaRequest"; }; - return RoutingRules; + return GetSchemaRequest; })(); - vschema.RoutingRule = (function() { + query.GetSchemaResponse = (function() { /** - * Properties of a RoutingRule. - * @memberof vschema - * @interface IRoutingRule - * @property {string|null} [from_table] RoutingRule from_table - * @property {Array.|null} [to_tables] RoutingRule to_tables + * Properties of a GetSchemaResponse. + * @memberof query + * @interface IGetSchemaResponse + * @property {Object.|null} [table_definition] GetSchemaResponse table_definition */ /** - * Constructs a new RoutingRule. - * @memberof vschema - * @classdesc Represents a RoutingRule. - * @implements IRoutingRule + * Constructs a new GetSchemaResponse. + * @memberof query + * @classdesc Represents a GetSchemaResponse. + * @implements IGetSchemaResponse * @constructor - * @param {vschema.IRoutingRule=} [properties] Properties to set + * @param {query.IGetSchemaResponse=} [properties] Properties to set */ - function RoutingRule(properties) { - this.to_tables = []; + function GetSchemaResponse(properties) { + this.table_definition = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -95268,92 +95814,95 @@ export const vschema = $root.vschema = (() => { } /** - * RoutingRule from_table. - * @member {string} from_table - * @memberof vschema.RoutingRule - * @instance - */ - RoutingRule.prototype.from_table = ""; - - /** - * RoutingRule to_tables. - * @member {Array.} to_tables - * @memberof vschema.RoutingRule + * GetSchemaResponse table_definition. + * @member {Object.} table_definition + * @memberof query.GetSchemaResponse * @instance */ - RoutingRule.prototype.to_tables = $util.emptyArray; + GetSchemaResponse.prototype.table_definition = $util.emptyObject; /** - * Creates a new RoutingRule instance using the specified properties. + * Creates a new GetSchemaResponse instance using the specified properties. * @function create - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static - * @param {vschema.IRoutingRule=} [properties] Properties to set - * @returns {vschema.RoutingRule} RoutingRule instance + * @param {query.IGetSchemaResponse=} [properties] Properties to set + * @returns {query.GetSchemaResponse} GetSchemaResponse instance */ - RoutingRule.create = function create(properties) { - return new RoutingRule(properties); + GetSchemaResponse.create = function create(properties) { + return new GetSchemaResponse(properties); }; /** - * Encodes the specified RoutingRule message. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. + * Encodes the specified GetSchemaResponse message. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. * @function encode - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static - * @param {vschema.IRoutingRule} message RoutingRule message or plain object to encode + * @param {query.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RoutingRule.encode = function encode(message, writer) { + GetSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.from_table != null && Object.hasOwnProperty.call(message, "from_table")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.from_table); - if (message.to_tables != null && message.to_tables.length) - for (let i = 0; i < message.to_tables.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.to_tables[i]); + if (message.table_definition != null && Object.hasOwnProperty.call(message, "table_definition")) + for (let keys = Object.keys(message.table_definition), i = 0; i < keys.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.table_definition[keys[i]]).ldelim(); return writer; }; /** - * Encodes the specified RoutingRule message, length delimited. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. + * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link query.GetSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static - * @param {vschema.IRoutingRule} message RoutingRule message or plain object to encode + * @param {query.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RoutingRule.encodeDelimited = function encodeDelimited(message, writer) { + GetSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RoutingRule message from the specified reader or buffer. + * Decodes a GetSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.RoutingRule} RoutingRule + * @returns {query.GetSchemaResponse} GetSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RoutingRule.decode = function decode(reader, length) { + GetSchemaResponse.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.RoutingRule(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.query.GetSchemaResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.from_table = reader.string(); - break; - } case 2: { - if (!(message.to_tables && message.to_tables.length)) - message.to_tables = []; - message.to_tables.push(reader.string()); + if (message.table_definition === $util.emptyObject) + message.table_definition = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = ""; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = reader.string(); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.table_definition[key] = value; break; } default: @@ -95365,149 +95914,169 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a RoutingRule message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.RoutingRule} RoutingRule + * @returns {query.GetSchemaResponse} GetSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RoutingRule.decodeDelimited = function decodeDelimited(reader) { + GetSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RoutingRule message. + * Verifies a GetSchemaResponse message. * @function verify - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RoutingRule.verify = function verify(message) { + GetSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.from_table != null && message.hasOwnProperty("from_table")) - if (!$util.isString(message.from_table)) - return "from_table: string expected"; - if (message.to_tables != null && message.hasOwnProperty("to_tables")) { - if (!Array.isArray(message.to_tables)) - return "to_tables: array expected"; - for (let i = 0; i < message.to_tables.length; ++i) - if (!$util.isString(message.to_tables[i])) - return "to_tables: string[] expected"; + if (message.table_definition != null && message.hasOwnProperty("table_definition")) { + if (!$util.isObject(message.table_definition)) + return "table_definition: object expected"; + let key = Object.keys(message.table_definition); + for (let i = 0; i < key.length; ++i) + if (!$util.isString(message.table_definition[key[i]])) + return "table_definition: string{k:string} expected"; } return null; }; /** - * Creates a RoutingRule message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vschema.RoutingRule} RoutingRule + * @returns {query.GetSchemaResponse} GetSchemaResponse */ - RoutingRule.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.RoutingRule) + GetSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.query.GetSchemaResponse) return object; - let message = new $root.vschema.RoutingRule(); - if (object.from_table != null) - message.from_table = String(object.from_table); - if (object.to_tables) { - if (!Array.isArray(object.to_tables)) - throw TypeError(".vschema.RoutingRule.to_tables: array expected"); - message.to_tables = []; - for (let i = 0; i < object.to_tables.length; ++i) - message.to_tables[i] = String(object.to_tables[i]); + let message = new $root.query.GetSchemaResponse(); + if (object.table_definition) { + if (typeof object.table_definition !== "object") + throw TypeError(".query.GetSchemaResponse.table_definition: object expected"); + message.table_definition = {}; + for (let keys = Object.keys(object.table_definition), i = 0; i < keys.length; ++i) + message.table_definition[keys[i]] = String(object.table_definition[keys[i]]); } return message; }; /** - * Creates a plain object from a RoutingRule message. Also converts values to other types if specified. + * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static - * @param {vschema.RoutingRule} message RoutingRule + * @param {query.GetSchemaResponse} message GetSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RoutingRule.toObject = function toObject(message, options) { + GetSchemaResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.to_tables = []; - if (options.defaults) - object.from_table = ""; - if (message.from_table != null && message.hasOwnProperty("from_table")) - object.from_table = message.from_table; - if (message.to_tables && message.to_tables.length) { - object.to_tables = []; - for (let j = 0; j < message.to_tables.length; ++j) - object.to_tables[j] = message.to_tables[j]; + if (options.objects || options.defaults) + object.table_definition = {}; + let keys2; + if (message.table_definition && (keys2 = Object.keys(message.table_definition)).length) { + object.table_definition = {}; + for (let j = 0; j < keys2.length; ++j) + object.table_definition[keys2[j]] = message.table_definition[keys2[j]]; } return object; }; /** - * Converts this RoutingRule to JSON. + * Converts this GetSchemaResponse to JSON. * @function toJSON - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @instance * @returns {Object.} JSON object */ - RoutingRule.prototype.toJSON = function toJSON() { + GetSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RoutingRule + * Gets the default type url for GetSchemaResponse * @function getTypeUrl - * @memberof vschema.RoutingRule + * @memberof query.GetSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RoutingRule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.RoutingRule"; + return typeUrlPrefix + "/query.GetSchemaResponse"; }; - return RoutingRule; + return GetSchemaResponse; })(); - vschema.Keyspace = (function() { + return query; +})(); + +export const replicationdata = $root.replicationdata = (() => { + + /** + * Namespace replicationdata. + * @exports replicationdata + * @namespace + */ + const replicationdata = {}; + + replicationdata.Status = (function() { /** - * Properties of a Keyspace. - * @memberof vschema - * @interface IKeyspace - * @property {boolean|null} [sharded] Keyspace sharded - * @property {Object.|null} [vindexes] Keyspace vindexes - * @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 + * Properties of a Status. + * @memberof replicationdata + * @interface IStatus + * @property {string|null} [position] Status position + * @property {number|null} [replication_lag_seconds] Status replication_lag_seconds + * @property {string|null} [source_host] Status source_host + * @property {number|null} [source_port] Status source_port + * @property {number|null} [connect_retry] Status connect_retry + * @property {string|null} [relay_log_position] Status relay_log_position + * @property {string|null} [file_position] Status file_position + * @property {string|null} [relay_log_source_binlog_equivalent_position] Status relay_log_source_binlog_equivalent_position + * @property {number|null} [source_server_id] Status source_server_id + * @property {string|null} [source_uuid] Status source_uuid + * @property {number|null} [io_state] Status io_state + * @property {string|null} [last_io_error] Status last_io_error + * @property {number|null} [sql_state] Status sql_state + * @property {string|null} [last_sql_error] Status last_sql_error + * @property {string|null} [relay_log_file_position] Status relay_log_file_position + * @property {string|null} [source_user] Status source_user + * @property {number|null} [sql_delay] Status sql_delay + * @property {boolean|null} [auto_position] Status auto_position + * @property {boolean|null} [using_gtid] Status using_gtid + * @property {boolean|null} [has_replication_filters] Status has_replication_filters + * @property {boolean|null} [ssl_allowed] Status ssl_allowed + * @property {boolean|null} [replication_lag_unknown] Status replication_lag_unknown */ /** - * Constructs a new Keyspace. - * @memberof vschema - * @classdesc Represents a Keyspace. - * @implements IKeyspace + * Constructs a new Status. + * @memberof replicationdata + * @classdesc Represents a Status. + * @implements IStatus * @constructor - * @param {vschema.IKeyspace=} [properties] Properties to set + * @param {replicationdata.IStatus=} [properties] Properties to set */ - function Keyspace(properties) { - this.vindexes = {}; - this.tables = {}; + function Status(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -95515,715 +96084,666 @@ export const vschema = $root.vschema = (() => { } /** - * Keyspace sharded. - * @member {boolean} sharded - * @memberof vschema.Keyspace + * Status position. + * @member {string} position + * @memberof replicationdata.Status * @instance */ - Keyspace.prototype.sharded = false; + Status.prototype.position = ""; /** - * Keyspace vindexes. - * @member {Object.} vindexes - * @memberof vschema.Keyspace + * Status replication_lag_seconds. + * @member {number} replication_lag_seconds + * @memberof replicationdata.Status * @instance */ - Keyspace.prototype.vindexes = $util.emptyObject; + Status.prototype.replication_lag_seconds = 0; /** - * Keyspace tables. - * @member {Object.} tables - * @memberof vschema.Keyspace + * Status source_host. + * @member {string} source_host + * @memberof replicationdata.Status * @instance */ - Keyspace.prototype.tables = $util.emptyObject; + Status.prototype.source_host = ""; /** - * Keyspace require_explicit_routing. - * @member {boolean} require_explicit_routing - * @memberof vschema.Keyspace + * Status source_port. + * @member {number} source_port + * @memberof replicationdata.Status * @instance */ - Keyspace.prototype.require_explicit_routing = false; + Status.prototype.source_port = 0; /** - * Keyspace foreign_key_mode. - * @member {vschema.Keyspace.ForeignKeyMode} foreign_key_mode - * @memberof vschema.Keyspace + * Status connect_retry. + * @member {number} connect_retry + * @memberof replicationdata.Status * @instance */ - Keyspace.prototype.foreign_key_mode = 0; + Status.prototype.connect_retry = 0; /** - * Creates a new Keyspace instance using the specified properties. - * @function create - * @memberof vschema.Keyspace - * @static - * @param {vschema.IKeyspace=} [properties] Properties to set - * @returns {vschema.Keyspace} Keyspace instance + * Status relay_log_position. + * @member {string} relay_log_position + * @memberof replicationdata.Status + * @instance */ - Keyspace.create = function create(properties) { - return new Keyspace(properties); - }; + Status.prototype.relay_log_position = ""; /** - * Encodes the specified Keyspace message. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. - * @function encode - * @memberof vschema.Keyspace - * @static - * @param {vschema.IKeyspace} message Keyspace message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * Status file_position. + * @member {string} file_position + * @memberof replicationdata.Status + * @instance */ - Keyspace.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.sharded != null && Object.hasOwnProperty.call(message, "sharded")) - writer.uint32(/* id 1, wireType 0 =*/8).bool(message.sharded); - if (message.vindexes != null && Object.hasOwnProperty.call(message, "vindexes")) - for (let keys = Object.keys(message.vindexes), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vschema.Vindex.encode(message.vindexes[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) - for (let keys = Object.keys(message.tables), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 3, wireType 2 =*/26).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vschema.Table.encode(message.tables[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.require_explicit_routing != null && Object.hasOwnProperty.call(message, "require_explicit_routing")) - 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); - return writer; - }; + Status.prototype.file_position = ""; /** - * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. - * @function encodeDelimited - * @memberof vschema.Keyspace - * @static - * @param {vschema.IKeyspace} message Keyspace message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * Status relay_log_source_binlog_equivalent_position. + * @member {string} relay_log_source_binlog_equivalent_position + * @memberof replicationdata.Status + * @instance */ - Keyspace.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + Status.prototype.relay_log_source_binlog_equivalent_position = ""; /** - * Decodes a Keyspace message from the specified reader or buffer. - * @function decode - * @memberof vschema.Keyspace - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vschema.Keyspace} Keyspace - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * Status source_server_id. + * @member {number} source_server_id + * @memberof replicationdata.Status + * @instance */ - Keyspace.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.Keyspace(), key, value; - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.sharded = reader.bool(); - break; - } - case 2: { - if (message.vindexes === $util.emptyObject) - message.vindexes = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vschema.Vindex.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.vindexes[key] = value; - break; - } - case 3: { - if (message.tables === $util.emptyObject) - message.tables = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vschema.Table.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.tables[key] = value; - break; - } - case 4: { - message.require_explicit_routing = reader.bool(); - break; - } - case 5: { - message.foreign_key_mode = reader.int32(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; + Status.prototype.source_server_id = 0; /** - * Decodes a Keyspace message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vschema.Keyspace - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.Keyspace} Keyspace - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * Status source_uuid. + * @member {string} source_uuid + * @memberof replicationdata.Status + * @instance */ - Keyspace.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + Status.prototype.source_uuid = ""; /** - * Verifies a Keyspace message. - * @function verify - * @memberof vschema.Keyspace - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not + * Status io_state. + * @member {number} io_state + * @memberof replicationdata.Status + * @instance */ - Keyspace.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.sharded != null && message.hasOwnProperty("sharded")) - if (typeof message.sharded !== "boolean") - return "sharded: boolean expected"; - if (message.vindexes != null && message.hasOwnProperty("vindexes")) { - if (!$util.isObject(message.vindexes)) - return "vindexes: object expected"; - let key = Object.keys(message.vindexes); - for (let i = 0; i < key.length; ++i) { - let error = $root.vschema.Vindex.verify(message.vindexes[key[i]]); - if (error) - return "vindexes." + error; - } - } - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!$util.isObject(message.tables)) - return "tables: object expected"; - let key = Object.keys(message.tables); - for (let i = 0; i < key.length; ++i) { - let error = $root.vschema.Table.verify(message.tables[key[i]]); - if (error) - return "tables." + error; - } - } - if (message.require_explicit_routing != null && message.hasOwnProperty("require_explicit_routing")) - if (typeof message.require_explicit_routing !== "boolean") - return "require_explicit_routing: boolean expected"; - if (message.foreign_key_mode != null && message.hasOwnProperty("foreign_key_mode")) - switch (message.foreign_key_mode) { - default: - return "foreign_key_mode: enum value expected"; - case 0: - case 1: - case 2: - case 3: - break; - } - return null; - }; + Status.prototype.io_state = 0; /** - * Creates a Keyspace message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vschema.Keyspace - * @static - * @param {Object.} object Plain object - * @returns {vschema.Keyspace} Keyspace + * Status last_io_error. + * @member {string} last_io_error + * @memberof replicationdata.Status + * @instance */ - Keyspace.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.Keyspace) - return object; - let message = new $root.vschema.Keyspace(); - if (object.sharded != null) - message.sharded = Boolean(object.sharded); - if (object.vindexes) { - if (typeof object.vindexes !== "object") - throw TypeError(".vschema.Keyspace.vindexes: object expected"); - message.vindexes = {}; - for (let keys = Object.keys(object.vindexes), i = 0; i < keys.length; ++i) { - if (typeof object.vindexes[keys[i]] !== "object") - throw TypeError(".vschema.Keyspace.vindexes: object expected"); - message.vindexes[keys[i]] = $root.vschema.Vindex.fromObject(object.vindexes[keys[i]]); - } - } - if (object.tables) { - if (typeof object.tables !== "object") - throw TypeError(".vschema.Keyspace.tables: object expected"); - message.tables = {}; - for (let keys = Object.keys(object.tables), i = 0; i < keys.length; ++i) { - if (typeof object.tables[keys[i]] !== "object") - throw TypeError(".vschema.Keyspace.tables: object expected"); - message.tables[keys[i]] = $root.vschema.Table.fromObject(object.tables[keys[i]]); - } - } - if (object.require_explicit_routing != null) - message.require_explicit_routing = Boolean(object.require_explicit_routing); - switch (object.foreign_key_mode) { - default: - if (typeof object.foreign_key_mode === "number") { - message.foreign_key_mode = object.foreign_key_mode; - break; - } - break; - case "FK_DEFAULT": - case 0: - message.foreign_key_mode = 0; - break; - case "FK_DISALLOW": - case 1: - message.foreign_key_mode = 1; - break; - case "FK_UNMANAGED": - case 2: - message.foreign_key_mode = 2; - break; - case "FK_MANAGED": - case 3: - message.foreign_key_mode = 3; - break; - } - return message; - }; + Status.prototype.last_io_error = ""; /** - * Creates a plain object from a Keyspace message. Also converts values to other types if specified. - * @function toObject - * @memberof vschema.Keyspace - * @static - * @param {vschema.Keyspace} message Keyspace - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object + * Status sql_state. + * @member {number} sql_state + * @memberof replicationdata.Status + * @instance */ - Keyspace.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.objects || options.defaults) { - object.vindexes = {}; - object.tables = {}; - } - if (options.defaults) { - object.sharded = false; - object.require_explicit_routing = false; - object.foreign_key_mode = options.enums === String ? "FK_DEFAULT" : 0; - } - if (message.sharded != null && message.hasOwnProperty("sharded")) - object.sharded = message.sharded; - let keys2; - if (message.vindexes && (keys2 = Object.keys(message.vindexes)).length) { - object.vindexes = {}; - for (let j = 0; j < keys2.length; ++j) - object.vindexes[keys2[j]] = $root.vschema.Vindex.toObject(message.vindexes[keys2[j]], options); - } - if (message.tables && (keys2 = Object.keys(message.tables)).length) { - object.tables = {}; - for (let j = 0; j < keys2.length; ++j) - object.tables[keys2[j]] = $root.vschema.Table.toObject(message.tables[keys2[j]], options); - } - if (message.require_explicit_routing != null && message.hasOwnProperty("require_explicit_routing")) - 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; - return object; - }; + Status.prototype.sql_state = 0; /** - * Converts this Keyspace to JSON. - * @function toJSON - * @memberof vschema.Keyspace + * Status last_sql_error. + * @member {string} last_sql_error + * @memberof replicationdata.Status * @instance - * @returns {Object.} JSON object */ - Keyspace.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + Status.prototype.last_sql_error = ""; /** - * Gets the default type url for Keyspace - * @function getTypeUrl - * @memberof vschema.Keyspace - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url + * Status relay_log_file_position. + * @member {string} relay_log_file_position + * @memberof replicationdata.Status + * @instance */ - Keyspace.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vschema.Keyspace"; - }; + Status.prototype.relay_log_file_position = ""; /** - * ForeignKeyMode enum. - * @name vschema.Keyspace.ForeignKeyMode - * @enum {number} - * @property {number} FK_DEFAULT=0 FK_DEFAULT value - * @property {number} FK_DISALLOW=1 FK_DISALLOW value - * @property {number} FK_UNMANAGED=2 FK_UNMANAGED value - * @property {number} FK_MANAGED=3 FK_MANAGED value + * Status source_user. + * @member {string} source_user + * @memberof replicationdata.Status + * @instance */ - Keyspace.ForeignKeyMode = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "FK_DEFAULT"] = 0; - values[valuesById[1] = "FK_DISALLOW"] = 1; - values[valuesById[2] = "FK_UNMANAGED"] = 2; - values[valuesById[3] = "FK_MANAGED"] = 3; - return values; - })(); - - return Keyspace; - })(); + Status.prototype.source_user = ""; - vschema.Vindex = (function() { + /** + * Status sql_delay. + * @member {number} sql_delay + * @memberof replicationdata.Status + * @instance + */ + Status.prototype.sql_delay = 0; /** - * Properties of a Vindex. - * @memberof vschema - * @interface IVindex - * @property {string|null} [type] Vindex type - * @property {Object.|null} [params] Vindex params - * @property {string|null} [owner] Vindex owner + * Status auto_position. + * @member {boolean} auto_position + * @memberof replicationdata.Status + * @instance */ + Status.prototype.auto_position = false; /** - * Constructs a new Vindex. - * @memberof vschema - * @classdesc Represents a Vindex. - * @implements IVindex - * @constructor - * @param {vschema.IVindex=} [properties] Properties to set + * Status using_gtid. + * @member {boolean} using_gtid + * @memberof replicationdata.Status + * @instance */ - function Vindex(properties) { - this.params = {}; - 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]]; - } + Status.prototype.using_gtid = false; /** - * Vindex type. - * @member {string} type - * @memberof vschema.Vindex + * Status has_replication_filters. + * @member {boolean} has_replication_filters + * @memberof replicationdata.Status * @instance */ - Vindex.prototype.type = ""; + Status.prototype.has_replication_filters = false; /** - * Vindex params. - * @member {Object.} params - * @memberof vschema.Vindex + * Status ssl_allowed. + * @member {boolean} ssl_allowed + * @memberof replicationdata.Status * @instance */ - Vindex.prototype.params = $util.emptyObject; + Status.prototype.ssl_allowed = false; /** - * Vindex owner. - * @member {string} owner - * @memberof vschema.Vindex + * Status replication_lag_unknown. + * @member {boolean} replication_lag_unknown + * @memberof replicationdata.Status * @instance */ - Vindex.prototype.owner = ""; + Status.prototype.replication_lag_unknown = false; /** - * Creates a new Vindex instance using the specified properties. + * Creates a new Status instance using the specified properties. * @function create - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static - * @param {vschema.IVindex=} [properties] Properties to set - * @returns {vschema.Vindex} Vindex instance + * @param {replicationdata.IStatus=} [properties] Properties to set + * @returns {replicationdata.Status} Status instance */ - Vindex.create = function create(properties) { - return new Vindex(properties); + Status.create = function create(properties) { + return new Status(properties); }; /** - * Encodes the specified Vindex message. Does not implicitly {@link vschema.Vindex.verify|verify} messages. + * Encodes the specified Status message. Does not implicitly {@link replicationdata.Status.verify|verify} messages. * @function encode - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static - * @param {vschema.IVindex} message Vindex message or plain object to encode + * @param {replicationdata.IStatus} message Status message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Vindex.encode = function encode(message, writer) { + Status.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.type); - if (message.params != null && Object.hasOwnProperty.call(message, "params")) - for (let keys = Object.keys(message.params), i = 0; i < keys.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.params[keys[i]]).ldelim(); - if (message.owner != null && Object.hasOwnProperty.call(message, "owner")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.owner); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); + if (message.replication_lag_seconds != null && Object.hasOwnProperty.call(message, "replication_lag_seconds")) + writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.replication_lag_seconds); + if (message.source_host != null && Object.hasOwnProperty.call(message, "source_host")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_host); + if (message.source_port != null && Object.hasOwnProperty.call(message, "source_port")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.source_port); + if (message.connect_retry != null && Object.hasOwnProperty.call(message, "connect_retry")) + writer.uint32(/* id 7, wireType 0 =*/56).int32(message.connect_retry); + if (message.relay_log_position != null && Object.hasOwnProperty.call(message, "relay_log_position")) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.relay_log_position); + if (message.file_position != null && Object.hasOwnProperty.call(message, "file_position")) + writer.uint32(/* id 9, wireType 2 =*/74).string(message.file_position); + if (message.relay_log_source_binlog_equivalent_position != null && Object.hasOwnProperty.call(message, "relay_log_source_binlog_equivalent_position")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.relay_log_source_binlog_equivalent_position); + if (message.source_server_id != null && Object.hasOwnProperty.call(message, "source_server_id")) + writer.uint32(/* id 11, wireType 0 =*/88).uint32(message.source_server_id); + if (message.source_uuid != null && Object.hasOwnProperty.call(message, "source_uuid")) + writer.uint32(/* id 12, wireType 2 =*/98).string(message.source_uuid); + if (message.io_state != null && Object.hasOwnProperty.call(message, "io_state")) + writer.uint32(/* id 13, wireType 0 =*/104).int32(message.io_state); + if (message.last_io_error != null && Object.hasOwnProperty.call(message, "last_io_error")) + writer.uint32(/* id 14, wireType 2 =*/114).string(message.last_io_error); + if (message.sql_state != null && Object.hasOwnProperty.call(message, "sql_state")) + writer.uint32(/* id 15, wireType 0 =*/120).int32(message.sql_state); + if (message.last_sql_error != null && Object.hasOwnProperty.call(message, "last_sql_error")) + writer.uint32(/* id 16, wireType 2 =*/130).string(message.last_sql_error); + if (message.relay_log_file_position != null && Object.hasOwnProperty.call(message, "relay_log_file_position")) + writer.uint32(/* id 17, wireType 2 =*/138).string(message.relay_log_file_position); + if (message.source_user != null && Object.hasOwnProperty.call(message, "source_user")) + writer.uint32(/* id 18, wireType 2 =*/146).string(message.source_user); + if (message.sql_delay != null && Object.hasOwnProperty.call(message, "sql_delay")) + writer.uint32(/* id 19, wireType 0 =*/152).uint32(message.sql_delay); + if (message.auto_position != null && Object.hasOwnProperty.call(message, "auto_position")) + writer.uint32(/* id 20, wireType 0 =*/160).bool(message.auto_position); + if (message.using_gtid != null && Object.hasOwnProperty.call(message, "using_gtid")) + writer.uint32(/* id 21, wireType 0 =*/168).bool(message.using_gtid); + if (message.has_replication_filters != null && Object.hasOwnProperty.call(message, "has_replication_filters")) + writer.uint32(/* id 22, wireType 0 =*/176).bool(message.has_replication_filters); + if (message.ssl_allowed != null && Object.hasOwnProperty.call(message, "ssl_allowed")) + writer.uint32(/* id 23, wireType 0 =*/184).bool(message.ssl_allowed); + if (message.replication_lag_unknown != null && Object.hasOwnProperty.call(message, "replication_lag_unknown")) + writer.uint32(/* id 24, wireType 0 =*/192).bool(message.replication_lag_unknown); return writer; }; /** - * Encodes the specified Vindex message, length delimited. Does not implicitly {@link vschema.Vindex.verify|verify} messages. + * Encodes the specified Status message, length delimited. Does not implicitly {@link replicationdata.Status.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static - * @param {vschema.IVindex} message Vindex message or plain object to encode + * @param {replicationdata.IStatus} message Status message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Vindex.encodeDelimited = function encodeDelimited(message, writer) { + Status.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Vindex message from the specified reader or buffer. + * Decodes a Status message from the specified reader or buffer. * @function decode - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.Vindex} Vindex + * @returns {replicationdata.Status} Status * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Vindex.decode = function decode(reader, length) { + Status.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.Vindex(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.replicationdata.Status(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.type = reader.string(); + message.position = reader.string(); break; } - case 2: { - if (message.params === $util.emptyObject) - message.params = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = ""; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = reader.string(); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.params[key] = value; + case 4: { + message.replication_lag_seconds = reader.uint32(); break; } - case 3: { - message.owner = reader.string(); + case 5: { + message.source_host = reader.string(); break; } - default: - reader.skipType(tag & 7); - break; - } + case 6: { + message.source_port = reader.int32(); + break; + } + case 7: { + message.connect_retry = reader.int32(); + break; + } + case 8: { + message.relay_log_position = reader.string(); + break; + } + case 9: { + message.file_position = reader.string(); + break; + } + case 10: { + message.relay_log_source_binlog_equivalent_position = reader.string(); + break; + } + case 11: { + message.source_server_id = reader.uint32(); + break; + } + case 12: { + message.source_uuid = reader.string(); + break; + } + case 13: { + message.io_state = reader.int32(); + break; + } + case 14: { + message.last_io_error = reader.string(); + break; + } + case 15: { + message.sql_state = reader.int32(); + break; + } + case 16: { + message.last_sql_error = reader.string(); + break; + } + case 17: { + message.relay_log_file_position = reader.string(); + break; + } + case 18: { + message.source_user = reader.string(); + break; + } + case 19: { + message.sql_delay = reader.uint32(); + break; + } + case 20: { + message.auto_position = reader.bool(); + break; + } + case 21: { + message.using_gtid = reader.bool(); + break; + } + case 22: { + message.has_replication_filters = reader.bool(); + break; + } + case 23: { + message.ssl_allowed = reader.bool(); + break; + } + case 24: { + message.replication_lag_unknown = reader.bool(); + break; + } + default: + reader.skipType(tag & 7); + break; + } } return message; }; /** - * Decodes a Vindex message from the specified reader or buffer, length delimited. + * Decodes a Status message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.Vindex} Vindex + * @returns {replicationdata.Status} Status * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Vindex.decodeDelimited = function decodeDelimited(reader) { + Status.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Vindex message. + * Verifies a Status message. * @function verify - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Vindex.verify = function verify(message) { + Status.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.type != null && message.hasOwnProperty("type")) - if (!$util.isString(message.type)) - return "type: string expected"; - if (message.params != null && message.hasOwnProperty("params")) { - if (!$util.isObject(message.params)) - return "params: object expected"; - let key = Object.keys(message.params); - for (let i = 0; i < key.length; ++i) - if (!$util.isString(message.params[key[i]])) - return "params: string{k:string} expected"; - } - if (message.owner != null && message.hasOwnProperty("owner")) - if (!$util.isString(message.owner)) - return "owner: string expected"; + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) + if (!$util.isInteger(message.replication_lag_seconds)) + return "replication_lag_seconds: integer expected"; + if (message.source_host != null && message.hasOwnProperty("source_host")) + if (!$util.isString(message.source_host)) + return "source_host: string expected"; + if (message.source_port != null && message.hasOwnProperty("source_port")) + if (!$util.isInteger(message.source_port)) + return "source_port: integer expected"; + if (message.connect_retry != null && message.hasOwnProperty("connect_retry")) + if (!$util.isInteger(message.connect_retry)) + return "connect_retry: integer expected"; + if (message.relay_log_position != null && message.hasOwnProperty("relay_log_position")) + if (!$util.isString(message.relay_log_position)) + return "relay_log_position: string expected"; + if (message.file_position != null && message.hasOwnProperty("file_position")) + if (!$util.isString(message.file_position)) + return "file_position: string expected"; + if (message.relay_log_source_binlog_equivalent_position != null && message.hasOwnProperty("relay_log_source_binlog_equivalent_position")) + if (!$util.isString(message.relay_log_source_binlog_equivalent_position)) + return "relay_log_source_binlog_equivalent_position: string expected"; + if (message.source_server_id != null && message.hasOwnProperty("source_server_id")) + if (!$util.isInteger(message.source_server_id)) + return "source_server_id: integer expected"; + if (message.source_uuid != null && message.hasOwnProperty("source_uuid")) + if (!$util.isString(message.source_uuid)) + return "source_uuid: string expected"; + if (message.io_state != null && message.hasOwnProperty("io_state")) + if (!$util.isInteger(message.io_state)) + return "io_state: integer expected"; + if (message.last_io_error != null && message.hasOwnProperty("last_io_error")) + if (!$util.isString(message.last_io_error)) + return "last_io_error: string expected"; + if (message.sql_state != null && message.hasOwnProperty("sql_state")) + if (!$util.isInteger(message.sql_state)) + return "sql_state: integer expected"; + if (message.last_sql_error != null && message.hasOwnProperty("last_sql_error")) + if (!$util.isString(message.last_sql_error)) + return "last_sql_error: string expected"; + if (message.relay_log_file_position != null && message.hasOwnProperty("relay_log_file_position")) + if (!$util.isString(message.relay_log_file_position)) + return "relay_log_file_position: string expected"; + if (message.source_user != null && message.hasOwnProperty("source_user")) + if (!$util.isString(message.source_user)) + return "source_user: string expected"; + if (message.sql_delay != null && message.hasOwnProperty("sql_delay")) + if (!$util.isInteger(message.sql_delay)) + return "sql_delay: integer expected"; + if (message.auto_position != null && message.hasOwnProperty("auto_position")) + if (typeof message.auto_position !== "boolean") + return "auto_position: boolean expected"; + if (message.using_gtid != null && message.hasOwnProperty("using_gtid")) + if (typeof message.using_gtid !== "boolean") + return "using_gtid: boolean expected"; + if (message.has_replication_filters != null && message.hasOwnProperty("has_replication_filters")) + if (typeof message.has_replication_filters !== "boolean") + return "has_replication_filters: boolean expected"; + if (message.ssl_allowed != null && message.hasOwnProperty("ssl_allowed")) + if (typeof message.ssl_allowed !== "boolean") + return "ssl_allowed: boolean expected"; + if (message.replication_lag_unknown != null && message.hasOwnProperty("replication_lag_unknown")) + if (typeof message.replication_lag_unknown !== "boolean") + return "replication_lag_unknown: boolean expected"; return null; }; /** - * Creates a Vindex message from a plain object. Also converts values to their respective internal types. + * Creates a Status message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static * @param {Object.} object Plain object - * @returns {vschema.Vindex} Vindex + * @returns {replicationdata.Status} Status */ - Vindex.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.Vindex) + Status.fromObject = function fromObject(object) { + if (object instanceof $root.replicationdata.Status) return object; - let message = new $root.vschema.Vindex(); - if (object.type != null) - message.type = String(object.type); - if (object.params) { - if (typeof object.params !== "object") - throw TypeError(".vschema.Vindex.params: object expected"); - message.params = {}; - for (let keys = Object.keys(object.params), i = 0; i < keys.length; ++i) - message.params[keys[i]] = String(object.params[keys[i]]); - } - if (object.owner != null) - message.owner = String(object.owner); + let message = new $root.replicationdata.Status(); + if (object.position != null) + message.position = String(object.position); + if (object.replication_lag_seconds != null) + message.replication_lag_seconds = object.replication_lag_seconds >>> 0; + if (object.source_host != null) + message.source_host = String(object.source_host); + if (object.source_port != null) + message.source_port = object.source_port | 0; + if (object.connect_retry != null) + message.connect_retry = object.connect_retry | 0; + if (object.relay_log_position != null) + message.relay_log_position = String(object.relay_log_position); + if (object.file_position != null) + message.file_position = String(object.file_position); + if (object.relay_log_source_binlog_equivalent_position != null) + message.relay_log_source_binlog_equivalent_position = String(object.relay_log_source_binlog_equivalent_position); + if (object.source_server_id != null) + message.source_server_id = object.source_server_id >>> 0; + if (object.source_uuid != null) + message.source_uuid = String(object.source_uuid); + if (object.io_state != null) + message.io_state = object.io_state | 0; + if (object.last_io_error != null) + message.last_io_error = String(object.last_io_error); + if (object.sql_state != null) + message.sql_state = object.sql_state | 0; + if (object.last_sql_error != null) + message.last_sql_error = String(object.last_sql_error); + if (object.relay_log_file_position != null) + message.relay_log_file_position = String(object.relay_log_file_position); + if (object.source_user != null) + message.source_user = String(object.source_user); + if (object.sql_delay != null) + message.sql_delay = object.sql_delay >>> 0; + if (object.auto_position != null) + message.auto_position = Boolean(object.auto_position); + if (object.using_gtid != null) + message.using_gtid = Boolean(object.using_gtid); + if (object.has_replication_filters != null) + message.has_replication_filters = Boolean(object.has_replication_filters); + if (object.ssl_allowed != null) + message.ssl_allowed = Boolean(object.ssl_allowed); + if (object.replication_lag_unknown != null) + message.replication_lag_unknown = Boolean(object.replication_lag_unknown); return message; }; /** - * Creates a plain object from a Vindex message. Also converts values to other types if specified. + * Creates a plain object from a Status message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static - * @param {vschema.Vindex} message Vindex + * @param {replicationdata.Status} message Status * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Vindex.toObject = function toObject(message, options) { + Status.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.params = {}; if (options.defaults) { - object.type = ""; - object.owner = ""; - } - if (message.type != null && message.hasOwnProperty("type")) - object.type = message.type; - let keys2; - if (message.params && (keys2 = Object.keys(message.params)).length) { - object.params = {}; - for (let j = 0; j < keys2.length; ++j) - object.params[keys2[j]] = message.params[keys2[j]]; + object.position = ""; + object.replication_lag_seconds = 0; + object.source_host = ""; + object.source_port = 0; + object.connect_retry = 0; + object.relay_log_position = ""; + object.file_position = ""; + object.relay_log_source_binlog_equivalent_position = ""; + object.source_server_id = 0; + object.source_uuid = ""; + object.io_state = 0; + object.last_io_error = ""; + object.sql_state = 0; + object.last_sql_error = ""; + object.relay_log_file_position = ""; + object.source_user = ""; + object.sql_delay = 0; + object.auto_position = false; + object.using_gtid = false; + object.has_replication_filters = false; + object.ssl_allowed = false; + object.replication_lag_unknown = false; } - if (message.owner != null && message.hasOwnProperty("owner")) - object.owner = message.owner; + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.replication_lag_seconds != null && message.hasOwnProperty("replication_lag_seconds")) + object.replication_lag_seconds = message.replication_lag_seconds; + if (message.source_host != null && message.hasOwnProperty("source_host")) + object.source_host = message.source_host; + if (message.source_port != null && message.hasOwnProperty("source_port")) + object.source_port = message.source_port; + if (message.connect_retry != null && message.hasOwnProperty("connect_retry")) + object.connect_retry = message.connect_retry; + if (message.relay_log_position != null && message.hasOwnProperty("relay_log_position")) + object.relay_log_position = message.relay_log_position; + if (message.file_position != null && message.hasOwnProperty("file_position")) + object.file_position = message.file_position; + if (message.relay_log_source_binlog_equivalent_position != null && message.hasOwnProperty("relay_log_source_binlog_equivalent_position")) + object.relay_log_source_binlog_equivalent_position = message.relay_log_source_binlog_equivalent_position; + if (message.source_server_id != null && message.hasOwnProperty("source_server_id")) + object.source_server_id = message.source_server_id; + if (message.source_uuid != null && message.hasOwnProperty("source_uuid")) + object.source_uuid = message.source_uuid; + if (message.io_state != null && message.hasOwnProperty("io_state")) + object.io_state = message.io_state; + if (message.last_io_error != null && message.hasOwnProperty("last_io_error")) + object.last_io_error = message.last_io_error; + if (message.sql_state != null && message.hasOwnProperty("sql_state")) + object.sql_state = message.sql_state; + if (message.last_sql_error != null && message.hasOwnProperty("last_sql_error")) + object.last_sql_error = message.last_sql_error; + if (message.relay_log_file_position != null && message.hasOwnProperty("relay_log_file_position")) + object.relay_log_file_position = message.relay_log_file_position; + if (message.source_user != null && message.hasOwnProperty("source_user")) + object.source_user = message.source_user; + if (message.sql_delay != null && message.hasOwnProperty("sql_delay")) + object.sql_delay = message.sql_delay; + if (message.auto_position != null && message.hasOwnProperty("auto_position")) + object.auto_position = message.auto_position; + if (message.using_gtid != null && message.hasOwnProperty("using_gtid")) + object.using_gtid = message.using_gtid; + if (message.has_replication_filters != null && message.hasOwnProperty("has_replication_filters")) + object.has_replication_filters = message.has_replication_filters; + if (message.ssl_allowed != null && message.hasOwnProperty("ssl_allowed")) + object.ssl_allowed = message.ssl_allowed; + if (message.replication_lag_unknown != null && message.hasOwnProperty("replication_lag_unknown")) + object.replication_lag_unknown = message.replication_lag_unknown; return object; }; /** - * Converts this Vindex to JSON. + * Converts this Status to JSON. * @function toJSON - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @instance * @returns {Object.} JSON object */ - Vindex.prototype.toJSON = function toJSON() { + Status.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Vindex + * Gets the default type url for Status * @function getTypeUrl - * @memberof vschema.Vindex + * @memberof replicationdata.Status * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Vindex.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Status.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.Vindex"; + return typeUrlPrefix + "/replicationdata.Status"; }; - return Vindex; + return Status; })(); - vschema.Table = (function() { + replicationdata.StopReplicationStatus = (function() { /** - * Properties of a Table. - * @memberof vschema - * @interface ITable - * @property {string|null} [type] Table type - * @property {Array.|null} [column_vindexes] Table column_vindexes - * @property {vschema.IAutoIncrement|null} [auto_increment] Table auto_increment - * @property {Array.|null} [columns] Table columns - * @property {string|null} [pinned] Table pinned - * @property {boolean|null} [column_list_authoritative] Table column_list_authoritative - * @property {string|null} [source] Table source + * Properties of a StopReplicationStatus. + * @memberof replicationdata + * @interface IStopReplicationStatus + * @property {replicationdata.IStatus|null} [before] StopReplicationStatus before + * @property {replicationdata.IStatus|null} [after] StopReplicationStatus after */ /** - * Constructs a new Table. - * @memberof vschema - * @classdesc Represents a Table. - * @implements ITable + * Constructs a new StopReplicationStatus. + * @memberof replicationdata + * @classdesc Represents a StopReplicationStatus. + * @implements IStopReplicationStatus * @constructor - * @param {vschema.ITable=} [properties] Properties to set + * @param {replicationdata.IStopReplicationStatus=} [properties] Properties to set */ - function Table(properties) { - this.column_vindexes = []; - this.columns = []; + function StopReplicationStatus(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -96231,165 +96751,89 @@ export const vschema = $root.vschema = (() => { } /** - * Table type. - * @member {string} type - * @memberof vschema.Table - * @instance - */ - Table.prototype.type = ""; - - /** - * Table column_vindexes. - * @member {Array.} column_vindexes - * @memberof vschema.Table - * @instance - */ - Table.prototype.column_vindexes = $util.emptyArray; - - /** - * Table auto_increment. - * @member {vschema.IAutoIncrement|null|undefined} auto_increment - * @memberof vschema.Table - * @instance - */ - Table.prototype.auto_increment = null; - - /** - * Table columns. - * @member {Array.} columns - * @memberof vschema.Table - * @instance - */ - Table.prototype.columns = $util.emptyArray; - - /** - * Table pinned. - * @member {string} pinned - * @memberof vschema.Table - * @instance - */ - Table.prototype.pinned = ""; - - /** - * Table column_list_authoritative. - * @member {boolean} column_list_authoritative - * @memberof vschema.Table + * StopReplicationStatus before. + * @member {replicationdata.IStatus|null|undefined} before + * @memberof replicationdata.StopReplicationStatus * @instance */ - Table.prototype.column_list_authoritative = false; + StopReplicationStatus.prototype.before = null; /** - * Table source. - * @member {string} source - * @memberof vschema.Table + * StopReplicationStatus after. + * @member {replicationdata.IStatus|null|undefined} after + * @memberof replicationdata.StopReplicationStatus * @instance */ - Table.prototype.source = ""; + StopReplicationStatus.prototype.after = null; /** - * Creates a new Table instance using the specified properties. + * Creates a new StopReplicationStatus instance using the specified properties. * @function create - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static - * @param {vschema.ITable=} [properties] Properties to set - * @returns {vschema.Table} Table instance + * @param {replicationdata.IStopReplicationStatus=} [properties] Properties to set + * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus instance */ - Table.create = function create(properties) { - return new Table(properties); + StopReplicationStatus.create = function create(properties) { + return new StopReplicationStatus(properties); }; /** - * Encodes the specified Table message. Does not implicitly {@link vschema.Table.verify|verify} messages. + * Encodes the specified StopReplicationStatus message. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. * @function encode - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static - * @param {vschema.ITable} message Table message or plain object to encode + * @param {replicationdata.IStopReplicationStatus} message StopReplicationStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Table.encode = function encode(message, writer) { + StopReplicationStatus.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.type); - if (message.column_vindexes != null && message.column_vindexes.length) - for (let i = 0; i < message.column_vindexes.length; ++i) - $root.vschema.ColumnVindex.encode(message.column_vindexes[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.auto_increment != null && Object.hasOwnProperty.call(message, "auto_increment")) - $root.vschema.AutoIncrement.encode(message.auto_increment, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.columns != null && message.columns.length) - for (let i = 0; i < message.columns.length; ++i) - $root.vschema.Column.encode(message.columns[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.pinned != null && Object.hasOwnProperty.call(message, "pinned")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.pinned); - if (message.column_list_authoritative != null && Object.hasOwnProperty.call(message, "column_list_authoritative")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.column_list_authoritative); - if (message.source != null && Object.hasOwnProperty.call(message, "source")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.source); + if (message.before != null && Object.hasOwnProperty.call(message, "before")) + $root.replicationdata.Status.encode(message.before, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.after != null && Object.hasOwnProperty.call(message, "after")) + $root.replicationdata.Status.encode(message.after, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified Table message, length delimited. Does not implicitly {@link vschema.Table.verify|verify} messages. + * Encodes the specified StopReplicationStatus message, length delimited. Does not implicitly {@link replicationdata.StopReplicationStatus.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static - * @param {vschema.ITable} message Table message or plain object to encode + * @param {replicationdata.IStopReplicationStatus} message StopReplicationStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Table.encodeDelimited = function encodeDelimited(message, writer) { + StopReplicationStatus.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Table message from the specified reader or buffer. + * Decodes a StopReplicationStatus message from the specified reader or buffer. * @function decode - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.Table} Table + * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Table.decode = function decode(reader, length) { + StopReplicationStatus.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.Table(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.replicationdata.StopReplicationStatus(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.type = reader.string(); + message.before = $root.replicationdata.Status.decode(reader, reader.uint32()); break; } case 2: { - if (!(message.column_vindexes && message.column_vindexes.length)) - message.column_vindexes = []; - message.column_vindexes.push($root.vschema.ColumnVindex.decode(reader, reader.uint32())); - break; - } - case 3: { - message.auto_increment = $root.vschema.AutoIncrement.decode(reader, reader.uint32()); - break; - } - case 4: { - if (!(message.columns && message.columns.length)) - message.columns = []; - message.columns.push($root.vschema.Column.decode(reader, reader.uint32())); - break; - } - case 5: { - message.pinned = reader.string(); - break; - } - case 6: { - message.column_list_authoritative = reader.bool(); - break; - } - case 7: { - message.source = reader.string(); + message.after = $root.replicationdata.Status.decode(reader, reader.uint32()); break; } default: @@ -96401,215 +96845,156 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a Table message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationStatus message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.Table} Table + * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Table.decodeDelimited = function decodeDelimited(reader) { + StopReplicationStatus.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Table message. + * Verifies a StopReplicationStatus message. * @function verify - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Table.verify = function verify(message) { + StopReplicationStatus.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.type != null && message.hasOwnProperty("type")) - if (!$util.isString(message.type)) - return "type: string expected"; - if (message.column_vindexes != null && message.hasOwnProperty("column_vindexes")) { - if (!Array.isArray(message.column_vindexes)) - return "column_vindexes: array expected"; - for (let i = 0; i < message.column_vindexes.length; ++i) { - let error = $root.vschema.ColumnVindex.verify(message.column_vindexes[i]); - if (error) - return "column_vindexes." + error; - } - } - if (message.auto_increment != null && message.hasOwnProperty("auto_increment")) { - let error = $root.vschema.AutoIncrement.verify(message.auto_increment); + if (message.before != null && message.hasOwnProperty("before")) { + let error = $root.replicationdata.Status.verify(message.before); if (error) - return "auto_increment." + error; + return "before." + error; } - if (message.columns != null && message.hasOwnProperty("columns")) { - if (!Array.isArray(message.columns)) - return "columns: array expected"; - for (let i = 0; i < message.columns.length; ++i) { - let error = $root.vschema.Column.verify(message.columns[i]); - if (error) - return "columns." + error; - } + if (message.after != null && message.hasOwnProperty("after")) { + let error = $root.replicationdata.Status.verify(message.after); + if (error) + return "after." + error; } - if (message.pinned != null && message.hasOwnProperty("pinned")) - if (!$util.isString(message.pinned)) - return "pinned: string expected"; - if (message.column_list_authoritative != null && message.hasOwnProperty("column_list_authoritative")) - if (typeof message.column_list_authoritative !== "boolean") - return "column_list_authoritative: boolean expected"; - if (message.source != null && message.hasOwnProperty("source")) - if (!$util.isString(message.source)) - return "source: string expected"; return null; }; /** - * Creates a Table message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationStatus message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static * @param {Object.} object Plain object - * @returns {vschema.Table} Table + * @returns {replicationdata.StopReplicationStatus} StopReplicationStatus */ - Table.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.Table) + StopReplicationStatus.fromObject = function fromObject(object) { + if (object instanceof $root.replicationdata.StopReplicationStatus) return object; - let message = new $root.vschema.Table(); - if (object.type != null) - message.type = String(object.type); - if (object.column_vindexes) { - if (!Array.isArray(object.column_vindexes)) - throw TypeError(".vschema.Table.column_vindexes: array expected"); - message.column_vindexes = []; - for (let i = 0; i < object.column_vindexes.length; ++i) { - if (typeof object.column_vindexes[i] !== "object") - throw TypeError(".vschema.Table.column_vindexes: object expected"); - message.column_vindexes[i] = $root.vschema.ColumnVindex.fromObject(object.column_vindexes[i]); - } - } - if (object.auto_increment != null) { - if (typeof object.auto_increment !== "object") - throw TypeError(".vschema.Table.auto_increment: object expected"); - message.auto_increment = $root.vschema.AutoIncrement.fromObject(object.auto_increment); + let message = new $root.replicationdata.StopReplicationStatus(); + if (object.before != null) { + if (typeof object.before !== "object") + throw TypeError(".replicationdata.StopReplicationStatus.before: object expected"); + message.before = $root.replicationdata.Status.fromObject(object.before); } - if (object.columns) { - if (!Array.isArray(object.columns)) - throw TypeError(".vschema.Table.columns: array expected"); - message.columns = []; - for (let i = 0; i < object.columns.length; ++i) { - if (typeof object.columns[i] !== "object") - throw TypeError(".vschema.Table.columns: object expected"); - message.columns[i] = $root.vschema.Column.fromObject(object.columns[i]); - } + if (object.after != null) { + if (typeof object.after !== "object") + throw TypeError(".replicationdata.StopReplicationStatus.after: object expected"); + message.after = $root.replicationdata.Status.fromObject(object.after); } - if (object.pinned != null) - message.pinned = String(object.pinned); - if (object.column_list_authoritative != null) - message.column_list_authoritative = Boolean(object.column_list_authoritative); - if (object.source != null) - message.source = String(object.source); return message; }; /** - * Creates a plain object from a Table message. Also converts values to other types if specified. + * Creates a plain object from a StopReplicationStatus message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static - * @param {vschema.Table} message Table + * @param {replicationdata.StopReplicationStatus} message StopReplicationStatus * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Table.toObject = function toObject(message, options) { + StopReplicationStatus.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.column_vindexes = []; - object.columns = []; - } if (options.defaults) { - object.type = ""; - object.auto_increment = null; - object.pinned = ""; - object.column_list_authoritative = false; - object.source = ""; - } - if (message.type != null && message.hasOwnProperty("type")) - object.type = message.type; - if (message.column_vindexes && message.column_vindexes.length) { - object.column_vindexes = []; - for (let j = 0; j < message.column_vindexes.length; ++j) - object.column_vindexes[j] = $root.vschema.ColumnVindex.toObject(message.column_vindexes[j], options); - } - if (message.auto_increment != null && message.hasOwnProperty("auto_increment")) - object.auto_increment = $root.vschema.AutoIncrement.toObject(message.auto_increment, options); - if (message.columns && message.columns.length) { - object.columns = []; - for (let j = 0; j < message.columns.length; ++j) - object.columns[j] = $root.vschema.Column.toObject(message.columns[j], options); + object.before = null; + object.after = null; } - if (message.pinned != null && message.hasOwnProperty("pinned")) - object.pinned = message.pinned; - if (message.column_list_authoritative != null && message.hasOwnProperty("column_list_authoritative")) - object.column_list_authoritative = message.column_list_authoritative; - if (message.source != null && message.hasOwnProperty("source")) - object.source = message.source; + if (message.before != null && message.hasOwnProperty("before")) + object.before = $root.replicationdata.Status.toObject(message.before, options); + if (message.after != null && message.hasOwnProperty("after")) + object.after = $root.replicationdata.Status.toObject(message.after, options); return object; }; /** - * Converts this Table to JSON. + * Converts this StopReplicationStatus to JSON. * @function toJSON - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @instance * @returns {Object.} JSON object */ - Table.prototype.toJSON = function toJSON() { + StopReplicationStatus.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Table + * Gets the default type url for StopReplicationStatus * @function getTypeUrl - * @memberof vschema.Table + * @memberof replicationdata.StopReplicationStatus * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Table.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StopReplicationStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.Table"; + return typeUrlPrefix + "/replicationdata.StopReplicationStatus"; }; - return Table; + return StopReplicationStatus; })(); - vschema.ColumnVindex = (function() { + /** + * StopReplicationMode enum. + * @name replicationdata.StopReplicationMode + * @enum {number} + * @property {number} IOANDSQLTHREAD=0 IOANDSQLTHREAD value + * @property {number} IOTHREADONLY=1 IOTHREADONLY value + */ + replicationdata.StopReplicationMode = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "IOANDSQLTHREAD"] = 0; + values[valuesById[1] = "IOTHREADONLY"] = 1; + return values; + })(); + + replicationdata.PrimaryStatus = (function() { /** - * Properties of a ColumnVindex. - * @memberof vschema - * @interface IColumnVindex - * @property {string|null} [column] ColumnVindex column - * @property {string|null} [name] ColumnVindex name - * @property {Array.|null} [columns] ColumnVindex columns + * Properties of a PrimaryStatus. + * @memberof replicationdata + * @interface IPrimaryStatus + * @property {string|null} [position] PrimaryStatus position + * @property {string|null} [file_position] PrimaryStatus file_position */ /** - * Constructs a new ColumnVindex. - * @memberof vschema - * @classdesc Represents a ColumnVindex. - * @implements IColumnVindex + * Constructs a new PrimaryStatus. + * @memberof replicationdata + * @classdesc Represents a PrimaryStatus. + * @implements IPrimaryStatus * @constructor - * @param {vschema.IColumnVindex=} [properties] Properties to set + * @param {replicationdata.IPrimaryStatus=} [properties] Properties to set */ - function ColumnVindex(properties) { - this.columns = []; + function PrimaryStatus(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -96617,106 +97002,89 @@ export const vschema = $root.vschema = (() => { } /** - * ColumnVindex column. - * @member {string} column - * @memberof vschema.ColumnVindex - * @instance - */ - ColumnVindex.prototype.column = ""; - - /** - * ColumnVindex name. - * @member {string} name - * @memberof vschema.ColumnVindex + * PrimaryStatus position. + * @member {string} position + * @memberof replicationdata.PrimaryStatus * @instance */ - ColumnVindex.prototype.name = ""; + PrimaryStatus.prototype.position = ""; /** - * ColumnVindex columns. - * @member {Array.} columns - * @memberof vschema.ColumnVindex + * PrimaryStatus file_position. + * @member {string} file_position + * @memberof replicationdata.PrimaryStatus * @instance */ - ColumnVindex.prototype.columns = $util.emptyArray; + PrimaryStatus.prototype.file_position = ""; /** - * Creates a new ColumnVindex instance using the specified properties. + * Creates a new PrimaryStatus instance using the specified properties. * @function create - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static - * @param {vschema.IColumnVindex=} [properties] Properties to set - * @returns {vschema.ColumnVindex} ColumnVindex instance + * @param {replicationdata.IPrimaryStatus=} [properties] Properties to set + * @returns {replicationdata.PrimaryStatus} PrimaryStatus instance */ - ColumnVindex.create = function create(properties) { - return new ColumnVindex(properties); + PrimaryStatus.create = function create(properties) { + return new PrimaryStatus(properties); }; /** - * Encodes the specified ColumnVindex message. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. + * Encodes the specified PrimaryStatus message. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. * @function encode - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static - * @param {vschema.IColumnVindex} message ColumnVindex message or plain object to encode + * @param {replicationdata.IPrimaryStatus} message PrimaryStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ColumnVindex.encode = function encode(message, writer) { + PrimaryStatus.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.column != null && Object.hasOwnProperty.call(message, "column")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.column); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.name); - if (message.columns != null && message.columns.length) - for (let i = 0; i < message.columns.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.columns[i]); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.position); + if (message.file_position != null && Object.hasOwnProperty.call(message, "file_position")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.file_position); return writer; }; /** - * Encodes the specified ColumnVindex message, length delimited. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. + * Encodes the specified PrimaryStatus message, length delimited. Does not implicitly {@link replicationdata.PrimaryStatus.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static - * @param {vschema.IColumnVindex} message ColumnVindex message or plain object to encode + * @param {replicationdata.IPrimaryStatus} message PrimaryStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ColumnVindex.encodeDelimited = function encodeDelimited(message, writer) { + PrimaryStatus.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ColumnVindex message from the specified reader or buffer. + * Decodes a PrimaryStatus message from the specified reader or buffer. * @function decode - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.ColumnVindex} ColumnVindex + * @returns {replicationdata.PrimaryStatus} PrimaryStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ColumnVindex.decode = function decode(reader, length) { + PrimaryStatus.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.ColumnVindex(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.replicationdata.PrimaryStatus(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.column = reader.string(); + message.position = reader.string(); break; } case 2: { - message.name = reader.string(); - break; - } - case 3: { - if (!(message.columns && message.columns.length)) - message.columns = []; - message.columns.push(reader.string()); + message.file_position = reader.string(); break; } default: @@ -96728,153 +97096,151 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a ColumnVindex message from the specified reader or buffer, length delimited. + * Decodes a PrimaryStatus message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.ColumnVindex} ColumnVindex + * @returns {replicationdata.PrimaryStatus} PrimaryStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ColumnVindex.decodeDelimited = function decodeDelimited(reader) { + PrimaryStatus.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ColumnVindex message. + * Verifies a PrimaryStatus message. * @function verify - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ColumnVindex.verify = function verify(message) { + PrimaryStatus.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.column != null && message.hasOwnProperty("column")) - if (!$util.isString(message.column)) - return "column: string expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.columns != null && message.hasOwnProperty("columns")) { - if (!Array.isArray(message.columns)) - return "columns: array expected"; - for (let i = 0; i < message.columns.length; ++i) - if (!$util.isString(message.columns[i])) - return "columns: string[] expected"; - } + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.file_position != null && message.hasOwnProperty("file_position")) + if (!$util.isString(message.file_position)) + return "file_position: string expected"; return null; }; /** - * Creates a ColumnVindex message from a plain object. Also converts values to their respective internal types. + * Creates a PrimaryStatus message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static * @param {Object.} object Plain object - * @returns {vschema.ColumnVindex} ColumnVindex + * @returns {replicationdata.PrimaryStatus} PrimaryStatus */ - ColumnVindex.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.ColumnVindex) + PrimaryStatus.fromObject = function fromObject(object) { + if (object instanceof $root.replicationdata.PrimaryStatus) return object; - let message = new $root.vschema.ColumnVindex(); - if (object.column != null) - message.column = String(object.column); - if (object.name != null) - message.name = String(object.name); - if (object.columns) { - if (!Array.isArray(object.columns)) - throw TypeError(".vschema.ColumnVindex.columns: array expected"); - message.columns = []; - for (let i = 0; i < object.columns.length; ++i) - message.columns[i] = String(object.columns[i]); - } + let message = new $root.replicationdata.PrimaryStatus(); + if (object.position != null) + message.position = String(object.position); + if (object.file_position != null) + message.file_position = String(object.file_position); return message; }; /** - * Creates a plain object from a ColumnVindex message. Also converts values to other types if specified. + * Creates a plain object from a PrimaryStatus message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static - * @param {vschema.ColumnVindex} message ColumnVindex + * @param {replicationdata.PrimaryStatus} message PrimaryStatus * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ColumnVindex.toObject = function toObject(message, options) { + PrimaryStatus.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.columns = []; if (options.defaults) { - object.column = ""; - object.name = ""; - } - if (message.column != null && message.hasOwnProperty("column")) - object.column = message.column; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.columns && message.columns.length) { - object.columns = []; - for (let j = 0; j < message.columns.length; ++j) - object.columns[j] = message.columns[j]; + object.position = ""; + object.file_position = ""; } + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.file_position != null && message.hasOwnProperty("file_position")) + object.file_position = message.file_position; return object; }; /** - * Converts this ColumnVindex to JSON. + * Converts this PrimaryStatus to JSON. * @function toJSON - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @instance * @returns {Object.} JSON object */ - ColumnVindex.prototype.toJSON = function toJSON() { + PrimaryStatus.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ColumnVindex + * Gets the default type url for PrimaryStatus * @function getTypeUrl - * @memberof vschema.ColumnVindex + * @memberof replicationdata.PrimaryStatus * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ColumnVindex.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PrimaryStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.ColumnVindex"; + return typeUrlPrefix + "/replicationdata.PrimaryStatus"; }; - return ColumnVindex; + return PrimaryStatus; })(); - vschema.AutoIncrement = (function() { + replicationdata.FullStatus = (function() { /** - * Properties of an AutoIncrement. - * @memberof vschema - * @interface IAutoIncrement - * @property {string|null} [column] AutoIncrement column - * @property {string|null} [sequence] AutoIncrement sequence + * Properties of a FullStatus. + * @memberof replicationdata + * @interface IFullStatus + * @property {number|null} [server_id] FullStatus server_id + * @property {string|null} [server_uuid] FullStatus server_uuid + * @property {replicationdata.IStatus|null} [replication_status] FullStatus replication_status + * @property {replicationdata.IPrimaryStatus|null} [primary_status] FullStatus primary_status + * @property {string|null} [gtid_purged] FullStatus gtid_purged + * @property {string|null} [version] FullStatus version + * @property {string|null} [version_comment] FullStatus version_comment + * @property {boolean|null} [read_only] FullStatus read_only + * @property {string|null} [gtid_mode] FullStatus gtid_mode + * @property {string|null} [binlog_format] FullStatus binlog_format + * @property {string|null} [binlog_row_image] FullStatus binlog_row_image + * @property {boolean|null} [log_bin_enabled] FullStatus log_bin_enabled + * @property {boolean|null} [log_replica_updates] FullStatus log_replica_updates + * @property {boolean|null} [semi_sync_primary_enabled] FullStatus semi_sync_primary_enabled + * @property {boolean|null} [semi_sync_replica_enabled] FullStatus semi_sync_replica_enabled + * @property {boolean|null} [semi_sync_primary_status] FullStatus semi_sync_primary_status + * @property {boolean|null} [semi_sync_replica_status] FullStatus semi_sync_replica_status + * @property {number|null} [semi_sync_primary_clients] FullStatus semi_sync_primary_clients + * @property {number|Long|null} [semi_sync_primary_timeout] FullStatus semi_sync_primary_timeout + * @property {number|null} [semi_sync_wait_for_replica_count] FullStatus semi_sync_wait_for_replica_count + * @property {boolean|null} [super_read_only] FullStatus super_read_only */ /** - * Constructs a new AutoIncrement. - * @memberof vschema - * @classdesc Represents an AutoIncrement. - * @implements IAutoIncrement + * Constructs a new FullStatus. + * @memberof replicationdata + * @classdesc Represents a FullStatus. + * @implements IFullStatus * @constructor - * @param {vschema.IAutoIncrement=} [properties] Properties to set + * @param {replicationdata.IFullStatus=} [properties] Properties to set */ - function AutoIncrement(properties) { + function FullStatus(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -96882,316 +97248,355 @@ export const vschema = $root.vschema = (() => { } /** - * AutoIncrement column. - * @member {string} column - * @memberof vschema.AutoIncrement + * FullStatus server_id. + * @member {number} server_id + * @memberof replicationdata.FullStatus * @instance */ - AutoIncrement.prototype.column = ""; + FullStatus.prototype.server_id = 0; /** - * AutoIncrement sequence. - * @member {string} sequence - * @memberof vschema.AutoIncrement + * FullStatus server_uuid. + * @member {string} server_uuid + * @memberof replicationdata.FullStatus * @instance */ - AutoIncrement.prototype.sequence = ""; + FullStatus.prototype.server_uuid = ""; /** - * Creates a new AutoIncrement instance using the specified properties. - * @function create - * @memberof vschema.AutoIncrement - * @static - * @param {vschema.IAutoIncrement=} [properties] Properties to set - * @returns {vschema.AutoIncrement} AutoIncrement instance + * FullStatus replication_status. + * @member {replicationdata.IStatus|null|undefined} replication_status + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.create = function create(properties) { - return new AutoIncrement(properties); - }; + FullStatus.prototype.replication_status = null; /** - * Encodes the specified AutoIncrement message. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. - * @function encode - * @memberof vschema.AutoIncrement - * @static - * @param {vschema.IAutoIncrement} message AutoIncrement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * FullStatus primary_status. + * @member {replicationdata.IPrimaryStatus|null|undefined} primary_status + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.column != null && Object.hasOwnProperty.call(message, "column")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.column); - if (message.sequence != null && Object.hasOwnProperty.call(message, "sequence")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.sequence); - return writer; - }; + FullStatus.prototype.primary_status = null; /** - * Encodes the specified AutoIncrement message, length delimited. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. - * @function encodeDelimited - * @memberof vschema.AutoIncrement - * @static - * @param {vschema.IAutoIncrement} message AutoIncrement message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * FullStatus gtid_purged. + * @member {string} gtid_purged + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + FullStatus.prototype.gtid_purged = ""; /** - * Decodes an AutoIncrement message from the specified reader or buffer. - * @function decode - * @memberof vschema.AutoIncrement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vschema.AutoIncrement} AutoIncrement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * FullStatus version. + * @member {string} version + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.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.AutoIncrement(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.column = reader.string(); - break; - } - case 2: { - message.sequence = reader.string(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; + FullStatus.prototype.version = ""; /** - * Decodes an AutoIncrement message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vschema.AutoIncrement - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.AutoIncrement} AutoIncrement - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * FullStatus version_comment. + * @member {string} version_comment + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + FullStatus.prototype.version_comment = ""; /** - * Verifies an AutoIncrement message. - * @function verify - * @memberof vschema.AutoIncrement - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not + * FullStatus read_only. + * @member {boolean} read_only + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.column != null && message.hasOwnProperty("column")) - if (!$util.isString(message.column)) - return "column: string expected"; - if (message.sequence != null && message.hasOwnProperty("sequence")) - if (!$util.isString(message.sequence)) - return "sequence: string expected"; - return null; - }; + FullStatus.prototype.read_only = false; /** - * Creates an AutoIncrement message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vschema.AutoIncrement - * @static - * @param {Object.} object Plain object - * @returns {vschema.AutoIncrement} AutoIncrement + * FullStatus gtid_mode. + * @member {string} gtid_mode + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.AutoIncrement) - return object; - let message = new $root.vschema.AutoIncrement(); - if (object.column != null) - message.column = String(object.column); - if (object.sequence != null) - message.sequence = String(object.sequence); - return message; - }; + FullStatus.prototype.gtid_mode = ""; /** - * Creates a plain object from an AutoIncrement message. Also converts values to other types if specified. - * @function toObject - * @memberof vschema.AutoIncrement - * @static - * @param {vschema.AutoIncrement} message AutoIncrement - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object + * FullStatus binlog_format. + * @member {string} binlog_format + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.column = ""; - object.sequence = ""; - } - if (message.column != null && message.hasOwnProperty("column")) - object.column = message.column; - if (message.sequence != null && message.hasOwnProperty("sequence")) - object.sequence = message.sequence; - return object; - }; + FullStatus.prototype.binlog_format = ""; /** - * Converts this AutoIncrement to JSON. - * @function toJSON - * @memberof vschema.AutoIncrement + * FullStatus binlog_row_image. + * @member {string} binlog_row_image + * @memberof replicationdata.FullStatus * @instance - * @returns {Object.} JSON object */ - AutoIncrement.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + FullStatus.prototype.binlog_row_image = ""; /** - * Gets the default type url for AutoIncrement - * @function getTypeUrl - * @memberof vschema.AutoIncrement - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url + * FullStatus log_bin_enabled. + * @member {boolean} log_bin_enabled + * @memberof replicationdata.FullStatus + * @instance */ - AutoIncrement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vschema.AutoIncrement"; - }; + FullStatus.prototype.log_bin_enabled = false; - return AutoIncrement; - })(); + /** + * FullStatus log_replica_updates. + * @member {boolean} log_replica_updates + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.log_replica_updates = false; - vschema.Column = (function() { + /** + * FullStatus semi_sync_primary_enabled. + * @member {boolean} semi_sync_primary_enabled + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.semi_sync_primary_enabled = false; /** - * Properties of a Column. - * @memberof vschema - * @interface IColumn - * @property {string|null} [name] Column name - * @property {query.Type|null} [type] Column type + * FullStatus semi_sync_replica_enabled. + * @member {boolean} semi_sync_replica_enabled + * @memberof replicationdata.FullStatus + * @instance */ + FullStatus.prototype.semi_sync_replica_enabled = false; /** - * Constructs a new Column. - * @memberof vschema - * @classdesc Represents a Column. - * @implements IColumn - * @constructor - * @param {vschema.IColumn=} [properties] Properties to set + * FullStatus semi_sync_primary_status. + * @member {boolean} semi_sync_primary_status + * @memberof replicationdata.FullStatus + * @instance */ - function Column(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]]; - } + FullStatus.prototype.semi_sync_primary_status = false; /** - * Column name. - * @member {string} name - * @memberof vschema.Column + * FullStatus semi_sync_replica_status. + * @member {boolean} semi_sync_replica_status + * @memberof replicationdata.FullStatus * @instance */ - Column.prototype.name = ""; + FullStatus.prototype.semi_sync_replica_status = false; /** - * Column type. - * @member {query.Type} type - * @memberof vschema.Column + * FullStatus semi_sync_primary_clients. + * @member {number} semi_sync_primary_clients + * @memberof replicationdata.FullStatus * @instance */ - Column.prototype.type = 0; + FullStatus.prototype.semi_sync_primary_clients = 0; /** - * Creates a new Column instance using the specified properties. + * FullStatus semi_sync_primary_timeout. + * @member {number|Long} semi_sync_primary_timeout + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.semi_sync_primary_timeout = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + + /** + * FullStatus semi_sync_wait_for_replica_count. + * @member {number} semi_sync_wait_for_replica_count + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.semi_sync_wait_for_replica_count = 0; + + /** + * FullStatus super_read_only. + * @member {boolean} super_read_only + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.super_read_only = false; + + /** + * Creates a new FullStatus instance using the specified properties. * @function create - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static - * @param {vschema.IColumn=} [properties] Properties to set - * @returns {vschema.Column} Column instance + * @param {replicationdata.IFullStatus=} [properties] Properties to set + * @returns {replicationdata.FullStatus} FullStatus instance */ - Column.create = function create(properties) { - return new Column(properties); + FullStatus.create = function create(properties) { + return new FullStatus(properties); }; /** - * Encodes the specified Column message. Does not implicitly {@link vschema.Column.verify|verify} messages. + * Encodes the specified FullStatus message. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. * @function encode - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static - * @param {vschema.IColumn} message Column message or plain object to encode + * @param {replicationdata.IFullStatus} message FullStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Column.encode = function encode(message, writer) { + FullStatus.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.type); + if (message.server_id != null && Object.hasOwnProperty.call(message, "server_id")) + writer.uint32(/* id 1, wireType 0 =*/8).uint32(message.server_id); + if (message.server_uuid != null && Object.hasOwnProperty.call(message, "server_uuid")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.server_uuid); + if (message.replication_status != null && Object.hasOwnProperty.call(message, "replication_status")) + $root.replicationdata.Status.encode(message.replication_status, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.primary_status != null && Object.hasOwnProperty.call(message, "primary_status")) + $root.replicationdata.PrimaryStatus.encode(message.primary_status, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.gtid_purged != null && Object.hasOwnProperty.call(message, "gtid_purged")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.gtid_purged); + if (message.version != null && Object.hasOwnProperty.call(message, "version")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.version); + if (message.version_comment != null && Object.hasOwnProperty.call(message, "version_comment")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.version_comment); + if (message.read_only != null && Object.hasOwnProperty.call(message, "read_only")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.read_only); + if (message.gtid_mode != null && Object.hasOwnProperty.call(message, "gtid_mode")) + writer.uint32(/* id 9, wireType 2 =*/74).string(message.gtid_mode); + if (message.binlog_format != null && Object.hasOwnProperty.call(message, "binlog_format")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.binlog_format); + if (message.binlog_row_image != null && Object.hasOwnProperty.call(message, "binlog_row_image")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.binlog_row_image); + if (message.log_bin_enabled != null && Object.hasOwnProperty.call(message, "log_bin_enabled")) + writer.uint32(/* id 12, wireType 0 =*/96).bool(message.log_bin_enabled); + if (message.log_replica_updates != null && Object.hasOwnProperty.call(message, "log_replica_updates")) + writer.uint32(/* id 13, wireType 0 =*/104).bool(message.log_replica_updates); + if (message.semi_sync_primary_enabled != null && Object.hasOwnProperty.call(message, "semi_sync_primary_enabled")) + writer.uint32(/* id 14, wireType 0 =*/112).bool(message.semi_sync_primary_enabled); + if (message.semi_sync_replica_enabled != null && Object.hasOwnProperty.call(message, "semi_sync_replica_enabled")) + writer.uint32(/* id 15, wireType 0 =*/120).bool(message.semi_sync_replica_enabled); + if (message.semi_sync_primary_status != null && Object.hasOwnProperty.call(message, "semi_sync_primary_status")) + writer.uint32(/* id 16, wireType 0 =*/128).bool(message.semi_sync_primary_status); + if (message.semi_sync_replica_status != null && Object.hasOwnProperty.call(message, "semi_sync_replica_status")) + writer.uint32(/* id 17, wireType 0 =*/136).bool(message.semi_sync_replica_status); + if (message.semi_sync_primary_clients != null && Object.hasOwnProperty.call(message, "semi_sync_primary_clients")) + writer.uint32(/* id 18, wireType 0 =*/144).uint32(message.semi_sync_primary_clients); + if (message.semi_sync_primary_timeout != null && Object.hasOwnProperty.call(message, "semi_sync_primary_timeout")) + writer.uint32(/* id 19, wireType 0 =*/152).uint64(message.semi_sync_primary_timeout); + if (message.semi_sync_wait_for_replica_count != null && Object.hasOwnProperty.call(message, "semi_sync_wait_for_replica_count")) + writer.uint32(/* id 20, wireType 0 =*/160).uint32(message.semi_sync_wait_for_replica_count); + if (message.super_read_only != null && Object.hasOwnProperty.call(message, "super_read_only")) + writer.uint32(/* id 21, wireType 0 =*/168).bool(message.super_read_only); return writer; }; /** - * Encodes the specified Column message, length delimited. Does not implicitly {@link vschema.Column.verify|verify} messages. + * Encodes the specified FullStatus message, length delimited. Does not implicitly {@link replicationdata.FullStatus.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static - * @param {vschema.IColumn} message Column message or plain object to encode + * @param {replicationdata.IFullStatus} message FullStatus message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Column.encodeDelimited = function encodeDelimited(message, writer) { + FullStatus.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Column message from the specified reader or buffer. + * Decodes a FullStatus message from the specified reader or buffer. * @function decode - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.Column} Column + * @returns {replicationdata.FullStatus} FullStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Column.decode = function decode(reader, length) { + FullStatus.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.Column(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.replicationdata.FullStatus(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.server_id = reader.uint32(); break; } case 2: { - message.type = reader.int32(); + message.server_uuid = reader.string(); + break; + } + case 3: { + message.replication_status = $root.replicationdata.Status.decode(reader, reader.uint32()); + break; + } + case 4: { + message.primary_status = $root.replicationdata.PrimaryStatus.decode(reader, reader.uint32()); + break; + } + case 5: { + message.gtid_purged = reader.string(); + break; + } + case 6: { + message.version = reader.string(); + break; + } + case 7: { + message.version_comment = reader.string(); + break; + } + case 8: { + message.read_only = reader.bool(); + break; + } + case 9: { + message.gtid_mode = reader.string(); + break; + } + case 10: { + message.binlog_format = reader.string(); + break; + } + case 11: { + message.binlog_row_image = reader.string(); + break; + } + case 12: { + message.log_bin_enabled = reader.bool(); + break; + } + case 13: { + message.log_replica_updates = reader.bool(); + break; + } + case 14: { + message.semi_sync_primary_enabled = reader.bool(); + break; + } + case 15: { + message.semi_sync_replica_enabled = reader.bool(); + break; + } + case 16: { + message.semi_sync_primary_status = reader.bool(); + break; + } + case 17: { + message.semi_sync_replica_status = reader.bool(); + break; + } + case 18: { + message.semi_sync_primary_clients = reader.uint32(); + break; + } + case 19: { + message.semi_sync_primary_timeout = reader.uint64(); + break; + } + case 20: { + message.semi_sync_wait_for_replica_count = reader.uint32(); + break; + } + case 21: { + message.super_read_only = reader.bool(); break; } default: @@ -97203,318 +97608,320 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a Column message from the specified reader or buffer, length delimited. + * Decodes a FullStatus message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.Column} Column + * @returns {replicationdata.FullStatus} FullStatus * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Column.decodeDelimited = function decodeDelimited(reader) { + FullStatus.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Column message. + * Verifies a FullStatus message. * @function verify - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Column.verify = function verify(message) { + FullStatus.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "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 Column message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vschema.Column - * @static - * @param {Object.} object Plain object - * @returns {vschema.Column} Column - */ - Column.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.Column) - return object; - let message = new $root.vschema.Column(); - if (object.name != null) - message.name = String(object.name); - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; - } - break; - case "NULL_TYPE": - case 0: - message.type = 0; - break; - case "INT8": - case 257: - message.type = 257; - break; - case "UINT8": - case 770: - message.type = 770; - break; - case "INT16": - case 259: - message.type = 259; - break; - case "UINT16": - case 772: - message.type = 772; - break; - case "INT24": - case 261: - message.type = 261; - break; - case "UINT24": - case 774: - message.type = 774; - break; - case "INT32": - case 263: - message.type = 263; - break; - case "UINT32": - case 776: - message.type = 776; - break; - case "INT64": - case 265: - message.type = 265; - break; - case "UINT64": - case 778: - message.type = 778; - break; - case "FLOAT32": - case 1035: - message.type = 1035; - break; - case "FLOAT64": - case 1036: - message.type = 1036; - break; - case "TIMESTAMP": - case 2061: - message.type = 2061; - break; - case "DATE": - case 2062: - message.type = 2062; - break; - case "TIME": - case 2063: - message.type = 2063; - break; - case "DATETIME": - case 2064: - message.type = 2064; - break; - case "YEAR": - case 785: - message.type = 785; - break; - case "DECIMAL": - case 18: - message.type = 18; - break; - case "TEXT": - case 6163: - message.type = 6163; - break; - case "BLOB": - case 10260: - message.type = 10260; - break; - case "VARCHAR": - case 6165: - message.type = 6165; - break; - case "VARBINARY": - case 10262: - message.type = 10262; - break; - case "CHAR": - case 6167: - message.type = 6167; - break; - case "BINARY": - case 10264: - message.type = 10264; - break; - case "BIT": - case 2073: - message.type = 2073; - break; - case "ENUM": - case 2074: - message.type = 2074; - break; - case "SET": - case 2075: - message.type = 2075; - break; - case "TUPLE": - case 28: - message.type = 28; - break; - case "GEOMETRY": - case 2077: - message.type = 2077; - break; - case "JSON": - case 2078: - message.type = 2078; - break; - case "EXPRESSION": - case 31: - message.type = 31; - break; - case "HEXNUM": - case 4128: - message.type = 4128; - break; - case "HEXVAL": - case 4129: - message.type = 4129; - break; - case "BITNUM": - case 4130: - message.type = 4130; - break; + if (message.server_id != null && message.hasOwnProperty("server_id")) + if (!$util.isInteger(message.server_id)) + return "server_id: integer expected"; + if (message.server_uuid != null && message.hasOwnProperty("server_uuid")) + if (!$util.isString(message.server_uuid)) + return "server_uuid: string expected"; + if (message.replication_status != null && message.hasOwnProperty("replication_status")) { + let error = $root.replicationdata.Status.verify(message.replication_status); + if (error) + return "replication_status." + error; + } + if (message.primary_status != null && message.hasOwnProperty("primary_status")) { + let error = $root.replicationdata.PrimaryStatus.verify(message.primary_status); + if (error) + return "primary_status." + error; + } + if (message.gtid_purged != null && message.hasOwnProperty("gtid_purged")) + if (!$util.isString(message.gtid_purged)) + return "gtid_purged: string expected"; + if (message.version != null && message.hasOwnProperty("version")) + if (!$util.isString(message.version)) + return "version: string expected"; + if (message.version_comment != null && message.hasOwnProperty("version_comment")) + if (!$util.isString(message.version_comment)) + return "version_comment: string expected"; + if (message.read_only != null && message.hasOwnProperty("read_only")) + if (typeof message.read_only !== "boolean") + return "read_only: boolean expected"; + if (message.gtid_mode != null && message.hasOwnProperty("gtid_mode")) + if (!$util.isString(message.gtid_mode)) + return "gtid_mode: string expected"; + if (message.binlog_format != null && message.hasOwnProperty("binlog_format")) + if (!$util.isString(message.binlog_format)) + return "binlog_format: string expected"; + if (message.binlog_row_image != null && message.hasOwnProperty("binlog_row_image")) + if (!$util.isString(message.binlog_row_image)) + return "binlog_row_image: string expected"; + if (message.log_bin_enabled != null && message.hasOwnProperty("log_bin_enabled")) + if (typeof message.log_bin_enabled !== "boolean") + return "log_bin_enabled: boolean expected"; + if (message.log_replica_updates != null && message.hasOwnProperty("log_replica_updates")) + if (typeof message.log_replica_updates !== "boolean") + return "log_replica_updates: boolean expected"; + if (message.semi_sync_primary_enabled != null && message.hasOwnProperty("semi_sync_primary_enabled")) + if (typeof message.semi_sync_primary_enabled !== "boolean") + return "semi_sync_primary_enabled: boolean expected"; + if (message.semi_sync_replica_enabled != null && message.hasOwnProperty("semi_sync_replica_enabled")) + if (typeof message.semi_sync_replica_enabled !== "boolean") + return "semi_sync_replica_enabled: boolean expected"; + if (message.semi_sync_primary_status != null && message.hasOwnProperty("semi_sync_primary_status")) + if (typeof message.semi_sync_primary_status !== "boolean") + return "semi_sync_primary_status: boolean expected"; + if (message.semi_sync_replica_status != null && message.hasOwnProperty("semi_sync_replica_status")) + if (typeof message.semi_sync_replica_status !== "boolean") + return "semi_sync_replica_status: boolean expected"; + if (message.semi_sync_primary_clients != null && message.hasOwnProperty("semi_sync_primary_clients")) + if (!$util.isInteger(message.semi_sync_primary_clients)) + return "semi_sync_primary_clients: integer expected"; + if (message.semi_sync_primary_timeout != null && message.hasOwnProperty("semi_sync_primary_timeout")) + if (!$util.isInteger(message.semi_sync_primary_timeout) && !(message.semi_sync_primary_timeout && $util.isInteger(message.semi_sync_primary_timeout.low) && $util.isInteger(message.semi_sync_primary_timeout.high))) + return "semi_sync_primary_timeout: integer|Long expected"; + if (message.semi_sync_wait_for_replica_count != null && message.hasOwnProperty("semi_sync_wait_for_replica_count")) + if (!$util.isInteger(message.semi_sync_wait_for_replica_count)) + return "semi_sync_wait_for_replica_count: integer expected"; + if (message.super_read_only != null && message.hasOwnProperty("super_read_only")) + if (typeof message.super_read_only !== "boolean") + return "super_read_only: boolean expected"; + return null; + }; + + /** + * Creates a FullStatus message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof replicationdata.FullStatus + * @static + * @param {Object.} object Plain object + * @returns {replicationdata.FullStatus} FullStatus + */ + FullStatus.fromObject = function fromObject(object) { + if (object instanceof $root.replicationdata.FullStatus) + return object; + let message = new $root.replicationdata.FullStatus(); + if (object.server_id != null) + message.server_id = object.server_id >>> 0; + if (object.server_uuid != null) + message.server_uuid = String(object.server_uuid); + if (object.replication_status != null) { + if (typeof object.replication_status !== "object") + throw TypeError(".replicationdata.FullStatus.replication_status: object expected"); + message.replication_status = $root.replicationdata.Status.fromObject(object.replication_status); + } + if (object.primary_status != null) { + if (typeof object.primary_status !== "object") + throw TypeError(".replicationdata.FullStatus.primary_status: object expected"); + message.primary_status = $root.replicationdata.PrimaryStatus.fromObject(object.primary_status); } + if (object.gtid_purged != null) + message.gtid_purged = String(object.gtid_purged); + if (object.version != null) + message.version = String(object.version); + if (object.version_comment != null) + message.version_comment = String(object.version_comment); + if (object.read_only != null) + message.read_only = Boolean(object.read_only); + if (object.gtid_mode != null) + message.gtid_mode = String(object.gtid_mode); + if (object.binlog_format != null) + message.binlog_format = String(object.binlog_format); + if (object.binlog_row_image != null) + message.binlog_row_image = String(object.binlog_row_image); + if (object.log_bin_enabled != null) + message.log_bin_enabled = Boolean(object.log_bin_enabled); + if (object.log_replica_updates != null) + message.log_replica_updates = Boolean(object.log_replica_updates); + if (object.semi_sync_primary_enabled != null) + message.semi_sync_primary_enabled = Boolean(object.semi_sync_primary_enabled); + if (object.semi_sync_replica_enabled != null) + message.semi_sync_replica_enabled = Boolean(object.semi_sync_replica_enabled); + if (object.semi_sync_primary_status != null) + message.semi_sync_primary_status = Boolean(object.semi_sync_primary_status); + if (object.semi_sync_replica_status != null) + message.semi_sync_replica_status = Boolean(object.semi_sync_replica_status); + if (object.semi_sync_primary_clients != null) + message.semi_sync_primary_clients = object.semi_sync_primary_clients >>> 0; + if (object.semi_sync_primary_timeout != null) + if ($util.Long) + (message.semi_sync_primary_timeout = $util.Long.fromValue(object.semi_sync_primary_timeout)).unsigned = true; + else if (typeof object.semi_sync_primary_timeout === "string") + message.semi_sync_primary_timeout = parseInt(object.semi_sync_primary_timeout, 10); + else if (typeof object.semi_sync_primary_timeout === "number") + message.semi_sync_primary_timeout = object.semi_sync_primary_timeout; + else if (typeof object.semi_sync_primary_timeout === "object") + message.semi_sync_primary_timeout = new $util.LongBits(object.semi_sync_primary_timeout.low >>> 0, object.semi_sync_primary_timeout.high >>> 0).toNumber(true); + if (object.semi_sync_wait_for_replica_count != null) + message.semi_sync_wait_for_replica_count = object.semi_sync_wait_for_replica_count >>> 0; + if (object.super_read_only != null) + message.super_read_only = Boolean(object.super_read_only); return message; }; /** - * Creates a plain object from a Column message. Also converts values to other types if specified. + * Creates a plain object from a FullStatus message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static - * @param {vschema.Column} message Column + * @param {replicationdata.FullStatus} message FullStatus * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Column.toObject = function toObject(message, options) { + FullStatus.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.name = ""; - object.type = options.enums === String ? "NULL_TYPE" : 0; + object.server_id = 0; + object.server_uuid = ""; + object.replication_status = null; + object.primary_status = null; + object.gtid_purged = ""; + object.version = ""; + object.version_comment = ""; + object.read_only = false; + object.gtid_mode = ""; + object.binlog_format = ""; + object.binlog_row_image = ""; + object.log_bin_enabled = false; + object.log_replica_updates = false; + object.semi_sync_primary_enabled = false; + object.semi_sync_replica_enabled = false; + object.semi_sync_primary_status = false; + object.semi_sync_replica_status = false; + object.semi_sync_primary_clients = 0; + if ($util.Long) { + let long = new $util.Long(0, 0, true); + object.semi_sync_primary_timeout = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.semi_sync_primary_timeout = options.longs === String ? "0" : 0; + object.semi_sync_wait_for_replica_count = 0; + object.super_read_only = false; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; + if (message.server_id != null && message.hasOwnProperty("server_id")) + object.server_id = message.server_id; + if (message.server_uuid != null && message.hasOwnProperty("server_uuid")) + object.server_uuid = message.server_uuid; + if (message.replication_status != null && message.hasOwnProperty("replication_status")) + object.replication_status = $root.replicationdata.Status.toObject(message.replication_status, options); + if (message.primary_status != null && message.hasOwnProperty("primary_status")) + object.primary_status = $root.replicationdata.PrimaryStatus.toObject(message.primary_status, options); + if (message.gtid_purged != null && message.hasOwnProperty("gtid_purged")) + object.gtid_purged = message.gtid_purged; + if (message.version != null && message.hasOwnProperty("version")) + object.version = message.version; + if (message.version_comment != null && message.hasOwnProperty("version_comment")) + object.version_comment = message.version_comment; + if (message.read_only != null && message.hasOwnProperty("read_only")) + object.read_only = message.read_only; + if (message.gtid_mode != null && message.hasOwnProperty("gtid_mode")) + object.gtid_mode = message.gtid_mode; + if (message.binlog_format != null && message.hasOwnProperty("binlog_format")) + object.binlog_format = message.binlog_format; + if (message.binlog_row_image != null && message.hasOwnProperty("binlog_row_image")) + object.binlog_row_image = message.binlog_row_image; + if (message.log_bin_enabled != null && message.hasOwnProperty("log_bin_enabled")) + object.log_bin_enabled = message.log_bin_enabled; + if (message.log_replica_updates != null && message.hasOwnProperty("log_replica_updates")) + object.log_replica_updates = message.log_replica_updates; + if (message.semi_sync_primary_enabled != null && message.hasOwnProperty("semi_sync_primary_enabled")) + object.semi_sync_primary_enabled = message.semi_sync_primary_enabled; + if (message.semi_sync_replica_enabled != null && message.hasOwnProperty("semi_sync_replica_enabled")) + object.semi_sync_replica_enabled = message.semi_sync_replica_enabled; + if (message.semi_sync_primary_status != null && message.hasOwnProperty("semi_sync_primary_status")) + object.semi_sync_primary_status = message.semi_sync_primary_status; + if (message.semi_sync_replica_status != null && message.hasOwnProperty("semi_sync_replica_status")) + object.semi_sync_replica_status = message.semi_sync_replica_status; + if (message.semi_sync_primary_clients != null && message.hasOwnProperty("semi_sync_primary_clients")) + object.semi_sync_primary_clients = message.semi_sync_primary_clients; + if (message.semi_sync_primary_timeout != null && message.hasOwnProperty("semi_sync_primary_timeout")) + if (typeof message.semi_sync_primary_timeout === "number") + object.semi_sync_primary_timeout = options.longs === String ? String(message.semi_sync_primary_timeout) : message.semi_sync_primary_timeout; + else + object.semi_sync_primary_timeout = options.longs === String ? $util.Long.prototype.toString.call(message.semi_sync_primary_timeout) : options.longs === Number ? new $util.LongBits(message.semi_sync_primary_timeout.low >>> 0, message.semi_sync_primary_timeout.high >>> 0).toNumber(true) : message.semi_sync_primary_timeout; + if (message.semi_sync_wait_for_replica_count != null && message.hasOwnProperty("semi_sync_wait_for_replica_count")) + object.semi_sync_wait_for_replica_count = message.semi_sync_wait_for_replica_count; + if (message.super_read_only != null && message.hasOwnProperty("super_read_only")) + object.super_read_only = message.super_read_only; return object; }; /** - * Converts this Column to JSON. + * Converts this FullStatus to JSON. * @function toJSON - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @instance * @returns {Object.} JSON object */ - Column.prototype.toJSON = function toJSON() { + FullStatus.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Column + * Gets the default type url for FullStatus * @function getTypeUrl - * @memberof vschema.Column + * @memberof replicationdata.FullStatus * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Column.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + FullStatus.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.Column"; + return typeUrlPrefix + "/replicationdata.FullStatus"; }; - return Column; + return FullStatus; })(); - vschema.SrvVSchema = (function() { + return replicationdata; +})(); + +export const vschema = $root.vschema = (() => { + + /** + * Namespace vschema. + * @exports vschema + * @namespace + */ + const vschema = {}; + + vschema.RoutingRules = (function() { /** - * Properties of a SrvVSchema. + * Properties of a RoutingRules. * @memberof vschema - * @interface ISrvVSchema - * @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 + * @interface IRoutingRules + * @property {Array.|null} [rules] RoutingRules rules */ /** - * Constructs a new SrvVSchema. + * Constructs a new RoutingRules. * @memberof vschema - * @classdesc Represents a SrvVSchema. - * @implements ISrvVSchema + * @classdesc Represents a RoutingRules. + * @implements IRoutingRules * @constructor - * @param {vschema.ISrvVSchema=} [properties] Properties to set + * @param {vschema.IRoutingRules=} [properties] Properties to set */ - function SrvVSchema(properties) { - this.keyspaces = {}; + function RoutingRules(properties) { + this.rules = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -97522,125 +97929,78 @@ export const vschema = $root.vschema = (() => { } /** - * SrvVSchema keyspaces. - * @member {Object.} keyspaces - * @memberof vschema.SrvVSchema - * @instance - */ - SrvVSchema.prototype.keyspaces = $util.emptyObject; - - /** - * SrvVSchema routing_rules. - * @member {vschema.IRoutingRules|null|undefined} routing_rules - * @memberof vschema.SrvVSchema - * @instance - */ - SrvVSchema.prototype.routing_rules = null; - - /** - * SrvVSchema shard_routing_rules. - * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules - * @memberof vschema.SrvVSchema + * RoutingRules rules. + * @member {Array.} rules + * @memberof vschema.RoutingRules * @instance */ - SrvVSchema.prototype.shard_routing_rules = null; + RoutingRules.prototype.rules = $util.emptyArray; /** - * Creates a new SrvVSchema instance using the specified properties. + * Creates a new RoutingRules instance using the specified properties. * @function create - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static - * @param {vschema.ISrvVSchema=} [properties] Properties to set - * @returns {vschema.SrvVSchema} SrvVSchema instance + * @param {vschema.IRoutingRules=} [properties] Properties to set + * @returns {vschema.RoutingRules} RoutingRules instance */ - SrvVSchema.create = function create(properties) { - return new SrvVSchema(properties); + RoutingRules.create = function create(properties) { + return new RoutingRules(properties); }; /** - * Encodes the specified SrvVSchema message. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. + * Encodes the specified RoutingRules message. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. * @function encode - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static - * @param {vschema.ISrvVSchema} message SrvVSchema message or plain object to encode + * @param {vschema.IRoutingRules} message RoutingRules message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SrvVSchema.encode = function encode(message, writer) { + RoutingRules.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspaces != null && Object.hasOwnProperty.call(message, "keyspaces")) - for (let keys = Object.keys(message.keyspaces), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vschema.Keyspace.encode(message.keyspaces[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) - $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.rules != null && message.rules.length) + for (let i = 0; i < message.rules.length; ++i) + $root.vschema.RoutingRule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SrvVSchema message, length delimited. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. + * Encodes the specified RoutingRules message, length delimited. Does not implicitly {@link vschema.RoutingRules.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static - * @param {vschema.ISrvVSchema} message SrvVSchema message or plain object to encode + * @param {vschema.IRoutingRules} message RoutingRules message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SrvVSchema.encodeDelimited = function encodeDelimited(message, writer) { + RoutingRules.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SrvVSchema message from the specified reader or buffer. + * Decodes a RoutingRules message from the specified reader or buffer. * @function decode - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.SrvVSchema} SrvVSchema + * @returns {vschema.RoutingRules} RoutingRules * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SrvVSchema.decode = function decode(reader, length) { + RoutingRules.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.SrvVSchema(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.RoutingRules(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.keyspaces === $util.emptyObject) - message.keyspaces = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vschema.Keyspace.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.keyspaces[key] = value; - break; - } - case 2: { - message.routing_rules = $root.vschema.RoutingRules.decode(reader, reader.uint32()); - break; - } - case 3: { - message.shard_routing_rules = $root.vschema.ShardRoutingRules.decode(reader, reader.uint32()); + if (!(message.rules && message.rules.length)) + message.rules = []; + message.rules.push($root.vschema.RoutingRule.decode(reader, reader.uint32())); break; } default: @@ -97652,170 +98012,141 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a SrvVSchema message from the specified reader or buffer, length delimited. + * Decodes a RoutingRules message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.SrvVSchema} SrvVSchema + * @returns {vschema.RoutingRules} RoutingRules * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SrvVSchema.decodeDelimited = function decodeDelimited(reader) { + RoutingRules.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SrvVSchema message. + * Verifies a RoutingRules message. * @function verify - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SrvVSchema.verify = function verify(message) { + RoutingRules.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspaces != null && message.hasOwnProperty("keyspaces")) { - if (!$util.isObject(message.keyspaces)) - return "keyspaces: object expected"; - let key = Object.keys(message.keyspaces); - for (let i = 0; i < key.length; ++i) { - let error = $root.vschema.Keyspace.verify(message.keyspaces[key[i]]); + 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.RoutingRule.verify(message.rules[i]); if (error) - return "keyspaces." + error; + return "rules." + error; } } - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { - let error = $root.vschema.RoutingRules.verify(message.routing_rules); - if (error) - return "routing_rules." + error; - } - if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { - let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); - if (error) - return "shard_routing_rules." + error; - } return null; }; /** - * Creates a SrvVSchema message from a plain object. Also converts values to their respective internal types. + * Creates a RoutingRules message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static * @param {Object.} object Plain object - * @returns {vschema.SrvVSchema} SrvVSchema + * @returns {vschema.RoutingRules} RoutingRules */ - SrvVSchema.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.SrvVSchema) + RoutingRules.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.RoutingRules) return object; - let message = new $root.vschema.SrvVSchema(); - if (object.keyspaces) { - if (typeof object.keyspaces !== "object") - throw TypeError(".vschema.SrvVSchema.keyspaces: object expected"); - message.keyspaces = {}; - for (let keys = Object.keys(object.keyspaces), i = 0; i < keys.length; ++i) { - if (typeof object.keyspaces[keys[i]] !== "object") - throw TypeError(".vschema.SrvVSchema.keyspaces: object expected"); - message.keyspaces[keys[i]] = $root.vschema.Keyspace.fromObject(object.keyspaces[keys[i]]); + let message = new $root.vschema.RoutingRules(); + if (object.rules) { + if (!Array.isArray(object.rules)) + throw TypeError(".vschema.RoutingRules.rules: array expected"); + message.rules = []; + for (let i = 0; i < object.rules.length; ++i) { + if (typeof object.rules[i] !== "object") + throw TypeError(".vschema.RoutingRules.rules: object expected"); + message.rules[i] = $root.vschema.RoutingRule.fromObject(object.rules[i]); } } - if (object.routing_rules != null) { - if (typeof object.routing_rules !== "object") - throw TypeError(".vschema.SrvVSchema.routing_rules: object expected"); - message.routing_rules = $root.vschema.RoutingRules.fromObject(object.routing_rules); - } - if (object.shard_routing_rules != null) { - if (typeof object.shard_routing_rules !== "object") - throw TypeError(".vschema.SrvVSchema.shard_routing_rules: object expected"); - message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_routing_rules); - } return message; }; /** - * Creates a plain object from a SrvVSchema message. Also converts values to other types if specified. + * Creates a plain object from a RoutingRules message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static - * @param {vschema.SrvVSchema} message SrvVSchema + * @param {vschema.RoutingRules} message RoutingRules * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SrvVSchema.toObject = function toObject(message, options) { + RoutingRules.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.keyspaces = {}; - if (options.defaults) { - object.routing_rules = null; - object.shard_routing_rules = null; - } - let keys2; - if (message.keyspaces && (keys2 = Object.keys(message.keyspaces)).length) { - object.keyspaces = {}; - for (let j = 0; j < keys2.length; ++j) - object.keyspaces[keys2[j]] = $root.vschema.Keyspace.toObject(message.keyspaces[keys2[j]], options); + 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.RoutingRule.toObject(message.rules[j], options); } - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) - 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); return object; }; /** - * Converts this SrvVSchema to JSON. + * Converts this RoutingRules to JSON. * @function toJSON - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @instance * @returns {Object.} JSON object */ - SrvVSchema.prototype.toJSON = function toJSON() { + RoutingRules.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SrvVSchema + * Gets the default type url for RoutingRules * @function getTypeUrl - * @memberof vschema.SrvVSchema + * @memberof vschema.RoutingRules * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SrvVSchema.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RoutingRules.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.SrvVSchema"; + return typeUrlPrefix + "/vschema.RoutingRules"; }; - return SrvVSchema; + return RoutingRules; })(); - vschema.ShardRoutingRules = (function() { + vschema.RoutingRule = (function() { /** - * Properties of a ShardRoutingRules. + * Properties of a RoutingRule. * @memberof vschema - * @interface IShardRoutingRules - * @property {Array.|null} [rules] ShardRoutingRules rules + * @interface IRoutingRule + * @property {string|null} [from_table] RoutingRule from_table + * @property {Array.|null} [to_tables] RoutingRule to_tables */ /** - * Constructs a new ShardRoutingRules. + * Constructs a new RoutingRule. * @memberof vschema - * @classdesc Represents a ShardRoutingRules. - * @implements IShardRoutingRules + * @classdesc Represents a RoutingRule. + * @implements IRoutingRule * @constructor - * @param {vschema.IShardRoutingRules=} [properties] Properties to set + * @param {vschema.IRoutingRule=} [properties] Properties to set */ - function ShardRoutingRules(properties) { - this.rules = []; + function RoutingRule(properties) { + this.to_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -97823,78 +98154,92 @@ export const vschema = $root.vschema = (() => { } /** - * ShardRoutingRules rules. - * @member {Array.} rules - * @memberof vschema.ShardRoutingRules + * RoutingRule from_table. + * @member {string} from_table + * @memberof vschema.RoutingRule * @instance */ - ShardRoutingRules.prototype.rules = $util.emptyArray; + RoutingRule.prototype.from_table = ""; /** - * Creates a new ShardRoutingRules instance using the specified properties. + * RoutingRule to_tables. + * @member {Array.} to_tables + * @memberof vschema.RoutingRule + * @instance + */ + RoutingRule.prototype.to_tables = $util.emptyArray; + + /** + * Creates a new RoutingRule instance using the specified properties. * @function create - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static - * @param {vschema.IShardRoutingRules=} [properties] Properties to set - * @returns {vschema.ShardRoutingRules} ShardRoutingRules instance + * @param {vschema.IRoutingRule=} [properties] Properties to set + * @returns {vschema.RoutingRule} RoutingRule instance */ - ShardRoutingRules.create = function create(properties) { - return new ShardRoutingRules(properties); + RoutingRule.create = function create(properties) { + return new RoutingRule(properties); }; /** - * Encodes the specified ShardRoutingRules message. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. + * Encodes the specified RoutingRule message. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. * @function encode - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static - * @param {vschema.IShardRoutingRules} message ShardRoutingRules message or plain object to encode + * @param {vschema.IRoutingRule} message RoutingRule message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardRoutingRules.encode = function encode(message, writer) { + RoutingRule.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.ShardRoutingRule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.from_table != null && Object.hasOwnProperty.call(message, "from_table")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.from_table); + if (message.to_tables != null && message.to_tables.length) + for (let i = 0; i < message.to_tables.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.to_tables[i]); return writer; }; /** - * Encodes the specified ShardRoutingRules message, length delimited. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. + * Encodes the specified RoutingRule message, length delimited. Does not implicitly {@link vschema.RoutingRule.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static - * @param {vschema.IShardRoutingRules} message ShardRoutingRules message or plain object to encode + * @param {vschema.IRoutingRule} message RoutingRule message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardRoutingRules.encodeDelimited = function encodeDelimited(message, writer) { + RoutingRule.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardRoutingRules message from the specified reader or buffer. + * Decodes a RoutingRule message from the specified reader or buffer. * @function decode - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.ShardRoutingRules} ShardRoutingRules + * @returns {vschema.RoutingRule} RoutingRule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardRoutingRules.decode = function decode(reader, length) { + RoutingRule.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.ShardRoutingRules(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.RoutingRule(); 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.ShardRoutingRule.decode(reader, reader.uint32())); + message.from_table = reader.string(); + break; + } + case 2: { + if (!(message.to_tables && message.to_tables.length)) + message.to_tables = []; + message.to_tables.push(reader.string()); break; } default: @@ -97906,141 +98251,149 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a ShardRoutingRules message from the specified reader or buffer, length delimited. + * Decodes a RoutingRule message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.ShardRoutingRules} ShardRoutingRules + * @returns {vschema.RoutingRule} RoutingRule * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardRoutingRules.decodeDelimited = function decodeDelimited(reader) { + RoutingRule.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardRoutingRules message. + * Verifies a RoutingRule message. * @function verify - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardRoutingRules.verify = function verify(message) { + RoutingRule.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.ShardRoutingRule.verify(message.rules[i]); - if (error) - return "rules." + error; - } + if (message.from_table != null && message.hasOwnProperty("from_table")) + if (!$util.isString(message.from_table)) + return "from_table: string expected"; + if (message.to_tables != null && message.hasOwnProperty("to_tables")) { + if (!Array.isArray(message.to_tables)) + return "to_tables: array expected"; + for (let i = 0; i < message.to_tables.length; ++i) + if (!$util.isString(message.to_tables[i])) + return "to_tables: string[] expected"; } return null; }; /** - * Creates a ShardRoutingRules message from a plain object. Also converts values to their respective internal types. + * Creates a RoutingRule message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static * @param {Object.} object Plain object - * @returns {vschema.ShardRoutingRules} ShardRoutingRules + * @returns {vschema.RoutingRule} RoutingRule */ - ShardRoutingRules.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.ShardRoutingRules) + RoutingRule.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.RoutingRule) return object; - let message = new $root.vschema.ShardRoutingRules(); - if (object.rules) { - if (!Array.isArray(object.rules)) - throw TypeError(".vschema.ShardRoutingRules.rules: array expected"); - message.rules = []; - for (let i = 0; i < object.rules.length; ++i) { - if (typeof object.rules[i] !== "object") - throw TypeError(".vschema.ShardRoutingRules.rules: object expected"); - message.rules[i] = $root.vschema.ShardRoutingRule.fromObject(object.rules[i]); - } + let message = new $root.vschema.RoutingRule(); + if (object.from_table != null) + message.from_table = String(object.from_table); + if (object.to_tables) { + if (!Array.isArray(object.to_tables)) + throw TypeError(".vschema.RoutingRule.to_tables: array expected"); + message.to_tables = []; + for (let i = 0; i < object.to_tables.length; ++i) + message.to_tables[i] = String(object.to_tables[i]); } return message; }; /** - * Creates a plain object from a ShardRoutingRules message. Also converts values to other types if specified. + * Creates a plain object from a RoutingRule message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static - * @param {vschema.ShardRoutingRules} message ShardRoutingRules + * @param {vschema.RoutingRule} message RoutingRule * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardRoutingRules.toObject = function toObject(message, options) { + RoutingRule.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.ShardRoutingRule.toObject(message.rules[j], options); + object.to_tables = []; + if (options.defaults) + object.from_table = ""; + if (message.from_table != null && message.hasOwnProperty("from_table")) + object.from_table = message.from_table; + if (message.to_tables && message.to_tables.length) { + object.to_tables = []; + for (let j = 0; j < message.to_tables.length; ++j) + object.to_tables[j] = message.to_tables[j]; } return object; }; /** - * Converts this ShardRoutingRules to JSON. + * Converts this RoutingRule to JSON. * @function toJSON - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @instance * @returns {Object.} JSON object */ - ShardRoutingRules.prototype.toJSON = function toJSON() { + RoutingRule.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardRoutingRules + * Gets the default type url for RoutingRule * @function getTypeUrl - * @memberof vschema.ShardRoutingRules + * @memberof vschema.RoutingRule * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardRoutingRules.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RoutingRule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.ShardRoutingRules"; + return typeUrlPrefix + "/vschema.RoutingRule"; }; - return ShardRoutingRules; + return RoutingRule; })(); - vschema.ShardRoutingRule = (function() { + vschema.Keyspace = (function() { /** - * Properties of a ShardRoutingRule. + * Properties of a Keyspace. * @memberof vschema - * @interface IShardRoutingRule - * @property {string|null} [from_keyspace] ShardRoutingRule from_keyspace - * @property {string|null} [to_keyspace] ShardRoutingRule to_keyspace - * @property {string|null} [shard] ShardRoutingRule shard + * @interface IKeyspace + * @property {boolean|null} [sharded] Keyspace sharded + * @property {Object.|null} [vindexes] Keyspace vindexes + * @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 */ /** - * Constructs a new ShardRoutingRule. + * Constructs a new Keyspace. * @memberof vschema - * @classdesc Represents a ShardRoutingRule. - * @implements IShardRoutingRule + * @classdesc Represents a Keyspace. + * @implements IKeyspace * @constructor - * @param {vschema.IShardRoutingRule=} [properties] Properties to set + * @param {vschema.IKeyspace=} [properties] Properties to set */ - function ShardRoutingRule(properties) { + function Keyspace(properties) { + this.vindexes = {}; + this.tables = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -98048,103 +98401,175 @@ export const vschema = $root.vschema = (() => { } /** - * ShardRoutingRule from_keyspace. - * @member {string} from_keyspace - * @memberof vschema.ShardRoutingRule + * Keyspace sharded. + * @member {boolean} sharded + * @memberof vschema.Keyspace * @instance */ - ShardRoutingRule.prototype.from_keyspace = ""; + Keyspace.prototype.sharded = false; /** - * ShardRoutingRule to_keyspace. - * @member {string} to_keyspace - * @memberof vschema.ShardRoutingRule + * Keyspace vindexes. + * @member {Object.} vindexes + * @memberof vschema.Keyspace * @instance */ - ShardRoutingRule.prototype.to_keyspace = ""; + Keyspace.prototype.vindexes = $util.emptyObject; /** - * ShardRoutingRule shard. - * @member {string} shard - * @memberof vschema.ShardRoutingRule + * Keyspace tables. + * @member {Object.} tables + * @memberof vschema.Keyspace * @instance */ - ShardRoutingRule.prototype.shard = ""; + Keyspace.prototype.tables = $util.emptyObject; /** - * Creates a new ShardRoutingRule instance using the specified properties. + * Keyspace require_explicit_routing. + * @member {boolean} require_explicit_routing + * @memberof vschema.Keyspace + * @instance + */ + Keyspace.prototype.require_explicit_routing = false; + + /** + * Keyspace foreign_key_mode. + * @member {vschema.Keyspace.ForeignKeyMode} foreign_key_mode + * @memberof vschema.Keyspace + * @instance + */ + Keyspace.prototype.foreign_key_mode = 0; + + /** + * Creates a new Keyspace instance using the specified properties. * @function create - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static - * @param {vschema.IShardRoutingRule=} [properties] Properties to set - * @returns {vschema.ShardRoutingRule} ShardRoutingRule instance + * @param {vschema.IKeyspace=} [properties] Properties to set + * @returns {vschema.Keyspace} Keyspace instance */ - ShardRoutingRule.create = function create(properties) { - return new ShardRoutingRule(properties); + Keyspace.create = function create(properties) { + return new Keyspace(properties); }; /** - * Encodes the specified ShardRoutingRule message. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * Encodes the specified Keyspace message. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. * @function encode - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static - * @param {vschema.IShardRoutingRule} message ShardRoutingRule message or plain object to encode + * @param {vschema.IKeyspace} message Keyspace message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardRoutingRule.encode = function encode(message, writer) { + Keyspace.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); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); + if (message.sharded != null && Object.hasOwnProperty.call(message, "sharded")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.sharded); + if (message.vindexes != null && Object.hasOwnProperty.call(message, "vindexes")) + for (let keys = Object.keys(message.vindexes), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vschema.Vindex.encode(message.vindexes[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) + for (let keys = Object.keys(message.tables), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 3, wireType 2 =*/26).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vschema.Table.encode(message.tables[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.require_explicit_routing != null && Object.hasOwnProperty.call(message, "require_explicit_routing")) + 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); return writer; }; /** - * Encodes the specified ShardRoutingRule message, length delimited. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vschema.Keyspace.verify|verify} messages. * @function encodeDelimited - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static - * @param {vschema.IShardRoutingRule} message ShardRoutingRule message or plain object to encode + * @param {vschema.IKeyspace} message Keyspace message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardRoutingRule.encodeDelimited = function encodeDelimited(message, writer) { + Keyspace.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardRoutingRule message from the specified reader or buffer. + * Decodes a Keyspace message from the specified reader or buffer. * @function decode - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vschema.ShardRoutingRule} ShardRoutingRule + * @returns {vschema.Keyspace} Keyspace * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardRoutingRule.decode = function decode(reader, length) { + Keyspace.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.ShardRoutingRule(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.Keyspace(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.from_keyspace = reader.string(); + message.sharded = reader.bool(); break; } case 2: { - message.to_keyspace = reader.string(); + if (message.vindexes === $util.emptyObject) + message.vindexes = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vschema.Vindex.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.vindexes[key] = value; break; } case 3: { - message.shard = reader.string(); + if (message.tables === $util.emptyObject) + message.tables = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vschema.Table.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.tables[key] = value; + break; + } + case 4: { + message.require_explicit_routing = reader.bool(); + break; + } + case 5: { + message.foreign_key_mode = reader.int32(); break; } default: @@ -98156,153 +98581,244 @@ export const vschema = $root.vschema = (() => { }; /** - * Decodes a ShardRoutingRule message from the specified reader or buffer, length delimited. + * Decodes a Keyspace message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vschema.ShardRoutingRule} ShardRoutingRule + * @returns {vschema.Keyspace} Keyspace * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardRoutingRule.decodeDelimited = function decodeDelimited(reader) { + Keyspace.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardRoutingRule message. + * Verifies a Keyspace message. * @function verify - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardRoutingRule.verify = function verify(message) { + Keyspace.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"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; + if (message.sharded != null && message.hasOwnProperty("sharded")) + if (typeof message.sharded !== "boolean") + return "sharded: boolean expected"; + if (message.vindexes != null && message.hasOwnProperty("vindexes")) { + if (!$util.isObject(message.vindexes)) + return "vindexes: object expected"; + let key = Object.keys(message.vindexes); + for (let i = 0; i < key.length; ++i) { + let error = $root.vschema.Vindex.verify(message.vindexes[key[i]]); + if (error) + return "vindexes." + error; + } + } + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!$util.isObject(message.tables)) + return "tables: object expected"; + let key = Object.keys(message.tables); + for (let i = 0; i < key.length; ++i) { + let error = $root.vschema.Table.verify(message.tables[key[i]]); + if (error) + return "tables." + error; + } + } + if (message.require_explicit_routing != null && message.hasOwnProperty("require_explicit_routing")) + if (typeof message.require_explicit_routing !== "boolean") + return "require_explicit_routing: boolean expected"; + if (message.foreign_key_mode != null && message.hasOwnProperty("foreign_key_mode")) + switch (message.foreign_key_mode) { + default: + return "foreign_key_mode: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } return null; }; /** - * Creates a ShardRoutingRule message from a plain object. Also converts values to their respective internal types. + * Creates a Keyspace message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static * @param {Object.} object Plain object - * @returns {vschema.ShardRoutingRule} ShardRoutingRule + * @returns {vschema.Keyspace} Keyspace */ - ShardRoutingRule.fromObject = function fromObject(object) { - if (object instanceof $root.vschema.ShardRoutingRule) + Keyspace.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.Keyspace) return object; - let message = new $root.vschema.ShardRoutingRule(); - if (object.from_keyspace != null) - message.from_keyspace = String(object.from_keyspace); - if (object.to_keyspace != null) - message.to_keyspace = String(object.to_keyspace); - if (object.shard != null) - message.shard = String(object.shard); + let message = new $root.vschema.Keyspace(); + if (object.sharded != null) + message.sharded = Boolean(object.sharded); + if (object.vindexes) { + if (typeof object.vindexes !== "object") + throw TypeError(".vschema.Keyspace.vindexes: object expected"); + message.vindexes = {}; + for (let keys = Object.keys(object.vindexes), i = 0; i < keys.length; ++i) { + if (typeof object.vindexes[keys[i]] !== "object") + throw TypeError(".vschema.Keyspace.vindexes: object expected"); + message.vindexes[keys[i]] = $root.vschema.Vindex.fromObject(object.vindexes[keys[i]]); + } + } + if (object.tables) { + if (typeof object.tables !== "object") + throw TypeError(".vschema.Keyspace.tables: object expected"); + message.tables = {}; + for (let keys = Object.keys(object.tables), i = 0; i < keys.length; ++i) { + if (typeof object.tables[keys[i]] !== "object") + throw TypeError(".vschema.Keyspace.tables: object expected"); + message.tables[keys[i]] = $root.vschema.Table.fromObject(object.tables[keys[i]]); + } + } + if (object.require_explicit_routing != null) + message.require_explicit_routing = Boolean(object.require_explicit_routing); + switch (object.foreign_key_mode) { + default: + if (typeof object.foreign_key_mode === "number") { + message.foreign_key_mode = object.foreign_key_mode; + break; + } + break; + case "FK_DEFAULT": + case 0: + message.foreign_key_mode = 0; + break; + case "FK_DISALLOW": + case 1: + message.foreign_key_mode = 1; + break; + case "FK_UNMANAGED": + case 2: + message.foreign_key_mode = 2; + break; + case "FK_MANAGED": + case 3: + message.foreign_key_mode = 3; + break; + } return message; }; /** - * Creates a plain object from a ShardRoutingRule message. Also converts values to other types if specified. + * Creates a plain object from a Keyspace message. Also converts values to other types if specified. * @function toObject - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static - * @param {vschema.ShardRoutingRule} message ShardRoutingRule + * @param {vschema.Keyspace} message Keyspace * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardRoutingRule.toObject = function toObject(message, options) { + Keyspace.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.objects || options.defaults) { + object.vindexes = {}; + object.tables = {}; + } if (options.defaults) { - object.from_keyspace = ""; - object.to_keyspace = ""; - object.shard = ""; + object.sharded = false; + object.require_explicit_routing = false; + object.foreign_key_mode = options.enums === String ? "FK_DEFAULT" : 0; } - 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; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; + if (message.sharded != null && message.hasOwnProperty("sharded")) + object.sharded = message.sharded; + let keys2; + if (message.vindexes && (keys2 = Object.keys(message.vindexes)).length) { + object.vindexes = {}; + for (let j = 0; j < keys2.length; ++j) + object.vindexes[keys2[j]] = $root.vschema.Vindex.toObject(message.vindexes[keys2[j]], options); + } + if (message.tables && (keys2 = Object.keys(message.tables)).length) { + object.tables = {}; + for (let j = 0; j < keys2.length; ++j) + object.tables[keys2[j]] = $root.vschema.Table.toObject(message.tables[keys2[j]], options); + } + if (message.require_explicit_routing != null && message.hasOwnProperty("require_explicit_routing")) + 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; return object; }; /** - * Converts this ShardRoutingRule to JSON. + * Converts this Keyspace to JSON. * @function toJSON - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @instance * @returns {Object.} JSON object */ - ShardRoutingRule.prototype.toJSON = function toJSON() { + Keyspace.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardRoutingRule + * Gets the default type url for Keyspace * @function getTypeUrl - * @memberof vschema.ShardRoutingRule + * @memberof vschema.Keyspace * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardRoutingRule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Keyspace.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vschema.ShardRoutingRule"; + return typeUrlPrefix + "/vschema.Keyspace"; }; - return ShardRoutingRule; - })(); - - return vschema; -})(); - -export const vtctldata = $root.vtctldata = (() => { + /** + * ForeignKeyMode enum. + * @name vschema.Keyspace.ForeignKeyMode + * @enum {number} + * @property {number} FK_DEFAULT=0 FK_DEFAULT value + * @property {number} FK_DISALLOW=1 FK_DISALLOW value + * @property {number} FK_UNMANAGED=2 FK_UNMANAGED value + * @property {number} FK_MANAGED=3 FK_MANAGED value + */ + Keyspace.ForeignKeyMode = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "FK_DEFAULT"] = 0; + values[valuesById[1] = "FK_DISALLOW"] = 1; + values[valuesById[2] = "FK_UNMANAGED"] = 2; + values[valuesById[3] = "FK_MANAGED"] = 3; + return values; + })(); - /** - * Namespace vtctldata. - * @exports vtctldata - * @namespace - */ - const vtctldata = {}; + return Keyspace; + })(); - vtctldata.ExecuteVtctlCommandRequest = (function() { + vschema.Vindex = (function() { /** - * Properties of an ExecuteVtctlCommandRequest. - * @memberof vtctldata - * @interface IExecuteVtctlCommandRequest - * @property {Array.|null} [args] ExecuteVtctlCommandRequest args - * @property {number|Long|null} [action_timeout] ExecuteVtctlCommandRequest action_timeout + * Properties of a Vindex. + * @memberof vschema + * @interface IVindex + * @property {string|null} [type] Vindex type + * @property {Object.|null} [params] Vindex params + * @property {string|null} [owner] Vindex owner */ /** - * Constructs a new ExecuteVtctlCommandRequest. - * @memberof vtctldata - * @classdesc Represents an ExecuteVtctlCommandRequest. - * @implements IExecuteVtctlCommandRequest + * Constructs a new Vindex. + * @memberof vschema + * @classdesc Represents a Vindex. + * @implements IVindex * @constructor - * @param {vtctldata.IExecuteVtctlCommandRequest=} [properties] Properties to set + * @param {vschema.IVindex=} [properties] Properties to set */ - function ExecuteVtctlCommandRequest(properties) { - this.args = []; + function Vindex(properties) { + this.params = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -98310,92 +98826,123 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteVtctlCommandRequest args. - * @member {Array.} args - * @memberof vtctldata.ExecuteVtctlCommandRequest + * Vindex type. + * @member {string} type + * @memberof vschema.Vindex * @instance */ - ExecuteVtctlCommandRequest.prototype.args = $util.emptyArray; + Vindex.prototype.type = ""; /** - * ExecuteVtctlCommandRequest action_timeout. - * @member {number|Long} action_timeout - * @memberof vtctldata.ExecuteVtctlCommandRequest + * Vindex params. + * @member {Object.} params + * @memberof vschema.Vindex * @instance */ - ExecuteVtctlCommandRequest.prototype.action_timeout = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + Vindex.prototype.params = $util.emptyObject; /** - * Creates a new ExecuteVtctlCommandRequest instance using the specified properties. + * Vindex owner. + * @member {string} owner + * @memberof vschema.Vindex + * @instance + */ + Vindex.prototype.owner = ""; + + /** + * Creates a new Vindex instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static - * @param {vtctldata.IExecuteVtctlCommandRequest=} [properties] Properties to set - * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest instance + * @param {vschema.IVindex=} [properties] Properties to set + * @returns {vschema.Vindex} Vindex instance */ - ExecuteVtctlCommandRequest.create = function create(properties) { - return new ExecuteVtctlCommandRequest(properties); + Vindex.create = function create(properties) { + return new Vindex(properties); }; /** - * Encodes the specified ExecuteVtctlCommandRequest message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * Encodes the specified Vindex message. Does not implicitly {@link vschema.Vindex.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static - * @param {vtctldata.IExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest message or plain object to encode + * @param {vschema.IVindex} message Vindex message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteVtctlCommandRequest.encode = function encode(message, writer) { + Vindex.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.args != null && message.args.length) - for (let i = 0; i < message.args.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.args[i]); - if (message.action_timeout != null && Object.hasOwnProperty.call(message, "action_timeout")) - writer.uint32(/* id 2, wireType 0 =*/16).int64(message.action_timeout); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.type); + if (message.params != null && Object.hasOwnProperty.call(message, "params")) + for (let keys = Object.keys(message.params), i = 0; i < keys.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]).uint32(/* id 2, wireType 2 =*/18).string(message.params[keys[i]]).ldelim(); + if (message.owner != null && Object.hasOwnProperty.call(message, "owner")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.owner); return writer; }; /** - * Encodes the specified ExecuteVtctlCommandRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * Encodes the specified Vindex message, length delimited. Does not implicitly {@link vschema.Vindex.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static - * @param {vtctldata.IExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest message or plain object to encode + * @param {vschema.IVindex} message Vindex message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteVtctlCommandRequest.encodeDelimited = function encodeDelimited(message, writer) { + Vindex.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer. + * Decodes a Vindex message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + * @returns {vschema.Vindex} Vindex * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteVtctlCommandRequest.decode = function decode(reader, length) { + Vindex.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.ExecuteVtctlCommandRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.Vindex(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.args && message.args.length)) - message.args = []; - message.args.push(reader.string()); + message.type = reader.string(); break; } case 2: { - message.action_timeout = reader.int64(); + if (message.params === $util.emptyObject) + message.params = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = ""; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = reader.string(); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.params[key] = value; + break; + } + case 3: { + message.owner = reader.string(); break; } default: @@ -98407,157 +98954,162 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer, length delimited. + * Decodes a Vindex message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + * @returns {vschema.Vindex} Vindex * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteVtctlCommandRequest.decodeDelimited = function decodeDelimited(reader) { + Vindex.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteVtctlCommandRequest message. + * Verifies a Vindex message. * @function verify - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteVtctlCommandRequest.verify = function verify(message) { + Vindex.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.args != null && message.hasOwnProperty("args")) { - if (!Array.isArray(message.args)) - return "args: array expected"; - for (let i = 0; i < message.args.length; ++i) - if (!$util.isString(message.args[i])) - return "args: string[] expected"; + if (message.type != null && message.hasOwnProperty("type")) + if (!$util.isString(message.type)) + return "type: string expected"; + if (message.params != null && message.hasOwnProperty("params")) { + if (!$util.isObject(message.params)) + return "params: object expected"; + let key = Object.keys(message.params); + for (let i = 0; i < key.length; ++i) + if (!$util.isString(message.params[key[i]])) + return "params: string{k:string} expected"; } - if (message.action_timeout != null && message.hasOwnProperty("action_timeout")) - if (!$util.isInteger(message.action_timeout) && !(message.action_timeout && $util.isInteger(message.action_timeout.low) && $util.isInteger(message.action_timeout.high))) - return "action_timeout: integer|Long expected"; + if (message.owner != null && message.hasOwnProperty("owner")) + if (!$util.isString(message.owner)) + return "owner: string expected"; return null; }; /** - * Creates an ExecuteVtctlCommandRequest message from a plain object. Also converts values to their respective internal types. + * Creates a Vindex message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + * @returns {vschema.Vindex} Vindex */ - ExecuteVtctlCommandRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteVtctlCommandRequest) + Vindex.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.Vindex) return object; - let message = new $root.vtctldata.ExecuteVtctlCommandRequest(); - if (object.args) { - if (!Array.isArray(object.args)) - throw TypeError(".vtctldata.ExecuteVtctlCommandRequest.args: array expected"); - message.args = []; - for (let i = 0; i < object.args.length; ++i) - message.args[i] = String(object.args[i]); + let message = new $root.vschema.Vindex(); + if (object.type != null) + message.type = String(object.type); + if (object.params) { + if (typeof object.params !== "object") + throw TypeError(".vschema.Vindex.params: object expected"); + message.params = {}; + for (let keys = Object.keys(object.params), i = 0; i < keys.length; ++i) + message.params[keys[i]] = String(object.params[keys[i]]); } - if (object.action_timeout != null) - if ($util.Long) - (message.action_timeout = $util.Long.fromValue(object.action_timeout)).unsigned = false; - else if (typeof object.action_timeout === "string") - message.action_timeout = parseInt(object.action_timeout, 10); - else if (typeof object.action_timeout === "number") - message.action_timeout = object.action_timeout; - else if (typeof object.action_timeout === "object") - message.action_timeout = new $util.LongBits(object.action_timeout.low >>> 0, object.action_timeout.high >>> 0).toNumber(); + if (object.owner != null) + message.owner = String(object.owner); return message; }; /** - * Creates a plain object from an ExecuteVtctlCommandRequest message. Also converts values to other types if specified. + * Creates a plain object from a Vindex message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static - * @param {vtctldata.ExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest + * @param {vschema.Vindex} message Vindex * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteVtctlCommandRequest.toObject = function toObject(message, options) { + Vindex.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.args = []; - if (options.defaults) - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.action_timeout = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.action_timeout = options.longs === String ? "0" : 0; - if (message.args && message.args.length) { - object.args = []; - for (let j = 0; j < message.args.length; ++j) - object.args[j] = message.args[j]; + if (options.objects || options.defaults) + object.params = {}; + if (options.defaults) { + object.type = ""; + object.owner = ""; } - if (message.action_timeout != null && message.hasOwnProperty("action_timeout")) - if (typeof message.action_timeout === "number") - object.action_timeout = options.longs === String ? String(message.action_timeout) : message.action_timeout; - else - object.action_timeout = options.longs === String ? $util.Long.prototype.toString.call(message.action_timeout) : options.longs === Number ? new $util.LongBits(message.action_timeout.low >>> 0, message.action_timeout.high >>> 0).toNumber() : message.action_timeout; + if (message.type != null && message.hasOwnProperty("type")) + object.type = message.type; + let keys2; + if (message.params && (keys2 = Object.keys(message.params)).length) { + object.params = {}; + for (let j = 0; j < keys2.length; ++j) + object.params[keys2[j]] = message.params[keys2[j]]; + } + if (message.owner != null && message.hasOwnProperty("owner")) + object.owner = message.owner; return object; }; /** - * Converts this ExecuteVtctlCommandRequest to JSON. + * Converts this Vindex to JSON. * @function toJSON - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @instance * @returns {Object.} JSON object */ - ExecuteVtctlCommandRequest.prototype.toJSON = function toJSON() { + Vindex.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteVtctlCommandRequest + * Gets the default type url for Vindex * @function getTypeUrl - * @memberof vtctldata.ExecuteVtctlCommandRequest + * @memberof vschema.Vindex * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteVtctlCommandRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Vindex.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteVtctlCommandRequest"; + return typeUrlPrefix + "/vschema.Vindex"; }; - return ExecuteVtctlCommandRequest; + return Vindex; })(); - vtctldata.ExecuteVtctlCommandResponse = (function() { + vschema.Table = (function() { /** - * Properties of an ExecuteVtctlCommandResponse. - * @memberof vtctldata - * @interface IExecuteVtctlCommandResponse - * @property {logutil.IEvent|null} [event] ExecuteVtctlCommandResponse event + * Properties of a Table. + * @memberof vschema + * @interface ITable + * @property {string|null} [type] Table type + * @property {Array.|null} [column_vindexes] Table column_vindexes + * @property {vschema.IAutoIncrement|null} [auto_increment] Table auto_increment + * @property {Array.|null} [columns] Table columns + * @property {string|null} [pinned] Table pinned + * @property {boolean|null} [column_list_authoritative] Table column_list_authoritative + * @property {string|null} [source] Table source */ /** - * Constructs a new ExecuteVtctlCommandResponse. - * @memberof vtctldata - * @classdesc Represents an ExecuteVtctlCommandResponse. - * @implements IExecuteVtctlCommandResponse + * Constructs a new Table. + * @memberof vschema + * @classdesc Represents a Table. + * @implements ITable * @constructor - * @param {vtctldata.IExecuteVtctlCommandResponse=} [properties] Properties to set + * @param {vschema.ITable=} [properties] Properties to set */ - function ExecuteVtctlCommandResponse(properties) { + function Table(properties) { + this.column_vindexes = []; + this.columns = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -98565,75 +99117,165 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteVtctlCommandResponse event. - * @member {logutil.IEvent|null|undefined} event - * @memberof vtctldata.ExecuteVtctlCommandResponse + * Table type. + * @member {string} type + * @memberof vschema.Table * @instance */ - ExecuteVtctlCommandResponse.prototype.event = null; + Table.prototype.type = ""; /** - * Creates a new ExecuteVtctlCommandResponse instance using the specified properties. - * @function create - * @memberof vtctldata.ExecuteVtctlCommandResponse - * @static - * @param {vtctldata.IExecuteVtctlCommandResponse=} [properties] Properties to set - * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse instance + * Table column_vindexes. + * @member {Array.} column_vindexes + * @memberof vschema.Table + * @instance */ - ExecuteVtctlCommandResponse.create = function create(properties) { - return new ExecuteVtctlCommandResponse(properties); - }; + Table.prototype.column_vindexes = $util.emptyArray; /** - * Encodes the specified ExecuteVtctlCommandResponse message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. - * @function encode - * @memberof vtctldata.ExecuteVtctlCommandResponse - * @static - * @param {vtctldata.IExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * Table auto_increment. + * @member {vschema.IAutoIncrement|null|undefined} auto_increment + * @memberof vschema.Table + * @instance */ - ExecuteVtctlCommandResponse.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.event != null && Object.hasOwnProperty.call(message, "event")) - $root.logutil.Event.encode(message.event, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - return writer; - }; + Table.prototype.auto_increment = null; /** - * Encodes the specified ExecuteVtctlCommandResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.ExecuteVtctlCommandResponse - * @static - * @param {vtctldata.IExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer + * Table columns. + * @member {Array.} columns + * @memberof vschema.Table + * @instance */ - ExecuteVtctlCommandResponse.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + Table.prototype.columns = $util.emptyArray; /** - * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.ExecuteVtctlCommandResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing + * Table pinned. + * @member {string} pinned + * @memberof vschema.Table + * @instance */ - ExecuteVtctlCommandResponse.decode = function decode(reader, length) { + Table.prototype.pinned = ""; + + /** + * Table column_list_authoritative. + * @member {boolean} column_list_authoritative + * @memberof vschema.Table + * @instance + */ + Table.prototype.column_list_authoritative = false; + + /** + * Table source. + * @member {string} source + * @memberof vschema.Table + * @instance + */ + Table.prototype.source = ""; + + /** + * Creates a new Table instance using the specified properties. + * @function create + * @memberof vschema.Table + * @static + * @param {vschema.ITable=} [properties] Properties to set + * @returns {vschema.Table} Table instance + */ + Table.create = function create(properties) { + return new Table(properties); + }; + + /** + * Encodes the specified Table message. Does not implicitly {@link vschema.Table.verify|verify} messages. + * @function encode + * @memberof vschema.Table + * @static + * @param {vschema.ITable} message Table message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Table.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.type); + if (message.column_vindexes != null && message.column_vindexes.length) + for (let i = 0; i < message.column_vindexes.length; ++i) + $root.vschema.ColumnVindex.encode(message.column_vindexes[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.auto_increment != null && Object.hasOwnProperty.call(message, "auto_increment")) + $root.vschema.AutoIncrement.encode(message.auto_increment, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.columns != null && message.columns.length) + for (let i = 0; i < message.columns.length; ++i) + $root.vschema.Column.encode(message.columns[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.pinned != null && Object.hasOwnProperty.call(message, "pinned")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.pinned); + if (message.column_list_authoritative != null && Object.hasOwnProperty.call(message, "column_list_authoritative")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.column_list_authoritative); + if (message.source != null && Object.hasOwnProperty.call(message, "source")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.source); + return writer; + }; + + /** + * Encodes the specified Table message, length delimited. Does not implicitly {@link vschema.Table.verify|verify} messages. + * @function encodeDelimited + * @memberof vschema.Table + * @static + * @param {vschema.ITable} message Table message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Table.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Table message from the specified reader or buffer. + * @function decode + * @memberof vschema.Table + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vschema.Table} Table + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Table.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.ExecuteVtctlCommandResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.Table(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.event = $root.logutil.Event.decode(reader, reader.uint32()); + message.type = reader.string(); + break; + } + case 2: { + if (!(message.column_vindexes && message.column_vindexes.length)) + message.column_vindexes = []; + message.column_vindexes.push($root.vschema.ColumnVindex.decode(reader, reader.uint32())); + break; + } + case 3: { + message.auto_increment = $root.vschema.AutoIncrement.decode(reader, reader.uint32()); + break; + } + case 4: { + if (!(message.columns && message.columns.length)) + message.columns = []; + message.columns.push($root.vschema.Column.decode(reader, reader.uint32())); + break; + } + case 5: { + message.pinned = reader.string(); + break; + } + case 6: { + message.column_list_authoritative = reader.bool(); + break; + } + case 7: { + message.source = reader.string(); break; } default: @@ -98645,145 +99287,215 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer, length delimited. + * Decodes a Table message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse + * @returns {vschema.Table} Table * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteVtctlCommandResponse.decodeDelimited = function decodeDelimited(reader) { + Table.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteVtctlCommandResponse message. + * Verifies a Table message. * @function verify - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteVtctlCommandResponse.verify = function verify(message) { + Table.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.event != null && message.hasOwnProperty("event")) { - let error = $root.logutil.Event.verify(message.event); + if (message.type != null && message.hasOwnProperty("type")) + if (!$util.isString(message.type)) + return "type: string expected"; + if (message.column_vindexes != null && message.hasOwnProperty("column_vindexes")) { + if (!Array.isArray(message.column_vindexes)) + return "column_vindexes: array expected"; + for (let i = 0; i < message.column_vindexes.length; ++i) { + let error = $root.vschema.ColumnVindex.verify(message.column_vindexes[i]); + if (error) + return "column_vindexes." + error; + } + } + if (message.auto_increment != null && message.hasOwnProperty("auto_increment")) { + let error = $root.vschema.AutoIncrement.verify(message.auto_increment); if (error) - return "event." + error; + return "auto_increment." + error; + } + if (message.columns != null && message.hasOwnProperty("columns")) { + if (!Array.isArray(message.columns)) + return "columns: array expected"; + for (let i = 0; i < message.columns.length; ++i) { + let error = $root.vschema.Column.verify(message.columns[i]); + if (error) + return "columns." + error; + } } + if (message.pinned != null && message.hasOwnProperty("pinned")) + if (!$util.isString(message.pinned)) + return "pinned: string expected"; + if (message.column_list_authoritative != null && message.hasOwnProperty("column_list_authoritative")) + if (typeof message.column_list_authoritative !== "boolean") + return "column_list_authoritative: boolean expected"; + if (message.source != null && message.hasOwnProperty("source")) + if (!$util.isString(message.source)) + return "source: string expected"; return null; }; /** - * Creates an ExecuteVtctlCommandResponse message from a plain object. Also converts values to their respective internal types. + * Creates a Table message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse + * @returns {vschema.Table} Table */ - ExecuteVtctlCommandResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteVtctlCommandResponse) + Table.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.Table) return object; - let message = new $root.vtctldata.ExecuteVtctlCommandResponse(); - if (object.event != null) { - if (typeof object.event !== "object") - throw TypeError(".vtctldata.ExecuteVtctlCommandResponse.event: object expected"); - message.event = $root.logutil.Event.fromObject(object.event); + let message = new $root.vschema.Table(); + if (object.type != null) + message.type = String(object.type); + if (object.column_vindexes) { + if (!Array.isArray(object.column_vindexes)) + throw TypeError(".vschema.Table.column_vindexes: array expected"); + message.column_vindexes = []; + for (let i = 0; i < object.column_vindexes.length; ++i) { + if (typeof object.column_vindexes[i] !== "object") + throw TypeError(".vschema.Table.column_vindexes: object expected"); + message.column_vindexes[i] = $root.vschema.ColumnVindex.fromObject(object.column_vindexes[i]); + } + } + if (object.auto_increment != null) { + if (typeof object.auto_increment !== "object") + throw TypeError(".vschema.Table.auto_increment: object expected"); + message.auto_increment = $root.vschema.AutoIncrement.fromObject(object.auto_increment); + } + if (object.columns) { + if (!Array.isArray(object.columns)) + throw TypeError(".vschema.Table.columns: array expected"); + message.columns = []; + for (let i = 0; i < object.columns.length; ++i) { + if (typeof object.columns[i] !== "object") + throw TypeError(".vschema.Table.columns: object expected"); + message.columns[i] = $root.vschema.Column.fromObject(object.columns[i]); + } } + if (object.pinned != null) + message.pinned = String(object.pinned); + if (object.column_list_authoritative != null) + message.column_list_authoritative = Boolean(object.column_list_authoritative); + if (object.source != null) + message.source = String(object.source); return message; }; /** - * Creates a plain object from an ExecuteVtctlCommandResponse message. Also converts values to other types if specified. + * Creates a plain object from a Table message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @static - * @param {vtctldata.ExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse + * @param {vschema.Table} message Table * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteVtctlCommandResponse.toObject = function toObject(message, options) { + Table.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.event = null; - if (message.event != null && message.hasOwnProperty("event")) - object.event = $root.logutil.Event.toObject(message.event, options); + if (options.arrays || options.defaults) { + object.column_vindexes = []; + object.columns = []; + } + if (options.defaults) { + object.type = ""; + object.auto_increment = null; + object.pinned = ""; + object.column_list_authoritative = false; + object.source = ""; + } + if (message.type != null && message.hasOwnProperty("type")) + object.type = message.type; + if (message.column_vindexes && message.column_vindexes.length) { + object.column_vindexes = []; + for (let j = 0; j < message.column_vindexes.length; ++j) + object.column_vindexes[j] = $root.vschema.ColumnVindex.toObject(message.column_vindexes[j], options); + } + if (message.auto_increment != null && message.hasOwnProperty("auto_increment")) + object.auto_increment = $root.vschema.AutoIncrement.toObject(message.auto_increment, options); + if (message.columns && message.columns.length) { + object.columns = []; + for (let j = 0; j < message.columns.length; ++j) + object.columns[j] = $root.vschema.Column.toObject(message.columns[j], options); + } + if (message.pinned != null && message.hasOwnProperty("pinned")) + object.pinned = message.pinned; + if (message.column_list_authoritative != null && message.hasOwnProperty("column_list_authoritative")) + object.column_list_authoritative = message.column_list_authoritative; + if (message.source != null && message.hasOwnProperty("source")) + object.source = message.source; return object; }; /** - * Converts this ExecuteVtctlCommandResponse to JSON. + * Converts this Table to JSON. * @function toJSON - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @instance * @returns {Object.} JSON object */ - ExecuteVtctlCommandResponse.prototype.toJSON = function toJSON() { + Table.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteVtctlCommandResponse + * Gets the default type url for Table * @function getTypeUrl - * @memberof vtctldata.ExecuteVtctlCommandResponse + * @memberof vschema.Table * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteVtctlCommandResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Table.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteVtctlCommandResponse"; + return typeUrlPrefix + "/vschema.Table"; }; - return ExecuteVtctlCommandResponse; - })(); - - /** - * MaterializationIntent enum. - * @name vtctldata.MaterializationIntent - * @enum {number} - * @property {number} CUSTOM=0 CUSTOM value - * @property {number} MOVETABLES=1 MOVETABLES value - * @property {number} CREATELOOKUPINDEX=2 CREATELOOKUPINDEX value - */ - vtctldata.MaterializationIntent = (function() { - const valuesById = {}, values = Object.create(valuesById); - values[valuesById[0] = "CUSTOM"] = 0; - values[valuesById[1] = "MOVETABLES"] = 1; - values[valuesById[2] = "CREATELOOKUPINDEX"] = 2; - return values; + return Table; })(); - vtctldata.TableMaterializeSettings = (function() { + vschema.ColumnVindex = (function() { /** - * Properties of a TableMaterializeSettings. - * @memberof vtctldata - * @interface ITableMaterializeSettings - * @property {string|null} [target_table] TableMaterializeSettings target_table - * @property {string|null} [source_expression] TableMaterializeSettings source_expression - * @property {string|null} [create_ddl] TableMaterializeSettings create_ddl + * Properties of a ColumnVindex. + * @memberof vschema + * @interface IColumnVindex + * @property {string|null} [column] ColumnVindex column + * @property {string|null} [name] ColumnVindex name + * @property {Array.|null} [columns] ColumnVindex columns */ /** - * Constructs a new TableMaterializeSettings. - * @memberof vtctldata - * @classdesc Represents a TableMaterializeSettings. - * @implements ITableMaterializeSettings + * Constructs a new ColumnVindex. + * @memberof vschema + * @classdesc Represents a ColumnVindex. + * @implements IColumnVindex * @constructor - * @param {vtctldata.ITableMaterializeSettings=} [properties] Properties to set + * @param {vschema.IColumnVindex=} [properties] Properties to set */ - function TableMaterializeSettings(properties) { + function ColumnVindex(properties) { + this.columns = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -98791,103 +99503,106 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * TableMaterializeSettings target_table. - * @member {string} target_table - * @memberof vtctldata.TableMaterializeSettings + * ColumnVindex column. + * @member {string} column + * @memberof vschema.ColumnVindex * @instance */ - TableMaterializeSettings.prototype.target_table = ""; + ColumnVindex.prototype.column = ""; /** - * TableMaterializeSettings source_expression. - * @member {string} source_expression - * @memberof vtctldata.TableMaterializeSettings + * ColumnVindex name. + * @member {string} name + * @memberof vschema.ColumnVindex * @instance */ - TableMaterializeSettings.prototype.source_expression = ""; + ColumnVindex.prototype.name = ""; /** - * TableMaterializeSettings create_ddl. - * @member {string} create_ddl - * @memberof vtctldata.TableMaterializeSettings + * ColumnVindex columns. + * @member {Array.} columns + * @memberof vschema.ColumnVindex * @instance */ - TableMaterializeSettings.prototype.create_ddl = ""; + ColumnVindex.prototype.columns = $util.emptyArray; /** - * Creates a new TableMaterializeSettings instance using the specified properties. + * Creates a new ColumnVindex instance using the specified properties. * @function create - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static - * @param {vtctldata.ITableMaterializeSettings=} [properties] Properties to set - * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings instance + * @param {vschema.IColumnVindex=} [properties] Properties to set + * @returns {vschema.ColumnVindex} ColumnVindex instance */ - TableMaterializeSettings.create = function create(properties) { - return new TableMaterializeSettings(properties); + ColumnVindex.create = function create(properties) { + return new ColumnVindex(properties); }; /** - * Encodes the specified TableMaterializeSettings message. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * Encodes the specified ColumnVindex message. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. * @function encode - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static - * @param {vtctldata.ITableMaterializeSettings} message TableMaterializeSettings message or plain object to encode + * @param {vschema.IColumnVindex} message ColumnVindex message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TableMaterializeSettings.encode = function encode(message, writer) { + ColumnVindex.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.target_table != null && Object.hasOwnProperty.call(message, "target_table")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.target_table); - if (message.source_expression != null && Object.hasOwnProperty.call(message, "source_expression")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_expression); - if (message.create_ddl != null && Object.hasOwnProperty.call(message, "create_ddl")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.create_ddl); + if (message.column != null && Object.hasOwnProperty.call(message, "column")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.column); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.name); + if (message.columns != null && message.columns.length) + for (let i = 0; i < message.columns.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.columns[i]); return writer; }; /** - * Encodes the specified TableMaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * Encodes the specified ColumnVindex message, length delimited. Does not implicitly {@link vschema.ColumnVindex.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static - * @param {vtctldata.ITableMaterializeSettings} message TableMaterializeSettings message or plain object to encode + * @param {vschema.IColumnVindex} message ColumnVindex message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TableMaterializeSettings.encodeDelimited = function encodeDelimited(message, writer) { + ColumnVindex.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TableMaterializeSettings message from the specified reader or buffer. + * Decodes a ColumnVindex message from the specified reader or buffer. * @function decode - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + * @returns {vschema.ColumnVindex} ColumnVindex * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TableMaterializeSettings.decode = function decode(reader, length) { + ColumnVindex.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.TableMaterializeSettings(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.ColumnVindex(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.target_table = reader.string(); + message.column = reader.string(); break; } case 2: { - message.source_expression = reader.string(); + message.name = reader.string(); break; } case 3: { - message.create_ddl = reader.string(); + if (!(message.columns && message.columns.length)) + message.columns = []; + message.columns.push(reader.string()); break; } default: @@ -98899,154 +99614,153 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a TableMaterializeSettings message from the specified reader or buffer, length delimited. + * Decodes a ColumnVindex message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + * @returns {vschema.ColumnVindex} ColumnVindex * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TableMaterializeSettings.decodeDelimited = function decodeDelimited(reader) { + ColumnVindex.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TableMaterializeSettings message. + * Verifies a ColumnVindex message. * @function verify - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TableMaterializeSettings.verify = function verify(message) { + ColumnVindex.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.target_table != null && message.hasOwnProperty("target_table")) - if (!$util.isString(message.target_table)) - return "target_table: string expected"; - if (message.source_expression != null && message.hasOwnProperty("source_expression")) - if (!$util.isString(message.source_expression)) - return "source_expression: string expected"; - if (message.create_ddl != null && message.hasOwnProperty("create_ddl")) - if (!$util.isString(message.create_ddl)) - return "create_ddl: string expected"; + if (message.column != null && message.hasOwnProperty("column")) + if (!$util.isString(message.column)) + return "column: string expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.columns != null && message.hasOwnProperty("columns")) { + if (!Array.isArray(message.columns)) + return "columns: array expected"; + for (let i = 0; i < message.columns.length; ++i) + if (!$util.isString(message.columns[i])) + return "columns: string[] expected"; + } return null; }; /** - * Creates a TableMaterializeSettings message from a plain object. Also converts values to their respective internal types. + * Creates a ColumnVindex message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static * @param {Object.} object Plain object - * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + * @returns {vschema.ColumnVindex} ColumnVindex */ - TableMaterializeSettings.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.TableMaterializeSettings) + ColumnVindex.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.ColumnVindex) return object; - let message = new $root.vtctldata.TableMaterializeSettings(); - if (object.target_table != null) - message.target_table = String(object.target_table); - if (object.source_expression != null) - message.source_expression = String(object.source_expression); - if (object.create_ddl != null) - message.create_ddl = String(object.create_ddl); + let message = new $root.vschema.ColumnVindex(); + if (object.column != null) + message.column = String(object.column); + if (object.name != null) + message.name = String(object.name); + if (object.columns) { + if (!Array.isArray(object.columns)) + throw TypeError(".vschema.ColumnVindex.columns: array expected"); + message.columns = []; + for (let i = 0; i < object.columns.length; ++i) + message.columns[i] = String(object.columns[i]); + } return message; }; /** - * Creates a plain object from a TableMaterializeSettings message. Also converts values to other types if specified. + * Creates a plain object from a ColumnVindex message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static - * @param {vtctldata.TableMaterializeSettings} message TableMaterializeSettings + * @param {vschema.ColumnVindex} message ColumnVindex * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TableMaterializeSettings.toObject = function toObject(message, options) { + ColumnVindex.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.columns = []; if (options.defaults) { - object.target_table = ""; - object.source_expression = ""; - object.create_ddl = ""; + object.column = ""; + object.name = ""; + } + if (message.column != null && message.hasOwnProperty("column")) + object.column = message.column; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.columns && message.columns.length) { + object.columns = []; + for (let j = 0; j < message.columns.length; ++j) + object.columns[j] = message.columns[j]; } - if (message.target_table != null && message.hasOwnProperty("target_table")) - object.target_table = message.target_table; - if (message.source_expression != null && message.hasOwnProperty("source_expression")) - object.source_expression = message.source_expression; - if (message.create_ddl != null && message.hasOwnProperty("create_ddl")) - object.create_ddl = message.create_ddl; return object; }; /** - * Converts this TableMaterializeSettings to JSON. + * Converts this ColumnVindex to JSON. * @function toJSON - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @instance * @returns {Object.} JSON object */ - TableMaterializeSettings.prototype.toJSON = function toJSON() { + ColumnVindex.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TableMaterializeSettings + * Gets the default type url for ColumnVindex * @function getTypeUrl - * @memberof vtctldata.TableMaterializeSettings + * @memberof vschema.ColumnVindex * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TableMaterializeSettings.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ColumnVindex.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.TableMaterializeSettings"; + return typeUrlPrefix + "/vschema.ColumnVindex"; }; - return TableMaterializeSettings; + return ColumnVindex; })(); - vtctldata.MaterializeSettings = (function() { + vschema.AutoIncrement = (function() { /** - * Properties of a MaterializeSettings. - * @memberof vtctldata - * @interface IMaterializeSettings - * @property {string|null} [workflow] MaterializeSettings workflow - * @property {string|null} [source_keyspace] MaterializeSettings source_keyspace - * @property {string|null} [target_keyspace] MaterializeSettings target_keyspace - * @property {boolean|null} [stop_after_copy] MaterializeSettings stop_after_copy - * @property {Array.|null} [table_settings] MaterializeSettings table_settings - * @property {string|null} [cell] MaterializeSettings cell - * @property {string|null} [tablet_types] MaterializeSettings tablet_types - * @property {string|null} [external_cluster] MaterializeSettings external_cluster - * @property {vtctldata.MaterializationIntent|null} [materialization_intent] MaterializeSettings materialization_intent - * @property {string|null} [source_time_zone] MaterializeSettings source_time_zone - * @property {string|null} [target_time_zone] MaterializeSettings target_time_zone - * @property {Array.|null} [source_shards] MaterializeSettings source_shards - * @property {string|null} [on_ddl] MaterializeSettings on_ddl - * @property {boolean|null} [defer_secondary_keys] MaterializeSettings defer_secondary_keys + * Properties of an AutoIncrement. + * @memberof vschema + * @interface IAutoIncrement + * @property {string|null} [column] AutoIncrement column + * @property {string|null} [sequence] AutoIncrement sequence */ /** - * Constructs a new MaterializeSettings. - * @memberof vtctldata - * @classdesc Represents a MaterializeSettings. - * @implements IMaterializeSettings + * Constructs a new AutoIncrement. + * @memberof vschema + * @classdesc Represents an AutoIncrement. + * @implements IAutoIncrement * @constructor - * @param {vtctldata.IMaterializeSettings=} [properties] Properties to set + * @param {vschema.IAutoIncrement=} [properties] Properties to set */ - function MaterializeSettings(properties) { - this.table_settings = []; - this.source_shards = []; + function AutoIncrement(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -99054,263 +99768,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * MaterializeSettings workflow. - * @member {string} workflow - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.workflow = ""; - - /** - * MaterializeSettings source_keyspace. - * @member {string} source_keyspace - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.source_keyspace = ""; - - /** - * MaterializeSettings target_keyspace. - * @member {string} target_keyspace - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.target_keyspace = ""; - - /** - * MaterializeSettings stop_after_copy. - * @member {boolean} stop_after_copy - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.stop_after_copy = false; - - /** - * MaterializeSettings table_settings. - * @member {Array.} table_settings - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.table_settings = $util.emptyArray; - - /** - * MaterializeSettings cell. - * @member {string} cell - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.cell = ""; - - /** - * MaterializeSettings tablet_types. - * @member {string} tablet_types - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.tablet_types = ""; - - /** - * MaterializeSettings external_cluster. - * @member {string} external_cluster - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.external_cluster = ""; - - /** - * MaterializeSettings materialization_intent. - * @member {vtctldata.MaterializationIntent} materialization_intent - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.materialization_intent = 0; - - /** - * MaterializeSettings source_time_zone. - * @member {string} source_time_zone - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.source_time_zone = ""; - - /** - * MaterializeSettings target_time_zone. - * @member {string} target_time_zone - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.target_time_zone = ""; - - /** - * MaterializeSettings source_shards. - * @member {Array.} source_shards - * @memberof vtctldata.MaterializeSettings - * @instance - */ - MaterializeSettings.prototype.source_shards = $util.emptyArray; - - /** - * MaterializeSettings on_ddl. - * @member {string} on_ddl - * @memberof vtctldata.MaterializeSettings + * AutoIncrement column. + * @member {string} column + * @memberof vschema.AutoIncrement * @instance */ - MaterializeSettings.prototype.on_ddl = ""; + AutoIncrement.prototype.column = ""; /** - * MaterializeSettings defer_secondary_keys. - * @member {boolean} defer_secondary_keys - * @memberof vtctldata.MaterializeSettings + * AutoIncrement sequence. + * @member {string} sequence + * @memberof vschema.AutoIncrement * @instance */ - MaterializeSettings.prototype.defer_secondary_keys = false; + AutoIncrement.prototype.sequence = ""; /** - * Creates a new MaterializeSettings instance using the specified properties. + * Creates a new AutoIncrement instance using the specified properties. * @function create - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static - * @param {vtctldata.IMaterializeSettings=} [properties] Properties to set - * @returns {vtctldata.MaterializeSettings} MaterializeSettings instance + * @param {vschema.IAutoIncrement=} [properties] Properties to set + * @returns {vschema.AutoIncrement} AutoIncrement instance */ - MaterializeSettings.create = function create(properties) { - return new MaterializeSettings(properties); + AutoIncrement.create = function create(properties) { + return new AutoIncrement(properties); }; /** - * Encodes the specified MaterializeSettings message. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * Encodes the specified AutoIncrement message. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. * @function encode - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static - * @param {vtctldata.IMaterializeSettings} message MaterializeSettings message or plain object to encode + * @param {vschema.IAutoIncrement} message AutoIncrement message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MaterializeSettings.encode = function encode(message, writer) { + AutoIncrement.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); - if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_keyspace); - if (message.target_keyspace != null && Object.hasOwnProperty.call(message, "target_keyspace")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_keyspace); - if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.stop_after_copy); - if (message.table_settings != null && message.table_settings.length) - for (let i = 0; i < message.table_settings.length; ++i) - $root.vtctldata.TableMaterializeSettings.encode(message.table_settings[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.cell); - if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.tablet_types); - if (message.external_cluster != null && Object.hasOwnProperty.call(message, "external_cluster")) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.external_cluster); - if (message.materialization_intent != null && Object.hasOwnProperty.call(message, "materialization_intent")) - writer.uint32(/* id 9, wireType 0 =*/72).int32(message.materialization_intent); - if (message.source_time_zone != null && Object.hasOwnProperty.call(message, "source_time_zone")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.source_time_zone); - if (message.target_time_zone != null && Object.hasOwnProperty.call(message, "target_time_zone")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.target_time_zone); - if (message.source_shards != null && message.source_shards.length) - for (let i = 0; i < message.source_shards.length; ++i) - writer.uint32(/* id 12, wireType 2 =*/98).string(message.source_shards[i]); - if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) - writer.uint32(/* id 13, wireType 2 =*/106).string(message.on_ddl); - if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) - writer.uint32(/* id 14, wireType 0 =*/112).bool(message.defer_secondary_keys); + if (message.column != null && Object.hasOwnProperty.call(message, "column")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.column); + if (message.sequence != null && Object.hasOwnProperty.call(message, "sequence")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.sequence); return writer; }; /** - * Encodes the specified MaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * Encodes the specified AutoIncrement message, length delimited. Does not implicitly {@link vschema.AutoIncrement.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static - * @param {vtctldata.IMaterializeSettings} message MaterializeSettings message or plain object to encode + * @param {vschema.IAutoIncrement} message AutoIncrement message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - MaterializeSettings.encodeDelimited = function encodeDelimited(message, writer) { + AutoIncrement.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a MaterializeSettings message from the specified reader or buffer. + * Decodes an AutoIncrement message from the specified reader or buffer. * @function decode - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.MaterializeSettings} MaterializeSettings + * @returns {vschema.AutoIncrement} AutoIncrement * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MaterializeSettings.decode = function decode(reader, length) { + AutoIncrement.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.MaterializeSettings(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.AutoIncrement(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.workflow = reader.string(); + message.column = reader.string(); break; } case 2: { - message.source_keyspace = reader.string(); - break; - } - case 3: { - message.target_keyspace = reader.string(); - break; - } - case 4: { - message.stop_after_copy = reader.bool(); - break; - } - case 5: { - if (!(message.table_settings && message.table_settings.length)) - message.table_settings = []; - message.table_settings.push($root.vtctldata.TableMaterializeSettings.decode(reader, reader.uint32())); - break; - } - case 6: { - message.cell = reader.string(); - break; - } - case 7: { - message.tablet_types = reader.string(); - break; - } - case 8: { - message.external_cluster = reader.string(); - break; - } - case 9: { - message.materialization_intent = reader.int32(); - break; - } - case 10: { - message.source_time_zone = reader.string(); - break; - } - case 11: { - message.target_time_zone = reader.string(); - break; - } - case 12: { - if (!(message.source_shards && message.source_shards.length)) - message.source_shards = []; - message.source_shards.push(reader.string()); - break; - } - case 13: { - message.on_ddl = reader.string(); - break; - } - case 14: { - message.defer_secondary_keys = reader.bool(); + message.sequence = reader.string(); break; } default: @@ -99322,283 +99862,132 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a MaterializeSettings message from the specified reader or buffer, length delimited. + * Decodes an AutoIncrement message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.MaterializeSettings} MaterializeSettings + * @returns {vschema.AutoIncrement} AutoIncrement * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - MaterializeSettings.decodeDelimited = function decodeDelimited(reader) { + AutoIncrement.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a MaterializeSettings message. + * Verifies an AutoIncrement message. * @function verify - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - MaterializeSettings.verify = function verify(message) { + AutoIncrement.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.workflow != null && message.hasOwnProperty("workflow")) - if (!$util.isString(message.workflow)) - return "workflow: string expected"; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - if (!$util.isString(message.source_keyspace)) - return "source_keyspace: string expected"; - if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) - if (!$util.isString(message.target_keyspace)) - return "target_keyspace: string expected"; - 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.table_settings != null && message.hasOwnProperty("table_settings")) { - if (!Array.isArray(message.table_settings)) - return "table_settings: array expected"; - for (let i = 0; i < message.table_settings.length; ++i) { - let error = $root.vtctldata.TableMaterializeSettings.verify(message.table_settings[i]); - if (error) - return "table_settings." + error; - } - } - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - if (!$util.isString(message.tablet_types)) - return "tablet_types: string expected"; - if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) - if (!$util.isString(message.external_cluster)) - return "external_cluster: string expected"; - if (message.materialization_intent != null && message.hasOwnProperty("materialization_intent")) - switch (message.materialization_intent) { - default: - return "materialization_intent: enum value expected"; - case 0: - case 1: - case 2: - break; - } - if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) - if (!$util.isString(message.source_time_zone)) - return "source_time_zone: string expected"; - if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) - if (!$util.isString(message.target_time_zone)) - return "target_time_zone: string expected"; - if (message.source_shards != null && message.hasOwnProperty("source_shards")) { - if (!Array.isArray(message.source_shards)) - return "source_shards: array expected"; - for (let i = 0; i < message.source_shards.length; ++i) - if (!$util.isString(message.source_shards[i])) - return "source_shards: string[] expected"; - } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - if (!$util.isString(message.on_ddl)) - return "on_ddl: string expected"; - 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.column != null && message.hasOwnProperty("column")) + if (!$util.isString(message.column)) + return "column: string expected"; + if (message.sequence != null && message.hasOwnProperty("sequence")) + if (!$util.isString(message.sequence)) + return "sequence: string expected"; return null; }; /** - * Creates a MaterializeSettings message from a plain object. Also converts values to their respective internal types. + * Creates an AutoIncrement message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static * @param {Object.} object Plain object - * @returns {vtctldata.MaterializeSettings} MaterializeSettings + * @returns {vschema.AutoIncrement} AutoIncrement */ - MaterializeSettings.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.MaterializeSettings) + AutoIncrement.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.AutoIncrement) return object; - let message = new $root.vtctldata.MaterializeSettings(); - if (object.workflow != null) - message.workflow = String(object.workflow); - if (object.source_keyspace != null) - message.source_keyspace = String(object.source_keyspace); - if (object.target_keyspace != null) - message.target_keyspace = String(object.target_keyspace); - if (object.stop_after_copy != null) - message.stop_after_copy = Boolean(object.stop_after_copy); - if (object.table_settings) { - if (!Array.isArray(object.table_settings)) - throw TypeError(".vtctldata.MaterializeSettings.table_settings: array expected"); - message.table_settings = []; - for (let i = 0; i < object.table_settings.length; ++i) { - if (typeof object.table_settings[i] !== "object") - throw TypeError(".vtctldata.MaterializeSettings.table_settings: object expected"); - message.table_settings[i] = $root.vtctldata.TableMaterializeSettings.fromObject(object.table_settings[i]); - } - } - if (object.cell != null) - message.cell = String(object.cell); - if (object.tablet_types != null) - message.tablet_types = String(object.tablet_types); - if (object.external_cluster != null) - message.external_cluster = String(object.external_cluster); - switch (object.materialization_intent) { - default: - if (typeof object.materialization_intent === "number") { - message.materialization_intent = object.materialization_intent; - break; - } - break; - case "CUSTOM": - case 0: - message.materialization_intent = 0; - break; - case "MOVETABLES": - case 1: - message.materialization_intent = 1; - break; - case "CREATELOOKUPINDEX": - case 2: - message.materialization_intent = 2; - break; - } - if (object.source_time_zone != null) - message.source_time_zone = String(object.source_time_zone); - if (object.target_time_zone != null) - message.target_time_zone = String(object.target_time_zone); - if (object.source_shards) { - if (!Array.isArray(object.source_shards)) - throw TypeError(".vtctldata.MaterializeSettings.source_shards: array expected"); - message.source_shards = []; - for (let i = 0; i < object.source_shards.length; ++i) - message.source_shards[i] = String(object.source_shards[i]); - } - if (object.on_ddl != null) - message.on_ddl = String(object.on_ddl); - if (object.defer_secondary_keys != null) - message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + let message = new $root.vschema.AutoIncrement(); + if (object.column != null) + message.column = String(object.column); + if (object.sequence != null) + message.sequence = String(object.sequence); return message; }; /** - * Creates a plain object from a MaterializeSettings message. Also converts values to other types if specified. + * Creates a plain object from an AutoIncrement message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static - * @param {vtctldata.MaterializeSettings} message MaterializeSettings + * @param {vschema.AutoIncrement} message AutoIncrement * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - MaterializeSettings.toObject = function toObject(message, options) { + AutoIncrement.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.table_settings = []; - object.source_shards = []; - } if (options.defaults) { - object.workflow = ""; - object.source_keyspace = ""; - object.target_keyspace = ""; - object.stop_after_copy = false; - object.cell = ""; - object.tablet_types = ""; - object.external_cluster = ""; - object.materialization_intent = options.enums === String ? "CUSTOM" : 0; - object.source_time_zone = ""; - object.target_time_zone = ""; - object.on_ddl = ""; - object.defer_secondary_keys = false; - } - if (message.workflow != null && message.hasOwnProperty("workflow")) - object.workflow = message.workflow; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - object.source_keyspace = message.source_keyspace; - if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) - object.target_keyspace = message.target_keyspace; - if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) - object.stop_after_copy = message.stop_after_copy; - if (message.table_settings && message.table_settings.length) { - object.table_settings = []; - for (let j = 0; j < message.table_settings.length; ++j) - object.table_settings[j] = $root.vtctldata.TableMaterializeSettings.toObject(message.table_settings[j], options); - } - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - object.tablet_types = message.tablet_types; - if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) - object.external_cluster = message.external_cluster; - if (message.materialization_intent != null && message.hasOwnProperty("materialization_intent")) - object.materialization_intent = options.enums === String ? $root.vtctldata.MaterializationIntent[message.materialization_intent] === undefined ? message.materialization_intent : $root.vtctldata.MaterializationIntent[message.materialization_intent] : message.materialization_intent; - if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) - object.source_time_zone = message.source_time_zone; - if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) - object.target_time_zone = message.target_time_zone; - if (message.source_shards && message.source_shards.length) { - object.source_shards = []; - for (let j = 0; j < message.source_shards.length; ++j) - object.source_shards[j] = message.source_shards[j]; + object.column = ""; + object.sequence = ""; } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - object.on_ddl = message.on_ddl; - if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) - object.defer_secondary_keys = message.defer_secondary_keys; + if (message.column != null && message.hasOwnProperty("column")) + object.column = message.column; + if (message.sequence != null && message.hasOwnProperty("sequence")) + object.sequence = message.sequence; return object; }; /** - * Converts this MaterializeSettings to JSON. + * Converts this AutoIncrement to JSON. * @function toJSON - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @instance * @returns {Object.} JSON object */ - MaterializeSettings.prototype.toJSON = function toJSON() { + AutoIncrement.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for MaterializeSettings + * Gets the default type url for AutoIncrement * @function getTypeUrl - * @memberof vtctldata.MaterializeSettings + * @memberof vschema.AutoIncrement * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - MaterializeSettings.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + AutoIncrement.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.MaterializeSettings"; + return typeUrlPrefix + "/vschema.AutoIncrement"; }; - return MaterializeSettings; + return AutoIncrement; })(); - vtctldata.Keyspace = (function() { + vschema.Column = (function() { /** - * Properties of a Keyspace. - * @memberof vtctldata - * @interface IKeyspace - * @property {string|null} [name] Keyspace name - * @property {topodata.IKeyspace|null} [keyspace] Keyspace keyspace + * Properties of a Column. + * @memberof vschema + * @interface IColumn + * @property {string|null} [name] Column name + * @property {query.Type|null} [type] Column type */ /** - * Constructs a new Keyspace. - * @memberof vtctldata - * @classdesc Represents a Keyspace. - * @implements IKeyspace + * Constructs a new Column. + * @memberof vschema + * @classdesc Represents a Column. + * @implements IColumn * @constructor - * @param {vtctldata.IKeyspace=} [properties] Properties to set + * @param {vschema.IColumn=} [properties] Properties to set */ - function Keyspace(properties) { + function Column(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -99606,80 +99995,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Keyspace name. + * Column name. * @member {string} name - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @instance */ - Keyspace.prototype.name = ""; + Column.prototype.name = ""; /** - * Keyspace keyspace. - * @member {topodata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.Keyspace + * Column type. + * @member {query.Type} type + * @memberof vschema.Column * @instance */ - Keyspace.prototype.keyspace = null; + Column.prototype.type = 0; /** - * Creates a new Keyspace instance using the specified properties. + * Creates a new Column instance using the specified properties. * @function create - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static - * @param {vtctldata.IKeyspace=} [properties] Properties to set - * @returns {vtctldata.Keyspace} Keyspace instance + * @param {vschema.IColumn=} [properties] Properties to set + * @returns {vschema.Column} Column instance */ - Keyspace.create = function create(properties) { - return new Keyspace(properties); + Column.create = function create(properties) { + return new Column(properties); }; /** - * Encodes the specified Keyspace message. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * Encodes the specified Column message. Does not implicitly {@link vschema.Column.verify|verify} messages. * @function encode - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static - * @param {vtctldata.IKeyspace} message Keyspace message or plain object to encode + * @param {vschema.IColumn} message Column message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Keyspace.encode = function encode(message, writer) { + Column.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.name != null && Object.hasOwnProperty.call(message, "name")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.type); return writer; }; /** - * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * Encodes the specified Column message, length delimited. Does not implicitly {@link vschema.Column.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static - * @param {vtctldata.IKeyspace} message Keyspace message or plain object to encode + * @param {vschema.IColumn} message Column message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Keyspace.encodeDelimited = function encodeDelimited(message, writer) { + Column.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Keyspace message from the specified reader or buffer. + * Decodes a Column message from the specified reader or buffer. * @function decode - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Keyspace} Keyspace + * @returns {vschema.Column} Column * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Keyspace.decode = function decode(reader, length) { + Column.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.Keyspace(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.Column(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -99688,7 +100077,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); + message.type = reader.int32(); break; } default: @@ -99700,138 +100089,318 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a Keyspace message from the specified reader or buffer, length delimited. + * Decodes a Column message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Keyspace} Keyspace + * @returns {vschema.Column} Column * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Keyspace.decodeDelimited = function decodeDelimited(reader) { + Column.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Keyspace message. + * Verifies a Column message. * @function verify - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Keyspace.verify = function verify(message) { + Column.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.name != null && message.hasOwnProperty("name")) if (!$util.isString(message.name)) return "name: string expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.topodata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; - } + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "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 Keyspace message from a plain object. Also converts values to their respective internal types. + * Creates a Column message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static * @param {Object.} object Plain object - * @returns {vtctldata.Keyspace} Keyspace + * @returns {vschema.Column} Column */ - Keyspace.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Keyspace) + Column.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.Column) return object; - let message = new $root.vtctldata.Keyspace(); + let message = new $root.vschema.Column(); if (object.name != null) message.name = String(object.name); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.Keyspace.keyspace: object expected"); - message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; + } + break; + case "NULL_TYPE": + case 0: + message.type = 0; + break; + case "INT8": + case 257: + message.type = 257; + break; + case "UINT8": + case 770: + message.type = 770; + break; + case "INT16": + case 259: + message.type = 259; + break; + case "UINT16": + case 772: + message.type = 772; + break; + case "INT24": + case 261: + message.type = 261; + break; + case "UINT24": + case 774: + message.type = 774; + break; + case "INT32": + case 263: + message.type = 263; + break; + case "UINT32": + case 776: + message.type = 776; + break; + case "INT64": + case 265: + message.type = 265; + break; + case "UINT64": + case 778: + message.type = 778; + break; + case "FLOAT32": + case 1035: + message.type = 1035; + break; + case "FLOAT64": + case 1036: + message.type = 1036; + break; + case "TIMESTAMP": + case 2061: + message.type = 2061; + break; + case "DATE": + case 2062: + message.type = 2062; + break; + case "TIME": + case 2063: + message.type = 2063; + break; + case "DATETIME": + case 2064: + message.type = 2064; + break; + case "YEAR": + case 785: + message.type = 785; + break; + case "DECIMAL": + case 18: + message.type = 18; + break; + case "TEXT": + case 6163: + message.type = 6163; + break; + case "BLOB": + case 10260: + message.type = 10260; + break; + case "VARCHAR": + case 6165: + message.type = 6165; + break; + case "VARBINARY": + case 10262: + message.type = 10262; + break; + case "CHAR": + case 6167: + message.type = 6167; + break; + case "BINARY": + case 10264: + message.type = 10264; + break; + case "BIT": + case 2073: + message.type = 2073; + break; + case "ENUM": + case 2074: + message.type = 2074; + break; + case "SET": + case 2075: + message.type = 2075; + break; + case "TUPLE": + case 28: + message.type = 28; + break; + case "GEOMETRY": + case 2077: + message.type = 2077; + break; + case "JSON": + case 2078: + message.type = 2078; + break; + case "EXPRESSION": + case 31: + message.type = 31; + break; + case "HEXNUM": + case 4128: + message.type = 4128; + break; + case "HEXVAL": + case 4129: + message.type = 4129; + break; + case "BITNUM": + case 4130: + message.type = 4130; + break; } return message; }; /** - * Creates a plain object from a Keyspace message. Also converts values to other types if specified. + * Creates a plain object from a Column message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static - * @param {vtctldata.Keyspace} message Keyspace + * @param {vschema.Column} message Column * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Keyspace.toObject = function toObject(message, options) { + Column.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.name = ""; - object.keyspace = null; + object.type = options.enums === String ? "NULL_TYPE" : 0; } if (message.name != null && message.hasOwnProperty("name")) object.name = message.name; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.query.Type[message.type] === undefined ? message.type : $root.query.Type[message.type] : message.type; return object; }; /** - * Converts this Keyspace to JSON. + * Converts this Column to JSON. * @function toJSON - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @instance * @returns {Object.} JSON object */ - Keyspace.prototype.toJSON = function toJSON() { + Column.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Keyspace + * Gets the default type url for Column * @function getTypeUrl - * @memberof vtctldata.Keyspace + * @memberof vschema.Column * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Keyspace.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + Column.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.Keyspace"; + return typeUrlPrefix + "/vschema.Column"; }; - return Keyspace; + return Column; })(); - vtctldata.Shard = (function() { + vschema.SrvVSchema = (function() { /** - * Properties of a Shard. - * @memberof vtctldata - * @interface IShard - * @property {string|null} [keyspace] Shard keyspace - * @property {string|null} [name] Shard name - * @property {topodata.IShard|null} [shard] Shard shard + * Properties of a SrvVSchema. + * @memberof vschema + * @interface ISrvVSchema + * @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 */ /** - * Constructs a new Shard. - * @memberof vtctldata - * @classdesc Represents a Shard. - * @implements IShard + * Constructs a new SrvVSchema. + * @memberof vschema + * @classdesc Represents a SrvVSchema. + * @implements ISrvVSchema * @constructor - * @param {vtctldata.IShard=} [properties] Properties to set + * @param {vschema.ISrvVSchema=} [properties] Properties to set */ - function Shard(properties) { + function SrvVSchema(properties) { + this.keyspaces = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -99839,103 +100408,125 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Shard keyspace. - * @member {string} keyspace - * @memberof vtctldata.Shard + * SrvVSchema keyspaces. + * @member {Object.} keyspaces + * @memberof vschema.SrvVSchema * @instance */ - Shard.prototype.keyspace = ""; + SrvVSchema.prototype.keyspaces = $util.emptyObject; /** - * Shard name. - * @member {string} name - * @memberof vtctldata.Shard + * SrvVSchema routing_rules. + * @member {vschema.IRoutingRules|null|undefined} routing_rules + * @memberof vschema.SrvVSchema * @instance */ - Shard.prototype.name = ""; + SrvVSchema.prototype.routing_rules = null; /** - * Shard shard. - * @member {topodata.IShard|null|undefined} shard - * @memberof vtctldata.Shard + * SrvVSchema shard_routing_rules. + * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules + * @memberof vschema.SrvVSchema * @instance */ - Shard.prototype.shard = null; + SrvVSchema.prototype.shard_routing_rules = null; /** - * Creates a new Shard instance using the specified properties. + * Creates a new SrvVSchema instance using the specified properties. * @function create - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static - * @param {vtctldata.IShard=} [properties] Properties to set - * @returns {vtctldata.Shard} Shard instance + * @param {vschema.ISrvVSchema=} [properties] Properties to set + * @returns {vschema.SrvVSchema} SrvVSchema instance */ - Shard.create = function create(properties) { - return new Shard(properties); + SrvVSchema.create = function create(properties) { + return new SrvVSchema(properties); }; /** - * Encodes the specified Shard message. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * Encodes the specified SrvVSchema message. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. * @function encode - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static - * @param {vtctldata.IShard} message Shard message or plain object to encode + * @param {vschema.ISrvVSchema} message SrvVSchema message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Shard.encode = function encode(message, writer) { + SrvVSchema.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.name); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.keyspaces != null && Object.hasOwnProperty.call(message, "keyspaces")) + for (let keys = Object.keys(message.keyspaces), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vschema.Keyspace.encode(message.keyspaces[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) + $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(); return writer; }; /** - * Encodes the specified Shard message, length delimited. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * Encodes the specified SrvVSchema message, length delimited. Does not implicitly {@link vschema.SrvVSchema.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static - * @param {vtctldata.IShard} message Shard message or plain object to encode + * @param {vschema.ISrvVSchema} message SrvVSchema message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Shard.encodeDelimited = function encodeDelimited(message, writer) { + SrvVSchema.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Shard message from the specified reader or buffer. + * Decodes a SrvVSchema message from the specified reader or buffer. * @function decode - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Shard} Shard + * @returns {vschema.SrvVSchema} SrvVSchema * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Shard.decode = function decode(reader, length) { + SrvVSchema.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.Shard(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.SrvVSchema(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + if (message.keyspaces === $util.emptyObject) + message.keyspaces = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vschema.Keyspace.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.keyspaces[key] = value; break; } case 2: { - message.name = reader.string(); + message.routing_rules = $root.vschema.RoutingRules.decode(reader, reader.uint32()); break; } case 3: { - message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + message.shard_routing_rules = $root.vschema.ShardRoutingRules.decode(reader, reader.uint32()); break; } default: @@ -99947,151 +100538,170 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a Shard message from the specified reader or buffer, length delimited. + * Decodes a SrvVSchema message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Shard} Shard + * @returns {vschema.SrvVSchema} SrvVSchema * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Shard.decodeDelimited = function decodeDelimited(reader) { + SrvVSchema.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Shard message. + * Verifies a SrvVSchema message. * @function verify - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Shard.verify = function verify(message) { + SrvVSchema.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.topodata.Shard.verify(message.shard); + if (message.keyspaces != null && message.hasOwnProperty("keyspaces")) { + if (!$util.isObject(message.keyspaces)) + return "keyspaces: object expected"; + let key = Object.keys(message.keyspaces); + for (let i = 0; i < key.length; ++i) { + let error = $root.vschema.Keyspace.verify(message.keyspaces[key[i]]); + if (error) + return "keyspaces." + error; + } + } + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { + let error = $root.vschema.RoutingRules.verify(message.routing_rules); if (error) - return "shard." + error; + return "routing_rules." + error; + } + if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { + let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); + if (error) + return "shard_routing_rules." + error; } return null; }; /** - * Creates a Shard message from a plain object. Also converts values to their respective internal types. + * Creates a SrvVSchema message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static * @param {Object.} object Plain object - * @returns {vtctldata.Shard} Shard + * @returns {vschema.SrvVSchema} SrvVSchema */ - Shard.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Shard) + SrvVSchema.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.SrvVSchema) return object; - let message = new $root.vtctldata.Shard(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.name != null) - message.name = String(object.name); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.Shard.shard: object expected"); - message.shard = $root.topodata.Shard.fromObject(object.shard); + let message = new $root.vschema.SrvVSchema(); + if (object.keyspaces) { + if (typeof object.keyspaces !== "object") + throw TypeError(".vschema.SrvVSchema.keyspaces: object expected"); + message.keyspaces = {}; + for (let keys = Object.keys(object.keyspaces), i = 0; i < keys.length; ++i) { + if (typeof object.keyspaces[keys[i]] !== "object") + throw TypeError(".vschema.SrvVSchema.keyspaces: object expected"); + message.keyspaces[keys[i]] = $root.vschema.Keyspace.fromObject(object.keyspaces[keys[i]]); + } + } + if (object.routing_rules != null) { + if (typeof object.routing_rules !== "object") + throw TypeError(".vschema.SrvVSchema.routing_rules: object expected"); + message.routing_rules = $root.vschema.RoutingRules.fromObject(object.routing_rules); + } + if (object.shard_routing_rules != null) { + if (typeof object.shard_routing_rules !== "object") + throw TypeError(".vschema.SrvVSchema.shard_routing_rules: object expected"); + message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_routing_rules); } return message; }; /** - * Creates a plain object from a Shard message. Also converts values to other types if specified. + * Creates a plain object from a SrvVSchema message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static - * @param {vtctldata.Shard} message Shard + * @param {vschema.SrvVSchema} message SrvVSchema * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Shard.toObject = function toObject(message, options) { + SrvVSchema.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.objects || options.defaults) + object.keyspaces = {}; if (options.defaults) { - object.keyspace = ""; - object.name = ""; - object.shard = null; + object.routing_rules = null; + object.shard_routing_rules = null; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.topodata.Shard.toObject(message.shard, options); + let keys2; + if (message.keyspaces && (keys2 = Object.keys(message.keyspaces)).length) { + object.keyspaces = {}; + for (let j = 0; j < keys2.length; ++j) + object.keyspaces[keys2[j]] = $root.vschema.Keyspace.toObject(message.keyspaces[keys2[j]], options); + } + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) + 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); return object; }; /** - * Converts this Shard to JSON. + * Converts this SrvVSchema to JSON. * @function toJSON - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @instance * @returns {Object.} JSON object */ - Shard.prototype.toJSON = function toJSON() { + SrvVSchema.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Shard + * Gets the default type url for SrvVSchema * @function getTypeUrl - * @memberof vtctldata.Shard + * @memberof vschema.SrvVSchema * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Shard.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SrvVSchema.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.Shard"; + return typeUrlPrefix + "/vschema.SrvVSchema"; }; - return Shard; + return SrvVSchema; })(); - vtctldata.Workflow = (function() { + vschema.ShardRoutingRules = (function() { /** - * Properties of a Workflow. - * @memberof vtctldata - * @interface IWorkflow - * @property {string|null} [name] Workflow name - * @property {vtctldata.Workflow.IReplicationLocation|null} [source] Workflow source - * @property {vtctldata.Workflow.IReplicationLocation|null} [target] Workflow target - * @property {number|Long|null} [max_v_replication_lag] Workflow max_v_replication_lag - * @property {Object.|null} [shard_streams] Workflow shard_streams - * @property {string|null} [workflow_type] Workflow workflow_type - * @property {string|null} [workflow_sub_type] Workflow workflow_sub_type + * Properties of a ShardRoutingRules. + * @memberof vschema + * @interface IShardRoutingRules + * @property {Array.|null} [rules] ShardRoutingRules rules */ /** - * Constructs a new Workflow. - * @memberof vtctldata - * @classdesc Represents a Workflow. - * @implements IWorkflow + * Constructs a new ShardRoutingRules. + * @memberof vschema + * @classdesc Represents a ShardRoutingRules. + * @implements IShardRoutingRules * @constructor - * @param {vtctldata.IWorkflow=} [properties] Properties to set + * @param {vschema.IShardRoutingRules=} [properties] Properties to set */ - function Workflow(properties) { - this.shard_streams = {}; + function ShardRoutingRules(properties) { + this.rules = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -100099,181 +100709,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Workflow name. - * @member {string} name - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.name = ""; - - /** - * Workflow source. - * @member {vtctldata.Workflow.IReplicationLocation|null|undefined} source - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.source = null; - - /** - * Workflow target. - * @member {vtctldata.Workflow.IReplicationLocation|null|undefined} target - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.target = null; - - /** - * Workflow max_v_replication_lag. - * @member {number|Long} max_v_replication_lag - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.max_v_replication_lag = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * Workflow shard_streams. - * @member {Object.} shard_streams - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.shard_streams = $util.emptyObject; - - /** - * Workflow workflow_type. - * @member {string} workflow_type - * @memberof vtctldata.Workflow - * @instance - */ - Workflow.prototype.workflow_type = ""; - - /** - * Workflow workflow_sub_type. - * @member {string} workflow_sub_type - * @memberof vtctldata.Workflow + * ShardRoutingRules rules. + * @member {Array.} rules + * @memberof vschema.ShardRoutingRules * @instance */ - Workflow.prototype.workflow_sub_type = ""; + ShardRoutingRules.prototype.rules = $util.emptyArray; /** - * Creates a new Workflow instance using the specified properties. + * Creates a new ShardRoutingRules instance using the specified properties. * @function create - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static - * @param {vtctldata.IWorkflow=} [properties] Properties to set - * @returns {vtctldata.Workflow} Workflow instance + * @param {vschema.IShardRoutingRules=} [properties] Properties to set + * @returns {vschema.ShardRoutingRules} ShardRoutingRules instance */ - Workflow.create = function create(properties) { - return new Workflow(properties); + ShardRoutingRules.create = function create(properties) { + return new ShardRoutingRules(properties); }; /** - * Encodes the specified Workflow message. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * Encodes the specified ShardRoutingRules message. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. * @function encode - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static - * @param {vtctldata.IWorkflow} message Workflow message or plain object to encode + * @param {vschema.IShardRoutingRules} message ShardRoutingRules message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Workflow.encode = function encode(message, writer) { + ShardRoutingRules.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.source != null && Object.hasOwnProperty.call(message, "source")) - $root.vtctldata.Workflow.ReplicationLocation.encode(message.source, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.target != null && Object.hasOwnProperty.call(message, "target")) - $root.vtctldata.Workflow.ReplicationLocation.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.max_v_replication_lag != null && Object.hasOwnProperty.call(message, "max_v_replication_lag")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.max_v_replication_lag); - if (message.shard_streams != null && Object.hasOwnProperty.call(message, "shard_streams")) - for (let keys = Object.keys(message.shard_streams), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 5, wireType 2 =*/42).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.Workflow.ShardStream.encode(message.shard_streams[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.workflow_type); - if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.workflow_sub_type); + if (message.rules != null && message.rules.length) + for (let i = 0; i < message.rules.length; ++i) + $root.vschema.ShardRoutingRule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified Workflow message, length delimited. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * Encodes the specified ShardRoutingRules message, length delimited. Does not implicitly {@link vschema.ShardRoutingRules.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static - * @param {vtctldata.IWorkflow} message Workflow message or plain object to encode + * @param {vschema.IShardRoutingRules} message ShardRoutingRules message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - Workflow.encodeDelimited = function encodeDelimited(message, writer) { + ShardRoutingRules.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a Workflow message from the specified reader or buffer. + * Decodes a ShardRoutingRules message from the specified reader or buffer. * @function decode - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow} Workflow + * @returns {vschema.ShardRoutingRules} ShardRoutingRules * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Workflow.decode = function decode(reader, length) { + ShardRoutingRules.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.Workflow(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vschema.ShardRoutingRules(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); - break; - } - case 2: { - message.source = $root.vtctldata.Workflow.ReplicationLocation.decode(reader, reader.uint32()); - break; - } - case 3: { - message.target = $root.vtctldata.Workflow.ReplicationLocation.decode(reader, reader.uint32()); - break; - } - case 4: { - message.max_v_replication_lag = reader.int64(); - break; - } - case 5: { - if (message.shard_streams === $util.emptyObject) - message.shard_streams = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vtctldata.Workflow.ShardStream.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.shard_streams[key] = value; - break; - } - case 6: { - message.workflow_type = reader.string(); - break; - } - case 7: { - message.workflow_sub_type = reader.string(); + if (!(message.rules && message.rules.length)) + message.rules = []; + message.rules.push($root.vschema.ShardRoutingRule.decode(reader, reader.uint32())); break; } default: @@ -100285,1057 +100792,3483 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a Workflow message from the specified reader or buffer, length delimited. + * Decodes a ShardRoutingRules message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Workflow} Workflow + * @returns {vschema.ShardRoutingRules} ShardRoutingRules * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - Workflow.decodeDelimited = function decodeDelimited(reader) { + ShardRoutingRules.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a Workflow message. + * Verifies a ShardRoutingRules message. * @function verify - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - Workflow.verify = function verify(message) { + ShardRoutingRules.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.source != null && message.hasOwnProperty("source")) { - let error = $root.vtctldata.Workflow.ReplicationLocation.verify(message.source); - if (error) - return "source." + error; - } - if (message.target != null && message.hasOwnProperty("target")) { - let error = $root.vtctldata.Workflow.ReplicationLocation.verify(message.target); - if (error) - return "target." + error; - } - if (message.max_v_replication_lag != null && message.hasOwnProperty("max_v_replication_lag")) - if (!$util.isInteger(message.max_v_replication_lag) && !(message.max_v_replication_lag && $util.isInteger(message.max_v_replication_lag.low) && $util.isInteger(message.max_v_replication_lag.high))) - return "max_v_replication_lag: integer|Long expected"; - if (message.shard_streams != null && message.hasOwnProperty("shard_streams")) { - if (!$util.isObject(message.shard_streams)) - return "shard_streams: object expected"; - let key = Object.keys(message.shard_streams); - for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.Workflow.ShardStream.verify(message.shard_streams[key[i]]); + 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.ShardRoutingRule.verify(message.rules[i]); if (error) - return "shard_streams." + error; + return "rules." + error; } } - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - if (!$util.isString(message.workflow_type)) - return "workflow_type: string expected"; - if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) - if (!$util.isString(message.workflow_sub_type)) - return "workflow_sub_type: string expected"; return null; }; /** - * Creates a Workflow message from a plain object. Also converts values to their respective internal types. + * Creates a ShardRoutingRules message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static * @param {Object.} object Plain object - * @returns {vtctldata.Workflow} Workflow + * @returns {vschema.ShardRoutingRules} ShardRoutingRules */ - Workflow.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow) + ShardRoutingRules.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.ShardRoutingRules) return object; - let message = new $root.vtctldata.Workflow(); - if (object.name != null) - message.name = String(object.name); - if (object.source != null) { - if (typeof object.source !== "object") - throw TypeError(".vtctldata.Workflow.source: object expected"); - message.source = $root.vtctldata.Workflow.ReplicationLocation.fromObject(object.source); - } - if (object.target != null) { - if (typeof object.target !== "object") - throw TypeError(".vtctldata.Workflow.target: object expected"); - message.target = $root.vtctldata.Workflow.ReplicationLocation.fromObject(object.target); - } - if (object.max_v_replication_lag != null) - if ($util.Long) - (message.max_v_replication_lag = $util.Long.fromValue(object.max_v_replication_lag)).unsigned = false; - else if (typeof object.max_v_replication_lag === "string") - message.max_v_replication_lag = parseInt(object.max_v_replication_lag, 10); - else if (typeof object.max_v_replication_lag === "number") - message.max_v_replication_lag = object.max_v_replication_lag; - else if (typeof object.max_v_replication_lag === "object") - message.max_v_replication_lag = new $util.LongBits(object.max_v_replication_lag.low >>> 0, object.max_v_replication_lag.high >>> 0).toNumber(); - if (object.shard_streams) { - if (typeof object.shard_streams !== "object") - throw TypeError(".vtctldata.Workflow.shard_streams: object expected"); - message.shard_streams = {}; - for (let keys = Object.keys(object.shard_streams), i = 0; i < keys.length; ++i) { - if (typeof object.shard_streams[keys[i]] !== "object") - throw TypeError(".vtctldata.Workflow.shard_streams: object expected"); - message.shard_streams[keys[i]] = $root.vtctldata.Workflow.ShardStream.fromObject(object.shard_streams[keys[i]]); + let message = new $root.vschema.ShardRoutingRules(); + if (object.rules) { + if (!Array.isArray(object.rules)) + throw TypeError(".vschema.ShardRoutingRules.rules: array expected"); + message.rules = []; + for (let i = 0; i < object.rules.length; ++i) { + if (typeof object.rules[i] !== "object") + throw TypeError(".vschema.ShardRoutingRules.rules: object expected"); + message.rules[i] = $root.vschema.ShardRoutingRule.fromObject(object.rules[i]); } } - if (object.workflow_type != null) - message.workflow_type = String(object.workflow_type); - if (object.workflow_sub_type != null) - message.workflow_sub_type = String(object.workflow_sub_type); return message; }; /** - * Creates a plain object from a Workflow message. Also converts values to other types if specified. + * Creates a plain object from a ShardRoutingRules message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static - * @param {vtctldata.Workflow} message Workflow + * @param {vschema.ShardRoutingRules} message ShardRoutingRules * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - Workflow.toObject = function toObject(message, options) { + ShardRoutingRules.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.shard_streams = {}; - if (options.defaults) { - object.name = ""; - object.source = null; - object.target = null; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_v_replication_lag = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_v_replication_lag = options.longs === String ? "0" : 0; - object.workflow_type = ""; - object.workflow_sub_type = ""; - } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.source != null && message.hasOwnProperty("source")) - object.source = $root.vtctldata.Workflow.ReplicationLocation.toObject(message.source, options); - if (message.target != null && message.hasOwnProperty("target")) - object.target = $root.vtctldata.Workflow.ReplicationLocation.toObject(message.target, options); - if (message.max_v_replication_lag != null && message.hasOwnProperty("max_v_replication_lag")) - if (typeof message.max_v_replication_lag === "number") - object.max_v_replication_lag = options.longs === String ? String(message.max_v_replication_lag) : message.max_v_replication_lag; - else - object.max_v_replication_lag = options.longs === String ? $util.Long.prototype.toString.call(message.max_v_replication_lag) : options.longs === Number ? new $util.LongBits(message.max_v_replication_lag.low >>> 0, message.max_v_replication_lag.high >>> 0).toNumber() : message.max_v_replication_lag; - let keys2; - if (message.shard_streams && (keys2 = Object.keys(message.shard_streams)).length) { - object.shard_streams = {}; - for (let j = 0; j < keys2.length; ++j) - object.shard_streams[keys2[j]] = $root.vtctldata.Workflow.ShardStream.toObject(message.shard_streams[keys2[j]], options); + 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.ShardRoutingRule.toObject(message.rules[j], options); } - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - object.workflow_type = message.workflow_type; - if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) - object.workflow_sub_type = message.workflow_sub_type; return object; }; /** - * Converts this Workflow to JSON. + * Converts this ShardRoutingRules to JSON. * @function toJSON - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @instance * @returns {Object.} JSON object */ - Workflow.prototype.toJSON = function toJSON() { + ShardRoutingRules.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for Workflow + * Gets the default type url for ShardRoutingRules * @function getTypeUrl - * @memberof vtctldata.Workflow + * @memberof vschema.ShardRoutingRules * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - Workflow.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardRoutingRules.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.Workflow"; + return typeUrlPrefix + "/vschema.ShardRoutingRules"; }; - Workflow.ReplicationLocation = (function() { + return ShardRoutingRules; + })(); - /** - * Properties of a ReplicationLocation. - * @memberof vtctldata.Workflow - * @interface IReplicationLocation - * @property {string|null} [keyspace] ReplicationLocation keyspace - * @property {Array.|null} [shards] ReplicationLocation shards - */ + vschema.ShardRoutingRule = (function() { - /** - * Constructs a new ReplicationLocation. - * @memberof vtctldata.Workflow - * @classdesc Represents a ReplicationLocation. - * @implements IReplicationLocation - * @constructor - * @param {vtctldata.Workflow.IReplicationLocation=} [properties] Properties to set - */ - function ReplicationLocation(properties) { - this.shards = []; - 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]]; - } + /** + * Properties of a ShardRoutingRule. + * @memberof vschema + * @interface IShardRoutingRule + * @property {string|null} [from_keyspace] ShardRoutingRule from_keyspace + * @property {string|null} [to_keyspace] ShardRoutingRule to_keyspace + * @property {string|null} [shard] ShardRoutingRule shard + */ - /** - * ReplicationLocation keyspace. - * @member {string} keyspace - * @memberof vtctldata.Workflow.ReplicationLocation - * @instance - */ - ReplicationLocation.prototype.keyspace = ""; + /** + * Constructs a new ShardRoutingRule. + * @memberof vschema + * @classdesc Represents a ShardRoutingRule. + * @implements IShardRoutingRule + * @constructor + * @param {vschema.IShardRoutingRule=} [properties] Properties to set + */ + function ShardRoutingRule(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]]; + } - /** - * ReplicationLocation shards. - * @member {Array.} shards - * @memberof vtctldata.Workflow.ReplicationLocation - * @instance - */ - ReplicationLocation.prototype.shards = $util.emptyArray; + /** + * ShardRoutingRule from_keyspace. + * @member {string} from_keyspace + * @memberof vschema.ShardRoutingRule + * @instance + */ + ShardRoutingRule.prototype.from_keyspace = ""; - /** - * Creates a new ReplicationLocation instance using the specified properties. - * @function create - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {vtctldata.Workflow.IReplicationLocation=} [properties] Properties to set - * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation instance - */ - ReplicationLocation.create = function create(properties) { - return new ReplicationLocation(properties); - }; + /** + * ShardRoutingRule to_keyspace. + * @member {string} to_keyspace + * @memberof vschema.ShardRoutingRule + * @instance + */ + ShardRoutingRule.prototype.to_keyspace = ""; - /** - * Encodes the specified ReplicationLocation message. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. - * @function encode - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {vtctldata.Workflow.IReplicationLocation} message ReplicationLocation message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ReplicationLocation.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shards != null && message.shards.length) - for (let i = 0; i < message.shards.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shards[i]); - return writer; - }; + /** + * ShardRoutingRule shard. + * @member {string} shard + * @memberof vschema.ShardRoutingRule + * @instance + */ + ShardRoutingRule.prototype.shard = ""; - /** - * Encodes the specified ReplicationLocation message, length delimited. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {vtctldata.Workflow.IReplicationLocation} message ReplicationLocation message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ReplicationLocation.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * Creates a new ShardRoutingRule instance using the specified properties. + * @function create + * @memberof vschema.ShardRoutingRule + * @static + * @param {vschema.IShardRoutingRule=} [properties] Properties to set + * @returns {vschema.ShardRoutingRule} ShardRoutingRule instance + */ + ShardRoutingRule.create = function create(properties) { + return new ShardRoutingRule(properties); + }; - /** - * Decodes a ReplicationLocation message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ReplicationLocation.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.Workflow.ReplicationLocation(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - if (!(message.shards && message.shards.length)) - message.shards = []; - message.shards.push(reader.string()); - break; - } - default: - reader.skipType(tag & 7); + /** + * Encodes the specified ShardRoutingRule message. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * @function encode + * @memberof vschema.ShardRoutingRule + * @static + * @param {vschema.IShardRoutingRule} message ShardRoutingRule message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardRoutingRule.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); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); + return writer; + }; + + /** + * Encodes the specified ShardRoutingRule message, length delimited. Does not implicitly {@link vschema.ShardRoutingRule.verify|verify} messages. + * @function encodeDelimited + * @memberof vschema.ShardRoutingRule + * @static + * @param {vschema.IShardRoutingRule} message ShardRoutingRule message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardRoutingRule.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ShardRoutingRule message from the specified reader or buffer. + * @function decode + * @memberof vschema.ShardRoutingRule + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vschema.ShardRoutingRule} ShardRoutingRule + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardRoutingRule.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.ShardRoutingRule(); + 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; + } + case 3: { + message.shard = reader.string(); break; } + default: + reader.skipType(tag & 7); + break; } - return message; - }; - - /** - * Decodes a ReplicationLocation message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ReplicationLocation.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + } + return message; + }; - /** - * Verifies a ReplicationLocation message. - * @function verify - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - ReplicationLocation.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!Array.isArray(message.shards)) - return "shards: array expected"; - for (let i = 0; i < message.shards.length; ++i) - if (!$util.isString(message.shards[i])) - return "shards: string[] expected"; - } - return null; - }; + /** + * Decodes a ShardRoutingRule message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vschema.ShardRoutingRule + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vschema.ShardRoutingRule} ShardRoutingRule + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardRoutingRule.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Creates a ReplicationLocation message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation - */ - ReplicationLocation.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow.ReplicationLocation) - return object; - let message = new $root.vtctldata.Workflow.ReplicationLocation(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shards) { - if (!Array.isArray(object.shards)) - throw TypeError(".vtctldata.Workflow.ReplicationLocation.shards: array expected"); - message.shards = []; - for (let i = 0; i < object.shards.length; ++i) - message.shards[i] = String(object.shards[i]); - } - return message; - }; + /** + * Verifies a ShardRoutingRule message. + * @function verify + * @memberof vschema.ShardRoutingRule + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ShardRoutingRule.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"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + return null; + }; - /** - * Creates a plain object from a ReplicationLocation message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {vtctldata.Workflow.ReplicationLocation} message ReplicationLocation - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - ReplicationLocation.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.shards = []; - if (options.defaults) - object.keyspace = ""; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shards && message.shards.length) { - object.shards = []; - for (let j = 0; j < message.shards.length; ++j) - object.shards[j] = message.shards[j]; - } + /** + * Creates a ShardRoutingRule message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vschema.ShardRoutingRule + * @static + * @param {Object.} object Plain object + * @returns {vschema.ShardRoutingRule} ShardRoutingRule + */ + ShardRoutingRule.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.ShardRoutingRule) return object; - }; - - /** - * Converts this ReplicationLocation to JSON. - * @function toJSON - * @memberof vtctldata.Workflow.ReplicationLocation - * @instance - * @returns {Object.} JSON object - */ - ReplicationLocation.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + let message = new $root.vschema.ShardRoutingRule(); + if (object.from_keyspace != null) + message.from_keyspace = String(object.from_keyspace); + if (object.to_keyspace != null) + message.to_keyspace = String(object.to_keyspace); + if (object.shard != null) + message.shard = String(object.shard); + return message; + }; - /** - * Gets the default type url for ReplicationLocation - * @function getTypeUrl - * @memberof vtctldata.Workflow.ReplicationLocation - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - ReplicationLocation.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.Workflow.ReplicationLocation"; - }; + /** + * Creates a plain object from a ShardRoutingRule message. Also converts values to other types if specified. + * @function toObject + * @memberof vschema.ShardRoutingRule + * @static + * @param {vschema.ShardRoutingRule} message ShardRoutingRule + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ShardRoutingRule.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.from_keyspace = ""; + object.to_keyspace = ""; + object.shard = ""; + } + 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; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + return object; + }; - return ReplicationLocation; - })(); + /** + * Converts this ShardRoutingRule to JSON. + * @function toJSON + * @memberof vschema.ShardRoutingRule + * @instance + * @returns {Object.} JSON object + */ + ShardRoutingRule.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - Workflow.ShardStream = (function() { + /** + * Gets the default type url for ShardRoutingRule + * @function getTypeUrl + * @memberof vschema.ShardRoutingRule + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ShardRoutingRule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vschema.ShardRoutingRule"; + }; - /** - * Properties of a ShardStream. - * @memberof vtctldata.Workflow - * @interface IShardStream - * @property {Array.|null} [streams] ShardStream streams - * @property {Array.|null} [tablet_controls] ShardStream tablet_controls - * @property {boolean|null} [is_primary_serving] ShardStream is_primary_serving - */ + return ShardRoutingRule; + })(); - /** - * Constructs a new ShardStream. - * @memberof vtctldata.Workflow - * @classdesc Represents a ShardStream. - * @implements IShardStream - * @constructor - * @param {vtctldata.Workflow.IShardStream=} [properties] Properties to set - */ - function ShardStream(properties) { - this.streams = []; - this.tablet_controls = []; - 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]]; - } + return vschema; +})(); - /** - * ShardStream streams. - * @member {Array.} streams - * @memberof vtctldata.Workflow.ShardStream - * @instance - */ - ShardStream.prototype.streams = $util.emptyArray; +export const vtctldata = $root.vtctldata = (() => { - /** - * ShardStream tablet_controls. - * @member {Array.} tablet_controls - * @memberof vtctldata.Workflow.ShardStream - * @instance - */ - ShardStream.prototype.tablet_controls = $util.emptyArray; + /** + * Namespace vtctldata. + * @exports vtctldata + * @namespace + */ + const vtctldata = {}; - /** - * ShardStream is_primary_serving. - * @member {boolean} is_primary_serving - * @memberof vtctldata.Workflow.ShardStream - * @instance - */ - ShardStream.prototype.is_primary_serving = false; + vtctldata.ExecuteVtctlCommandRequest = (function() { - /** - * Creates a new ShardStream instance using the specified properties. - * @function create - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {vtctldata.Workflow.IShardStream=} [properties] Properties to set - * @returns {vtctldata.Workflow.ShardStream} ShardStream instance - */ - ShardStream.create = function create(properties) { - return new ShardStream(properties); - }; + /** + * Properties of an ExecuteVtctlCommandRequest. + * @memberof vtctldata + * @interface IExecuteVtctlCommandRequest + * @property {Array.|null} [args] ExecuteVtctlCommandRequest args + * @property {number|Long|null} [action_timeout] ExecuteVtctlCommandRequest action_timeout + */ - /** - * Encodes the specified ShardStream message. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. - * @function encode - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {vtctldata.Workflow.IShardStream} message ShardStream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ShardStream.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.streams != null && message.streams.length) - for (let i = 0; i < message.streams.length; ++i) - $root.vtctldata.Workflow.Stream.encode(message.streams[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.tablet_controls != null && message.tablet_controls.length) - for (let i = 0; i < message.tablet_controls.length; ++i) - $root.topodata.Shard.TabletControl.encode(message.tablet_controls[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.is_primary_serving != null && Object.hasOwnProperty.call(message, "is_primary_serving")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.is_primary_serving); - return writer; - }; + /** + * Constructs a new ExecuteVtctlCommandRequest. + * @memberof vtctldata + * @classdesc Represents an ExecuteVtctlCommandRequest. + * @implements IExecuteVtctlCommandRequest + * @constructor + * @param {vtctldata.IExecuteVtctlCommandRequest=} [properties] Properties to set + */ + function ExecuteVtctlCommandRequest(properties) { + this.args = []; + 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]]; + } - /** - * Encodes the specified ShardStream message, length delimited. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {vtctldata.Workflow.IShardStream} message ShardStream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ShardStream.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * ExecuteVtctlCommandRequest args. + * @member {Array.} args + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @instance + */ + ExecuteVtctlCommandRequest.prototype.args = $util.emptyArray; - /** - * Decodes a ShardStream message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow.ShardStream} ShardStream - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ShardStream.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.Workflow.ShardStream(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - if (!(message.streams && message.streams.length)) - message.streams = []; - message.streams.push($root.vtctldata.Workflow.Stream.decode(reader, reader.uint32())); - break; - } - case 2: { - if (!(message.tablet_controls && message.tablet_controls.length)) - message.tablet_controls = []; - message.tablet_controls.push($root.topodata.Shard.TabletControl.decode(reader, reader.uint32())); - break; - } - case 3: { - message.is_primary_serving = reader.bool(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; + /** + * ExecuteVtctlCommandRequest action_timeout. + * @member {number|Long} action_timeout + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @instance + */ + ExecuteVtctlCommandRequest.prototype.action_timeout = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * Decodes a ShardStream message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Workflow.ShardStream} ShardStream - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ShardStream.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + /** + * Creates a new ExecuteVtctlCommandRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {vtctldata.IExecuteVtctlCommandRequest=} [properties] Properties to set + * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest instance + */ + ExecuteVtctlCommandRequest.create = function create(properties) { + return new ExecuteVtctlCommandRequest(properties); + }; - /** - * Verifies a ShardStream message. - * @function verify - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - ShardStream.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.streams != null && message.hasOwnProperty("streams")) { - if (!Array.isArray(message.streams)) - return "streams: array expected"; - for (let i = 0; i < message.streams.length; ++i) { - let error = $root.vtctldata.Workflow.Stream.verify(message.streams[i]); - if (error) - return "streams." + error; - } - } - if (message.tablet_controls != null && message.hasOwnProperty("tablet_controls")) { - if (!Array.isArray(message.tablet_controls)) - return "tablet_controls: array expected"; - for (let i = 0; i < message.tablet_controls.length; ++i) { - let error = $root.topodata.Shard.TabletControl.verify(message.tablet_controls[i]); - if (error) - return "tablet_controls." + error; - } - } - if (message.is_primary_serving != null && message.hasOwnProperty("is_primary_serving")) - if (typeof message.is_primary_serving !== "boolean") - return "is_primary_serving: boolean expected"; - return null; - }; + /** + * Encodes the specified ExecuteVtctlCommandRequest message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {vtctldata.IExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecuteVtctlCommandRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.args != null && message.args.length) + for (let i = 0; i < message.args.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.args[i]); + if (message.action_timeout != null && Object.hasOwnProperty.call(message, "action_timeout")) + writer.uint32(/* id 2, wireType 0 =*/16).int64(message.action_timeout); + return writer; + }; - /** - * Creates a ShardStream message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.Workflow.ShardStream} ShardStream - */ - ShardStream.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow.ShardStream) - return object; - let message = new $root.vtctldata.Workflow.ShardStream(); - if (object.streams) { - if (!Array.isArray(object.streams)) - throw TypeError(".vtctldata.Workflow.ShardStream.streams: array expected"); - message.streams = []; - for (let i = 0; i < object.streams.length; ++i) { - if (typeof object.streams[i] !== "object") - throw TypeError(".vtctldata.Workflow.ShardStream.streams: object expected"); - message.streams[i] = $root.vtctldata.Workflow.Stream.fromObject(object.streams[i]); + /** + * Encodes the specified ExecuteVtctlCommandRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {vtctldata.IExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecuteVtctlCommandRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecuteVtctlCommandRequest.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.ExecuteVtctlCommandRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.args && message.args.length)) + message.args = []; + message.args.push(reader.string()); + break; } - } - if (object.tablet_controls) { - if (!Array.isArray(object.tablet_controls)) - throw TypeError(".vtctldata.Workflow.ShardStream.tablet_controls: array expected"); - message.tablet_controls = []; - for (let i = 0; i < object.tablet_controls.length; ++i) { - if (typeof object.tablet_controls[i] !== "object") - throw TypeError(".vtctldata.Workflow.ShardStream.tablet_controls: object expected"); - message.tablet_controls[i] = $root.topodata.Shard.TabletControl.fromObject(object.tablet_controls[i]); + case 2: { + message.action_timeout = reader.int64(); + break; } + default: + reader.skipType(tag & 7); + break; } - if (object.is_primary_serving != null) - message.is_primary_serving = Boolean(object.is_primary_serving); - return message; - }; - - /** - * Creates a plain object from a ShardStream message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {vtctldata.Workflow.ShardStream} message ShardStream - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - ShardStream.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) { - object.streams = []; - object.tablet_controls = []; - } - if (options.defaults) - object.is_primary_serving = false; - if (message.streams && message.streams.length) { - object.streams = []; - for (let j = 0; j < message.streams.length; ++j) - object.streams[j] = $root.vtctldata.Workflow.Stream.toObject(message.streams[j], options); - } - if (message.tablet_controls && message.tablet_controls.length) { - object.tablet_controls = []; - for (let j = 0; j < message.tablet_controls.length; ++j) - object.tablet_controls[j] = $root.topodata.Shard.TabletControl.toObject(message.tablet_controls[j], options); - } - if (message.is_primary_serving != null && message.hasOwnProperty("is_primary_serving")) - object.is_primary_serving = message.is_primary_serving; - return object; - }; + } + return message; + }; - /** - * Converts this ShardStream to JSON. - * @function toJSON - * @memberof vtctldata.Workflow.ShardStream - * @instance - * @returns {Object.} JSON object - */ - ShardStream.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Decodes an ExecuteVtctlCommandRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecuteVtctlCommandRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Gets the default type url for ShardStream - * @function getTypeUrl - * @memberof vtctldata.Workflow.ShardStream - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - ShardStream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.Workflow.ShardStream"; - }; + /** + * Verifies an ExecuteVtctlCommandRequest message. + * @function verify + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ExecuteVtctlCommandRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.args != null && message.hasOwnProperty("args")) { + if (!Array.isArray(message.args)) + return "args: array expected"; + for (let i = 0; i < message.args.length; ++i) + if (!$util.isString(message.args[i])) + return "args: string[] expected"; + } + if (message.action_timeout != null && message.hasOwnProperty("action_timeout")) + if (!$util.isInteger(message.action_timeout) && !(message.action_timeout && $util.isInteger(message.action_timeout.low) && $util.isInteger(message.action_timeout.high))) + return "action_timeout: integer|Long expected"; + return null; + }; - return ShardStream; - })(); + /** + * Creates an ExecuteVtctlCommandRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ExecuteVtctlCommandRequest} ExecuteVtctlCommandRequest + */ + ExecuteVtctlCommandRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteVtctlCommandRequest) + return object; + let message = new $root.vtctldata.ExecuteVtctlCommandRequest(); + if (object.args) { + if (!Array.isArray(object.args)) + throw TypeError(".vtctldata.ExecuteVtctlCommandRequest.args: array expected"); + message.args = []; + for (let i = 0; i < object.args.length; ++i) + message.args[i] = String(object.args[i]); + } + if (object.action_timeout != null) + if ($util.Long) + (message.action_timeout = $util.Long.fromValue(object.action_timeout)).unsigned = false; + else if (typeof object.action_timeout === "string") + message.action_timeout = parseInt(object.action_timeout, 10); + else if (typeof object.action_timeout === "number") + message.action_timeout = object.action_timeout; + else if (typeof object.action_timeout === "object") + message.action_timeout = new $util.LongBits(object.action_timeout.low >>> 0, object.action_timeout.high >>> 0).toNumber(); + return message; + }; - Workflow.Stream = (function() { + /** + * Creates a plain object from an ExecuteVtctlCommandRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {vtctldata.ExecuteVtctlCommandRequest} message ExecuteVtctlCommandRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ExecuteVtctlCommandRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.args = []; + if (options.defaults) + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.action_timeout = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.action_timeout = options.longs === String ? "0" : 0; + if (message.args && message.args.length) { + object.args = []; + for (let j = 0; j < message.args.length; ++j) + object.args[j] = message.args[j]; + } + if (message.action_timeout != null && message.hasOwnProperty("action_timeout")) + if (typeof message.action_timeout === "number") + object.action_timeout = options.longs === String ? String(message.action_timeout) : message.action_timeout; + else + object.action_timeout = options.longs === String ? $util.Long.prototype.toString.call(message.action_timeout) : options.longs === Number ? new $util.LongBits(message.action_timeout.low >>> 0, message.action_timeout.high >>> 0).toNumber() : message.action_timeout; + return object; + }; - /** - * Properties of a Stream. - * @memberof vtctldata.Workflow - * @interface IStream - * @property {number|Long|null} [id] Stream id - * @property {string|null} [shard] Stream shard - * @property {topodata.ITabletAlias|null} [tablet] Stream tablet - * @property {binlogdata.IBinlogSource|null} [binlog_source] Stream binlog_source - * @property {string|null} [position] Stream position - * @property {string|null} [stop_position] Stream stop_position - * @property {string|null} [state] Stream state - * @property {string|null} [db_name] Stream db_name - * @property {vttime.ITime|null} [transaction_timestamp] Stream transaction_timestamp - * @property {vttime.ITime|null} [time_updated] Stream time_updated - * @property {string|null} [message] Stream message - * @property {Array.|null} [copy_states] Stream copy_states - * @property {Array.|null} [logs] Stream logs - * @property {string|null} [log_fetch_error] Stream log_fetch_error - * @property {Array.|null} [tags] Stream tags - */ + /** + * Converts this ExecuteVtctlCommandRequest to JSON. + * @function toJSON + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @instance + * @returns {Object.} JSON object + */ + ExecuteVtctlCommandRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * Constructs a new Stream. - * @memberof vtctldata.Workflow - * @classdesc Represents a Stream. - * @implements IStream - * @constructor - * @param {vtctldata.Workflow.IStream=} [properties] Properties to set - */ - function Stream(properties) { - this.copy_states = []; - this.logs = []; - this.tags = []; - 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]]; + /** + * Gets the default type url for ExecuteVtctlCommandRequest + * @function getTypeUrl + * @memberof vtctldata.ExecuteVtctlCommandRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ExecuteVtctlCommandRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; } + return typeUrlPrefix + "/vtctldata.ExecuteVtctlCommandRequest"; + }; - /** - * Stream id. - * @member {number|Long} id - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + return ExecuteVtctlCommandRequest; + })(); - /** - * Stream shard. - * @member {string} shard - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.shard = ""; + vtctldata.ExecuteVtctlCommandResponse = (function() { - /** - * Stream tablet. - * @member {topodata.ITabletAlias|null|undefined} tablet - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.tablet = null; + /** + * Properties of an ExecuteVtctlCommandResponse. + * @memberof vtctldata + * @interface IExecuteVtctlCommandResponse + * @property {logutil.IEvent|null} [event] ExecuteVtctlCommandResponse event + */ - /** - * Stream binlog_source. - * @member {binlogdata.IBinlogSource|null|undefined} binlog_source - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.binlog_source = null; + /** + * Constructs a new ExecuteVtctlCommandResponse. + * @memberof vtctldata + * @classdesc Represents an ExecuteVtctlCommandResponse. + * @implements IExecuteVtctlCommandResponse + * @constructor + * @param {vtctldata.IExecuteVtctlCommandResponse=} [properties] Properties to set + */ + function ExecuteVtctlCommandResponse(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]]; + } - /** - * Stream position. - * @member {string} position - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.position = ""; + /** + * ExecuteVtctlCommandResponse event. + * @member {logutil.IEvent|null|undefined} event + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @instance + */ + ExecuteVtctlCommandResponse.prototype.event = null; - /** - * Stream stop_position. - * @member {string} stop_position - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.stop_position = ""; + /** + * Creates a new ExecuteVtctlCommandResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {vtctldata.IExecuteVtctlCommandResponse=} [properties] Properties to set + * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse instance + */ + ExecuteVtctlCommandResponse.create = function create(properties) { + return new ExecuteVtctlCommandResponse(properties); + }; - /** - * Stream state. - * @member {string} state - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.state = ""; + /** + * Encodes the specified ExecuteVtctlCommandResponse message. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {vtctldata.IExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecuteVtctlCommandResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.event != null && Object.hasOwnProperty.call(message, "event")) + $root.logutil.Event.encode(message.event, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; - /** - * Stream db_name. - * @member {string} db_name - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.db_name = ""; + /** + * Encodes the specified ExecuteVtctlCommandResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteVtctlCommandResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {vtctldata.IExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecuteVtctlCommandResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * Stream transaction_timestamp. - * @member {vttime.ITime|null|undefined} transaction_timestamp - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.transaction_timestamp = null; + /** + * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecuteVtctlCommandResponse.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.ExecuteVtctlCommandResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.event = $root.logutil.Event.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; - /** - * Stream time_updated. - * @member {vttime.ITime|null|undefined} time_updated - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.time_updated = null; + /** + * Decodes an ExecuteVtctlCommandResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecuteVtctlCommandResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Stream message. - * @member {string} message - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.message = ""; + /** + * Verifies an ExecuteVtctlCommandResponse message. + * @function verify + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ExecuteVtctlCommandResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.event != null && message.hasOwnProperty("event")) { + let error = $root.logutil.Event.verify(message.event); + if (error) + return "event." + error; + } + return null; + }; - /** - * Stream copy_states. - * @member {Array.} copy_states - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.copy_states = $util.emptyArray; + /** + * Creates an ExecuteVtctlCommandResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ExecuteVtctlCommandResponse} ExecuteVtctlCommandResponse + */ + ExecuteVtctlCommandResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteVtctlCommandResponse) + return object; + let message = new $root.vtctldata.ExecuteVtctlCommandResponse(); + if (object.event != null) { + if (typeof object.event !== "object") + throw TypeError(".vtctldata.ExecuteVtctlCommandResponse.event: object expected"); + message.event = $root.logutil.Event.fromObject(object.event); + } + return message; + }; - /** - * Stream logs. - * @member {Array.} logs - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.logs = $util.emptyArray; + /** + * Creates a plain object from an ExecuteVtctlCommandResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {vtctldata.ExecuteVtctlCommandResponse} message ExecuteVtctlCommandResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ExecuteVtctlCommandResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.event = null; + if (message.event != null && message.hasOwnProperty("event")) + object.event = $root.logutil.Event.toObject(message.event, options); + return object; + }; - /** - * Stream log_fetch_error. - * @member {string} log_fetch_error - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.log_fetch_error = ""; + /** + * Converts this ExecuteVtctlCommandResponse to JSON. + * @function toJSON + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @instance + * @returns {Object.} JSON object + */ + ExecuteVtctlCommandResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * Stream tags. - * @member {Array.} tags - * @memberof vtctldata.Workflow.Stream - * @instance - */ - Stream.prototype.tags = $util.emptyArray; + /** + * Gets the default type url for ExecuteVtctlCommandResponse + * @function getTypeUrl + * @memberof vtctldata.ExecuteVtctlCommandResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ExecuteVtctlCommandResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ExecuteVtctlCommandResponse"; + }; - /** - * Creates a new Stream instance using the specified properties. - * @function create - * @memberof vtctldata.Workflow.Stream - * @static - * @param {vtctldata.Workflow.IStream=} [properties] Properties to set - * @returns {vtctldata.Workflow.Stream} Stream instance - */ - Stream.create = function create(properties) { - return new Stream(properties); - }; + return ExecuteVtctlCommandResponse; + })(); - /** - * Encodes the specified Stream message. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. - * @function encode - * @memberof vtctldata.Workflow.Stream - * @static - * @param {vtctldata.Workflow.IStream} message Stream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Stream.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) - $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.binlog_source != null && Object.hasOwnProperty.call(message, "binlog_source")) - $root.binlogdata.BinlogSource.encode(message.binlog_source, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.position != null && Object.hasOwnProperty.call(message, "position")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.position); - if (message.stop_position != null && Object.hasOwnProperty.call(message, "stop_position")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.stop_position); - if (message.state != null && Object.hasOwnProperty.call(message, "state")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.state); - if (message.db_name != null && Object.hasOwnProperty.call(message, "db_name")) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.db_name); - if (message.transaction_timestamp != null && Object.hasOwnProperty.call(message, "transaction_timestamp")) - $root.vttime.Time.encode(message.transaction_timestamp, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); - if (message.time_updated != null && Object.hasOwnProperty.call(message, "time_updated")) - $root.vttime.Time.encode(message.time_updated, writer.uint32(/* id 10, wireType 2 =*/82).fork()).ldelim(); - if (message.message != null && Object.hasOwnProperty.call(message, "message")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.message); - if (message.copy_states != null && message.copy_states.length) - for (let i = 0; i < message.copy_states.length; ++i) - $root.vtctldata.Workflow.Stream.CopyState.encode(message.copy_states[i], writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim(); - if (message.logs != null && message.logs.length) - for (let i = 0; i < message.logs.length; ++i) - $root.vtctldata.Workflow.Stream.Log.encode(message.logs[i], writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim(); - if (message.log_fetch_error != null && Object.hasOwnProperty.call(message, "log_fetch_error")) - writer.uint32(/* id 14, wireType 2 =*/114).string(message.log_fetch_error); - if (message.tags != null && message.tags.length) - for (let i = 0; i < message.tags.length; ++i) - writer.uint32(/* id 15, wireType 2 =*/122).string(message.tags[i]); - return writer; - }; + /** + * MaterializationIntent enum. + * @name vtctldata.MaterializationIntent + * @enum {number} + * @property {number} CUSTOM=0 CUSTOM value + * @property {number} MOVETABLES=1 MOVETABLES value + * @property {number} CREATELOOKUPINDEX=2 CREATELOOKUPINDEX value + */ + vtctldata.MaterializationIntent = (function() { + const valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "CUSTOM"] = 0; + values[valuesById[1] = "MOVETABLES"] = 1; + values[valuesById[2] = "CREATELOOKUPINDEX"] = 2; + return values; + })(); - /** - * Encodes the specified Stream message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.Workflow.Stream - * @static - * @param {vtctldata.Workflow.IStream} message Stream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Stream.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + vtctldata.TableMaterializeSettings = (function() { - /** - * Decodes a Stream message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.Workflow.Stream - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow.Stream} Stream - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Stream.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.Workflow.Stream(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.id = reader.int64(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - message.binlog_source = $root.binlogdata.BinlogSource.decode(reader, reader.uint32()); - break; - } - case 5: { - message.position = reader.string(); - break; - } - case 6: { - message.stop_position = reader.string(); - break; - } - case 7: { - message.state = reader.string(); - break; - } - case 8: { - message.db_name = reader.string(); - break; - } - case 9: { - message.transaction_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 10: { - message.time_updated = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 11: { - message.message = reader.string(); - break; - } - case 12: { - if (!(message.copy_states && message.copy_states.length)) - message.copy_states = []; - message.copy_states.push($root.vtctldata.Workflow.Stream.CopyState.decode(reader, reader.uint32())); - break; - } - case 13: { - if (!(message.logs && message.logs.length)) - message.logs = []; - message.logs.push($root.vtctldata.Workflow.Stream.Log.decode(reader, reader.uint32())); - break; - } - case 14: { - message.log_fetch_error = reader.string(); - break; - } - case 15: { - if (!(message.tags && message.tags.length)) - message.tags = []; - message.tags.push(reader.string()); - break; - } - default: - reader.skipType(tag & 7); + /** + * Properties of a TableMaterializeSettings. + * @memberof vtctldata + * @interface ITableMaterializeSettings + * @property {string|null} [target_table] TableMaterializeSettings target_table + * @property {string|null} [source_expression] TableMaterializeSettings source_expression + * @property {string|null} [create_ddl] TableMaterializeSettings create_ddl + */ + + /** + * Constructs a new TableMaterializeSettings. + * @memberof vtctldata + * @classdesc Represents a TableMaterializeSettings. + * @implements ITableMaterializeSettings + * @constructor + * @param {vtctldata.ITableMaterializeSettings=} [properties] Properties to set + */ + function TableMaterializeSettings(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]]; + } + + /** + * TableMaterializeSettings target_table. + * @member {string} target_table + * @memberof vtctldata.TableMaterializeSettings + * @instance + */ + TableMaterializeSettings.prototype.target_table = ""; + + /** + * TableMaterializeSettings source_expression. + * @member {string} source_expression + * @memberof vtctldata.TableMaterializeSettings + * @instance + */ + TableMaterializeSettings.prototype.source_expression = ""; + + /** + * TableMaterializeSettings create_ddl. + * @member {string} create_ddl + * @memberof vtctldata.TableMaterializeSettings + * @instance + */ + TableMaterializeSettings.prototype.create_ddl = ""; + + /** + * Creates a new TableMaterializeSettings instance using the specified properties. + * @function create + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {vtctldata.ITableMaterializeSettings=} [properties] Properties to set + * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings instance + */ + TableMaterializeSettings.create = function create(properties) { + return new TableMaterializeSettings(properties); + }; + + /** + * Encodes the specified TableMaterializeSettings message. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * @function encode + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {vtctldata.ITableMaterializeSettings} message TableMaterializeSettings message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TableMaterializeSettings.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.target_table != null && Object.hasOwnProperty.call(message, "target_table")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.target_table); + if (message.source_expression != null && Object.hasOwnProperty.call(message, "source_expression")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_expression); + if (message.create_ddl != null && Object.hasOwnProperty.call(message, "create_ddl")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.create_ddl); + return writer; + }; + + /** + * Encodes the specified TableMaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.TableMaterializeSettings.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {vtctldata.ITableMaterializeSettings} message TableMaterializeSettings message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TableMaterializeSettings.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a TableMaterializeSettings message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TableMaterializeSettings.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.TableMaterializeSettings(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.target_table = reader.string(); + break; + } + case 2: { + message.source_expression = reader.string(); + break; + } + case 3: { + message.create_ddl = reader.string(); break; } + default: + reader.skipType(tag & 7); + break; } - return message; - }; - - /** + } + return message; + }; + + /** + * Decodes a TableMaterializeSettings message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TableMaterializeSettings.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a TableMaterializeSettings message. + * @function verify + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + TableMaterializeSettings.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.target_table != null && message.hasOwnProperty("target_table")) + if (!$util.isString(message.target_table)) + return "target_table: string expected"; + if (message.source_expression != null && message.hasOwnProperty("source_expression")) + if (!$util.isString(message.source_expression)) + return "source_expression: string expected"; + if (message.create_ddl != null && message.hasOwnProperty("create_ddl")) + if (!$util.isString(message.create_ddl)) + return "create_ddl: string expected"; + return null; + }; + + /** + * Creates a TableMaterializeSettings message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.TableMaterializeSettings} TableMaterializeSettings + */ + TableMaterializeSettings.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.TableMaterializeSettings) + return object; + let message = new $root.vtctldata.TableMaterializeSettings(); + if (object.target_table != null) + message.target_table = String(object.target_table); + if (object.source_expression != null) + message.source_expression = String(object.source_expression); + if (object.create_ddl != null) + message.create_ddl = String(object.create_ddl); + return message; + }; + + /** + * Creates a plain object from a TableMaterializeSettings message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {vtctldata.TableMaterializeSettings} message TableMaterializeSettings + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + TableMaterializeSettings.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.target_table = ""; + object.source_expression = ""; + object.create_ddl = ""; + } + if (message.target_table != null && message.hasOwnProperty("target_table")) + object.target_table = message.target_table; + if (message.source_expression != null && message.hasOwnProperty("source_expression")) + object.source_expression = message.source_expression; + if (message.create_ddl != null && message.hasOwnProperty("create_ddl")) + object.create_ddl = message.create_ddl; + return object; + }; + + /** + * Converts this TableMaterializeSettings to JSON. + * @function toJSON + * @memberof vtctldata.TableMaterializeSettings + * @instance + * @returns {Object.} JSON object + */ + TableMaterializeSettings.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for TableMaterializeSettings + * @function getTypeUrl + * @memberof vtctldata.TableMaterializeSettings + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + TableMaterializeSettings.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.TableMaterializeSettings"; + }; + + return TableMaterializeSettings; + })(); + + vtctldata.MaterializeSettings = (function() { + + /** + * Properties of a MaterializeSettings. + * @memberof vtctldata + * @interface IMaterializeSettings + * @property {string|null} [workflow] MaterializeSettings workflow + * @property {string|null} [source_keyspace] MaterializeSettings source_keyspace + * @property {string|null} [target_keyspace] MaterializeSettings target_keyspace + * @property {boolean|null} [stop_after_copy] MaterializeSettings stop_after_copy + * @property {Array.|null} [table_settings] MaterializeSettings table_settings + * @property {string|null} [cell] MaterializeSettings cell + * @property {string|null} [tablet_types] MaterializeSettings tablet_types + * @property {string|null} [external_cluster] MaterializeSettings external_cluster + * @property {vtctldata.MaterializationIntent|null} [materialization_intent] MaterializeSettings materialization_intent + * @property {string|null} [source_time_zone] MaterializeSettings source_time_zone + * @property {string|null} [target_time_zone] MaterializeSettings target_time_zone + * @property {Array.|null} [source_shards] MaterializeSettings source_shards + * @property {string|null} [on_ddl] MaterializeSettings on_ddl + * @property {boolean|null} [defer_secondary_keys] MaterializeSettings defer_secondary_keys + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] MaterializeSettings tablet_selection_preference + */ + + /** + * Constructs a new MaterializeSettings. + * @memberof vtctldata + * @classdesc Represents a MaterializeSettings. + * @implements IMaterializeSettings + * @constructor + * @param {vtctldata.IMaterializeSettings=} [properties] Properties to set + */ + function MaterializeSettings(properties) { + this.table_settings = []; + this.source_shards = []; + 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]]; + } + + /** + * MaterializeSettings workflow. + * @member {string} workflow + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.workflow = ""; + + /** + * MaterializeSettings source_keyspace. + * @member {string} source_keyspace + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.source_keyspace = ""; + + /** + * MaterializeSettings target_keyspace. + * @member {string} target_keyspace + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.target_keyspace = ""; + + /** + * MaterializeSettings stop_after_copy. + * @member {boolean} stop_after_copy + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.stop_after_copy = false; + + /** + * MaterializeSettings table_settings. + * @member {Array.} table_settings + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.table_settings = $util.emptyArray; + + /** + * MaterializeSettings cell. + * @member {string} cell + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.cell = ""; + + /** + * MaterializeSettings tablet_types. + * @member {string} tablet_types + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.tablet_types = ""; + + /** + * MaterializeSettings external_cluster. + * @member {string} external_cluster + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.external_cluster = ""; + + /** + * MaterializeSettings materialization_intent. + * @member {vtctldata.MaterializationIntent} materialization_intent + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.materialization_intent = 0; + + /** + * MaterializeSettings source_time_zone. + * @member {string} source_time_zone + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.source_time_zone = ""; + + /** + * MaterializeSettings target_time_zone. + * @member {string} target_time_zone + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.target_time_zone = ""; + + /** + * MaterializeSettings source_shards. + * @member {Array.} source_shards + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.source_shards = $util.emptyArray; + + /** + * MaterializeSettings on_ddl. + * @member {string} on_ddl + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.on_ddl = ""; + + /** + * MaterializeSettings defer_secondary_keys. + * @member {boolean} defer_secondary_keys + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.defer_secondary_keys = false; + + /** + * MaterializeSettings tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.tablet_selection_preference = 0; + + /** + * Creates a new MaterializeSettings instance using the specified properties. + * @function create + * @memberof vtctldata.MaterializeSettings + * @static + * @param {vtctldata.IMaterializeSettings=} [properties] Properties to set + * @returns {vtctldata.MaterializeSettings} MaterializeSettings instance + */ + MaterializeSettings.create = function create(properties) { + return new MaterializeSettings(properties); + }; + + /** + * Encodes the specified MaterializeSettings message. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * @function encode + * @memberof vtctldata.MaterializeSettings + * @static + * @param {vtctldata.IMaterializeSettings} message MaterializeSettings message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + MaterializeSettings.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_keyspace); + if (message.target_keyspace != null && Object.hasOwnProperty.call(message, "target_keyspace")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_keyspace); + if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.stop_after_copy); + if (message.table_settings != null && message.table_settings.length) + for (let i = 0; i < message.table_settings.length; ++i) + $root.vtctldata.TableMaterializeSettings.encode(message.table_settings[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.cell); + if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.tablet_types); + if (message.external_cluster != null && Object.hasOwnProperty.call(message, "external_cluster")) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.external_cluster); + if (message.materialization_intent != null && Object.hasOwnProperty.call(message, "materialization_intent")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.materialization_intent); + if (message.source_time_zone != null && Object.hasOwnProperty.call(message, "source_time_zone")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.source_time_zone); + if (message.target_time_zone != null && Object.hasOwnProperty.call(message, "target_time_zone")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.target_time_zone); + if (message.source_shards != null && message.source_shards.length) + for (let i = 0; i < message.source_shards.length; ++i) + writer.uint32(/* id 12, wireType 2 =*/98).string(message.source_shards[i]); + if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) + writer.uint32(/* id 13, wireType 2 =*/106).string(message.on_ddl); + if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) + writer.uint32(/* id 14, wireType 0 =*/112).bool(message.defer_secondary_keys); + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 15, wireType 0 =*/120).int32(message.tablet_selection_preference); + return writer; + }; + + /** + * Encodes the specified MaterializeSettings message, length delimited. Does not implicitly {@link vtctldata.MaterializeSettings.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.MaterializeSettings + * @static + * @param {vtctldata.IMaterializeSettings} message MaterializeSettings message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + MaterializeSettings.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a MaterializeSettings message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.MaterializeSettings + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.MaterializeSettings} MaterializeSettings + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + MaterializeSettings.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.MaterializeSettings(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.workflow = reader.string(); + break; + } + case 2: { + message.source_keyspace = reader.string(); + break; + } + case 3: { + message.target_keyspace = reader.string(); + break; + } + case 4: { + message.stop_after_copy = reader.bool(); + break; + } + case 5: { + if (!(message.table_settings && message.table_settings.length)) + message.table_settings = []; + message.table_settings.push($root.vtctldata.TableMaterializeSettings.decode(reader, reader.uint32())); + break; + } + case 6: { + message.cell = reader.string(); + break; + } + case 7: { + message.tablet_types = reader.string(); + break; + } + case 8: { + message.external_cluster = reader.string(); + break; + } + case 9: { + message.materialization_intent = reader.int32(); + break; + } + case 10: { + message.source_time_zone = reader.string(); + break; + } + case 11: { + message.target_time_zone = reader.string(); + break; + } + case 12: { + if (!(message.source_shards && message.source_shards.length)) + message.source_shards = []; + message.source_shards.push(reader.string()); + break; + } + case 13: { + message.on_ddl = reader.string(); + break; + } + case 14: { + message.defer_secondary_keys = reader.bool(); + break; + } + case 15: { + message.tablet_selection_preference = reader.int32(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a MaterializeSettings message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.MaterializeSettings + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.MaterializeSettings} MaterializeSettings + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + MaterializeSettings.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a MaterializeSettings message. + * @function verify + * @memberof vtctldata.MaterializeSettings + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + MaterializeSettings.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + if (!$util.isString(message.source_keyspace)) + return "source_keyspace: string expected"; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + if (!$util.isString(message.target_keyspace)) + return "target_keyspace: string expected"; + 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.table_settings != null && message.hasOwnProperty("table_settings")) { + if (!Array.isArray(message.table_settings)) + return "table_settings: array expected"; + for (let i = 0; i < message.table_settings.length; ++i) { + let error = $root.vtctldata.TableMaterializeSettings.verify(message.table_settings[i]); + if (error) + return "table_settings." + error; + } + } + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + if (!$util.isString(message.tablet_types)) + return "tablet_types: string expected"; + if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) + if (!$util.isString(message.external_cluster)) + return "external_cluster: string expected"; + if (message.materialization_intent != null && message.hasOwnProperty("materialization_intent")) + switch (message.materialization_intent) { + default: + return "materialization_intent: enum value expected"; + case 0: + case 1: + case 2: + break; + } + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + if (!$util.isString(message.source_time_zone)) + return "source_time_zone: string expected"; + if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) + if (!$util.isString(message.target_time_zone)) + return "target_time_zone: string expected"; + if (message.source_shards != null && message.hasOwnProperty("source_shards")) { + if (!Array.isArray(message.source_shards)) + return "source_shards: array expected"; + for (let i = 0; i < message.source_shards.length; ++i) + if (!$util.isString(message.source_shards[i])) + return "source_shards: string[] expected"; + } + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + if (!$util.isString(message.on_ddl)) + return "on_ddl: string expected"; + 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.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + return null; + }; + + /** + * Creates a MaterializeSettings message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.MaterializeSettings + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.MaterializeSettings} MaterializeSettings + */ + MaterializeSettings.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MaterializeSettings) + return object; + let message = new $root.vtctldata.MaterializeSettings(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.source_keyspace != null) + message.source_keyspace = String(object.source_keyspace); + if (object.target_keyspace != null) + message.target_keyspace = String(object.target_keyspace); + if (object.stop_after_copy != null) + message.stop_after_copy = Boolean(object.stop_after_copy); + if (object.table_settings) { + if (!Array.isArray(object.table_settings)) + throw TypeError(".vtctldata.MaterializeSettings.table_settings: array expected"); + message.table_settings = []; + for (let i = 0; i < object.table_settings.length; ++i) { + if (typeof object.table_settings[i] !== "object") + throw TypeError(".vtctldata.MaterializeSettings.table_settings: object expected"); + message.table_settings[i] = $root.vtctldata.TableMaterializeSettings.fromObject(object.table_settings[i]); + } + } + if (object.cell != null) + message.cell = String(object.cell); + if (object.tablet_types != null) + message.tablet_types = String(object.tablet_types); + if (object.external_cluster != null) + message.external_cluster = String(object.external_cluster); + switch (object.materialization_intent) { + default: + if (typeof object.materialization_intent === "number") { + message.materialization_intent = object.materialization_intent; + break; + } + break; + case "CUSTOM": + case 0: + message.materialization_intent = 0; + break; + case "MOVETABLES": + case 1: + message.materialization_intent = 1; + break; + case "CREATELOOKUPINDEX": + case 2: + message.materialization_intent = 2; + break; + } + if (object.source_time_zone != null) + message.source_time_zone = String(object.source_time_zone); + if (object.target_time_zone != null) + message.target_time_zone = String(object.target_time_zone); + if (object.source_shards) { + if (!Array.isArray(object.source_shards)) + throw TypeError(".vtctldata.MaterializeSettings.source_shards: array expected"); + message.source_shards = []; + for (let i = 0; i < object.source_shards.length; ++i) + message.source_shards[i] = String(object.source_shards[i]); + } + if (object.on_ddl != null) + message.on_ddl = String(object.on_ddl); + if (object.defer_secondary_keys != null) + message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; + break; + } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + return message; + }; + + /** + * Creates a plain object from a MaterializeSettings message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.MaterializeSettings + * @static + * @param {vtctldata.MaterializeSettings} message MaterializeSettings + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + MaterializeSettings.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.table_settings = []; + object.source_shards = []; + } + if (options.defaults) { + object.workflow = ""; + object.source_keyspace = ""; + object.target_keyspace = ""; + object.stop_after_copy = false; + object.cell = ""; + object.tablet_types = ""; + object.external_cluster = ""; + object.materialization_intent = options.enums === String ? "CUSTOM" : 0; + object.source_time_zone = ""; + object.target_time_zone = ""; + object.on_ddl = ""; + object.defer_secondary_keys = false; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + object.source_keyspace = message.source_keyspace; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + object.target_keyspace = message.target_keyspace; + if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) + object.stop_after_copy = message.stop_after_copy; + if (message.table_settings && message.table_settings.length) { + object.table_settings = []; + for (let j = 0; j < message.table_settings.length; ++j) + object.table_settings[j] = $root.vtctldata.TableMaterializeSettings.toObject(message.table_settings[j], options); + } + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + object.tablet_types = message.tablet_types; + if (message.external_cluster != null && message.hasOwnProperty("external_cluster")) + object.external_cluster = message.external_cluster; + if (message.materialization_intent != null && message.hasOwnProperty("materialization_intent")) + object.materialization_intent = options.enums === String ? $root.vtctldata.MaterializationIntent[message.materialization_intent] === undefined ? message.materialization_intent : $root.vtctldata.MaterializationIntent[message.materialization_intent] : message.materialization_intent; + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + object.source_time_zone = message.source_time_zone; + if (message.target_time_zone != null && message.hasOwnProperty("target_time_zone")) + object.target_time_zone = message.target_time_zone; + if (message.source_shards && message.source_shards.length) { + object.source_shards = []; + for (let j = 0; j < message.source_shards.length; ++j) + object.source_shards[j] = message.source_shards[j]; + } + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + object.on_ddl = message.on_ddl; + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + object.defer_secondary_keys = message.defer_secondary_keys; + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + return object; + }; + + /** + * Converts this MaterializeSettings to JSON. + * @function toJSON + * @memberof vtctldata.MaterializeSettings + * @instance + * @returns {Object.} JSON object + */ + MaterializeSettings.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for MaterializeSettings + * @function getTypeUrl + * @memberof vtctldata.MaterializeSettings + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + MaterializeSettings.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.MaterializeSettings"; + }; + + return MaterializeSettings; + })(); + + vtctldata.Keyspace = (function() { + + /** + * Properties of a Keyspace. + * @memberof vtctldata + * @interface IKeyspace + * @property {string|null} [name] Keyspace name + * @property {topodata.IKeyspace|null} [keyspace] Keyspace keyspace + */ + + /** + * Constructs a new Keyspace. + * @memberof vtctldata + * @classdesc Represents a Keyspace. + * @implements IKeyspace + * @constructor + * @param {vtctldata.IKeyspace=} [properties] Properties to set + */ + function Keyspace(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]]; + } + + /** + * Keyspace name. + * @member {string} name + * @memberof vtctldata.Keyspace + * @instance + */ + Keyspace.prototype.name = ""; + + /** + * Keyspace keyspace. + * @member {topodata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.Keyspace + * @instance + */ + Keyspace.prototype.keyspace = null; + + /** + * Creates a new Keyspace instance using the specified properties. + * @function create + * @memberof vtctldata.Keyspace + * @static + * @param {vtctldata.IKeyspace=} [properties] Properties to set + * @returns {vtctldata.Keyspace} Keyspace instance + */ + Keyspace.create = function create(properties) { + return new Keyspace(properties); + }; + + /** + * Encodes the specified Keyspace message. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * @function encode + * @memberof vtctldata.Keyspace + * @static + * @param {vtctldata.IKeyspace} message Keyspace message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Keyspace.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified Keyspace message, length delimited. Does not implicitly {@link vtctldata.Keyspace.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Keyspace + * @static + * @param {vtctldata.IKeyspace} message Keyspace message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Keyspace.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Keyspace message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Keyspace + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Keyspace} Keyspace + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Keyspace.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.Keyspace(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Keyspace message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Keyspace + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Keyspace} Keyspace + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Keyspace.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Keyspace message. + * @function verify + * @memberof vtctldata.Keyspace + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Keyspace.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.topodata.Keyspace.verify(message.keyspace); + if (error) + return "keyspace." + error; + } + return null; + }; + + /** + * Creates a Keyspace message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Keyspace + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Keyspace} Keyspace + */ + Keyspace.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Keyspace) + return object; + let message = new $root.vtctldata.Keyspace(); + if (object.name != null) + message.name = String(object.name); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.Keyspace.keyspace: object expected"); + message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); + } + return message; + }; + + /** + * Creates a plain object from a Keyspace message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Keyspace + * @static + * @param {vtctldata.Keyspace} message Keyspace + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Keyspace.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.name = ""; + object.keyspace = null; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); + return object; + }; + + /** + * Converts this Keyspace to JSON. + * @function toJSON + * @memberof vtctldata.Keyspace + * @instance + * @returns {Object.} JSON object + */ + Keyspace.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Keyspace + * @function getTypeUrl + * @memberof vtctldata.Keyspace + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Keyspace.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Keyspace"; + }; + + return Keyspace; + })(); + + vtctldata.Shard = (function() { + + /** + * Properties of a Shard. + * @memberof vtctldata + * @interface IShard + * @property {string|null} [keyspace] Shard keyspace + * @property {string|null} [name] Shard name + * @property {topodata.IShard|null} [shard] Shard shard + */ + + /** + * Constructs a new Shard. + * @memberof vtctldata + * @classdesc Represents a Shard. + * @implements IShard + * @constructor + * @param {vtctldata.IShard=} [properties] Properties to set + */ + function Shard(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]]; + } + + /** + * Shard keyspace. + * @member {string} keyspace + * @memberof vtctldata.Shard + * @instance + */ + Shard.prototype.keyspace = ""; + + /** + * Shard name. + * @member {string} name + * @memberof vtctldata.Shard + * @instance + */ + Shard.prototype.name = ""; + + /** + * Shard shard. + * @member {topodata.IShard|null|undefined} shard + * @memberof vtctldata.Shard + * @instance + */ + Shard.prototype.shard = null; + + /** + * Creates a new Shard instance using the specified properties. + * @function create + * @memberof vtctldata.Shard + * @static + * @param {vtctldata.IShard=} [properties] Properties to set + * @returns {vtctldata.Shard} Shard instance + */ + Shard.create = function create(properties) { + return new Shard(properties); + }; + + /** + * Encodes the specified Shard message. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * @function encode + * @memberof vtctldata.Shard + * @static + * @param {vtctldata.IShard} message Shard message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Shard.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.name); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified Shard message, length delimited. Does not implicitly {@link vtctldata.Shard.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Shard + * @static + * @param {vtctldata.IShard} message Shard message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Shard.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Shard message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Shard + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Shard} Shard + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Shard.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.Shard(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.name = reader.string(); + break; + } + case 3: { + message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Shard message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Shard + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Shard} Shard + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Shard.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Shard message. + * @function verify + * @memberof vtctldata.Shard + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Shard.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.topodata.Shard.verify(message.shard); + if (error) + return "shard." + error; + } + return null; + }; + + /** + * Creates a Shard message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Shard + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Shard} Shard + */ + Shard.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Shard) + return object; + let message = new $root.vtctldata.Shard(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.name != null) + message.name = String(object.name); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.Shard.shard: object expected"); + message.shard = $root.topodata.Shard.fromObject(object.shard); + } + return message; + }; + + /** + * Creates a plain object from a Shard message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Shard + * @static + * @param {vtctldata.Shard} message Shard + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Shard.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.keyspace = ""; + object.name = ""; + object.shard = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.topodata.Shard.toObject(message.shard, options); + return object; + }; + + /** + * Converts this Shard to JSON. + * @function toJSON + * @memberof vtctldata.Shard + * @instance + * @returns {Object.} JSON object + */ + Shard.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Shard + * @function getTypeUrl + * @memberof vtctldata.Shard + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Shard.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Shard"; + }; + + return Shard; + })(); + + vtctldata.Workflow = (function() { + + /** + * Properties of a Workflow. + * @memberof vtctldata + * @interface IWorkflow + * @property {string|null} [name] Workflow name + * @property {vtctldata.Workflow.IReplicationLocation|null} [source] Workflow source + * @property {vtctldata.Workflow.IReplicationLocation|null} [target] Workflow target + * @property {number|Long|null} [max_v_replication_lag] Workflow max_v_replication_lag + * @property {Object.|null} [shard_streams] Workflow shard_streams + * @property {string|null} [workflow_type] Workflow workflow_type + * @property {string|null} [workflow_sub_type] Workflow workflow_sub_type + */ + + /** + * Constructs a new Workflow. + * @memberof vtctldata + * @classdesc Represents a Workflow. + * @implements IWorkflow + * @constructor + * @param {vtctldata.IWorkflow=} [properties] Properties to set + */ + function Workflow(properties) { + this.shard_streams = {}; + 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]]; + } + + /** + * Workflow name. + * @member {string} name + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.name = ""; + + /** + * Workflow source. + * @member {vtctldata.Workflow.IReplicationLocation|null|undefined} source + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.source = null; + + /** + * Workflow target. + * @member {vtctldata.Workflow.IReplicationLocation|null|undefined} target + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.target = null; + + /** + * Workflow max_v_replication_lag. + * @member {number|Long} max_v_replication_lag + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.max_v_replication_lag = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Workflow shard_streams. + * @member {Object.} shard_streams + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.shard_streams = $util.emptyObject; + + /** + * Workflow workflow_type. + * @member {string} workflow_type + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.workflow_type = ""; + + /** + * Workflow workflow_sub_type. + * @member {string} workflow_sub_type + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.workflow_sub_type = ""; + + /** + * Creates a new Workflow instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow + * @static + * @param {vtctldata.IWorkflow=} [properties] Properties to set + * @returns {vtctldata.Workflow} Workflow instance + */ + Workflow.create = function create(properties) { + return new Workflow(properties); + }; + + /** + * Encodes the specified Workflow message. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow + * @static + * @param {vtctldata.IWorkflow} message Workflow message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Workflow.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.source != null && Object.hasOwnProperty.call(message, "source")) + $root.vtctldata.Workflow.ReplicationLocation.encode(message.source, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.vtctldata.Workflow.ReplicationLocation.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.max_v_replication_lag != null && Object.hasOwnProperty.call(message, "max_v_replication_lag")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.max_v_replication_lag); + if (message.shard_streams != null && Object.hasOwnProperty.call(message, "shard_streams")) + for (let keys = Object.keys(message.shard_streams), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 5, wireType 2 =*/42).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.Workflow.ShardStream.encode(message.shard_streams[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.workflow_type); + if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.workflow_sub_type); + return writer; + }; + + /** + * Encodes the specified Workflow message, length delimited. Does not implicitly {@link vtctldata.Workflow.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow + * @static + * @param {vtctldata.IWorkflow} message Workflow message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Workflow.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Workflow message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow} Workflow + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Workflow.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.Workflow(), key, value; + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + message.source = $root.vtctldata.Workflow.ReplicationLocation.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.vtctldata.Workflow.ReplicationLocation.decode(reader, reader.uint32()); + break; + } + case 4: { + message.max_v_replication_lag = reader.int64(); + break; + } + case 5: { + if (message.shard_streams === $util.emptyObject) + message.shard_streams = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.Workflow.ShardStream.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.shard_streams[key] = value; + break; + } + case 6: { + message.workflow_type = reader.string(); + break; + } + case 7: { + message.workflow_sub_type = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Workflow message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Workflow + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Workflow} Workflow + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Workflow.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Workflow message. + * @function verify + * @memberof vtctldata.Workflow + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Workflow.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.source != null && message.hasOwnProperty("source")) { + let error = $root.vtctldata.Workflow.ReplicationLocation.verify(message.source); + if (error) + return "source." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.vtctldata.Workflow.ReplicationLocation.verify(message.target); + if (error) + return "target." + error; + } + if (message.max_v_replication_lag != null && message.hasOwnProperty("max_v_replication_lag")) + if (!$util.isInteger(message.max_v_replication_lag) && !(message.max_v_replication_lag && $util.isInteger(message.max_v_replication_lag.low) && $util.isInteger(message.max_v_replication_lag.high))) + return "max_v_replication_lag: integer|Long expected"; + if (message.shard_streams != null && message.hasOwnProperty("shard_streams")) { + if (!$util.isObject(message.shard_streams)) + return "shard_streams: object expected"; + let key = Object.keys(message.shard_streams); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.Workflow.ShardStream.verify(message.shard_streams[key[i]]); + if (error) + return "shard_streams." + error; + } + } + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + if (!$util.isString(message.workflow_type)) + return "workflow_type: string expected"; + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + if (!$util.isString(message.workflow_sub_type)) + return "workflow_sub_type: string expected"; + return null; + }; + + /** + * Creates a Workflow message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow} Workflow + */ + Workflow.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow) + return object; + let message = new $root.vtctldata.Workflow(); + if (object.name != null) + message.name = String(object.name); + if (object.source != null) { + if (typeof object.source !== "object") + throw TypeError(".vtctldata.Workflow.source: object expected"); + message.source = $root.vtctldata.Workflow.ReplicationLocation.fromObject(object.source); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".vtctldata.Workflow.target: object expected"); + message.target = $root.vtctldata.Workflow.ReplicationLocation.fromObject(object.target); + } + if (object.max_v_replication_lag != null) + if ($util.Long) + (message.max_v_replication_lag = $util.Long.fromValue(object.max_v_replication_lag)).unsigned = false; + else if (typeof object.max_v_replication_lag === "string") + message.max_v_replication_lag = parseInt(object.max_v_replication_lag, 10); + else if (typeof object.max_v_replication_lag === "number") + message.max_v_replication_lag = object.max_v_replication_lag; + else if (typeof object.max_v_replication_lag === "object") + message.max_v_replication_lag = new $util.LongBits(object.max_v_replication_lag.low >>> 0, object.max_v_replication_lag.high >>> 0).toNumber(); + if (object.shard_streams) { + if (typeof object.shard_streams !== "object") + throw TypeError(".vtctldata.Workflow.shard_streams: object expected"); + message.shard_streams = {}; + for (let keys = Object.keys(object.shard_streams), i = 0; i < keys.length; ++i) { + if (typeof object.shard_streams[keys[i]] !== "object") + throw TypeError(".vtctldata.Workflow.shard_streams: object expected"); + message.shard_streams[keys[i]] = $root.vtctldata.Workflow.ShardStream.fromObject(object.shard_streams[keys[i]]); + } + } + if (object.workflow_type != null) + message.workflow_type = String(object.workflow_type); + if (object.workflow_sub_type != null) + message.workflow_sub_type = String(object.workflow_sub_type); + return message; + }; + + /** + * Creates a plain object from a Workflow message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow + * @static + * @param {vtctldata.Workflow} message Workflow + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Workflow.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.objects || options.defaults) + object.shard_streams = {}; + if (options.defaults) { + object.name = ""; + object.source = null; + object.target = null; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_v_replication_lag = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_v_replication_lag = options.longs === String ? "0" : 0; + object.workflow_type = ""; + object.workflow_sub_type = ""; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.source != null && message.hasOwnProperty("source")) + object.source = $root.vtctldata.Workflow.ReplicationLocation.toObject(message.source, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.vtctldata.Workflow.ReplicationLocation.toObject(message.target, options); + if (message.max_v_replication_lag != null && message.hasOwnProperty("max_v_replication_lag")) + if (typeof message.max_v_replication_lag === "number") + object.max_v_replication_lag = options.longs === String ? String(message.max_v_replication_lag) : message.max_v_replication_lag; + else + object.max_v_replication_lag = options.longs === String ? $util.Long.prototype.toString.call(message.max_v_replication_lag) : options.longs === Number ? new $util.LongBits(message.max_v_replication_lag.low >>> 0, message.max_v_replication_lag.high >>> 0).toNumber() : message.max_v_replication_lag; + let keys2; + if (message.shard_streams && (keys2 = Object.keys(message.shard_streams)).length) { + object.shard_streams = {}; + for (let j = 0; j < keys2.length; ++j) + object.shard_streams[keys2[j]] = $root.vtctldata.Workflow.ShardStream.toObject(message.shard_streams[keys2[j]], options); + } + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + object.workflow_type = message.workflow_type; + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + object.workflow_sub_type = message.workflow_sub_type; + return object; + }; + + /** + * Converts this Workflow to JSON. + * @function toJSON + * @memberof vtctldata.Workflow + * @instance + * @returns {Object.} JSON object + */ + Workflow.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Workflow + * @function getTypeUrl + * @memberof vtctldata.Workflow + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Workflow.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow"; + }; + + Workflow.ReplicationLocation = (function() { + + /** + * Properties of a ReplicationLocation. + * @memberof vtctldata.Workflow + * @interface IReplicationLocation + * @property {string|null} [keyspace] ReplicationLocation keyspace + * @property {Array.|null} [shards] ReplicationLocation shards + */ + + /** + * Constructs a new ReplicationLocation. + * @memberof vtctldata.Workflow + * @classdesc Represents a ReplicationLocation. + * @implements IReplicationLocation + * @constructor + * @param {vtctldata.Workflow.IReplicationLocation=} [properties] Properties to set + */ + function ReplicationLocation(properties) { + this.shards = []; + 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]]; + } + + /** + * ReplicationLocation keyspace. + * @member {string} keyspace + * @memberof vtctldata.Workflow.ReplicationLocation + * @instance + */ + ReplicationLocation.prototype.keyspace = ""; + + /** + * ReplicationLocation shards. + * @member {Array.} shards + * @memberof vtctldata.Workflow.ReplicationLocation + * @instance + */ + ReplicationLocation.prototype.shards = $util.emptyArray; + + /** + * Creates a new ReplicationLocation instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {vtctldata.Workflow.IReplicationLocation=} [properties] Properties to set + * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation instance + */ + ReplicationLocation.create = function create(properties) { + return new ReplicationLocation(properties); + }; + + /** + * Encodes the specified ReplicationLocation message. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {vtctldata.Workflow.IReplicationLocation} message ReplicationLocation message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReplicationLocation.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shards != null && message.shards.length) + for (let i = 0; i < message.shards.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shards[i]); + return writer; + }; + + /** + * Encodes the specified ReplicationLocation message, length delimited. Does not implicitly {@link vtctldata.Workflow.ReplicationLocation.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {vtctldata.Workflow.IReplicationLocation} message ReplicationLocation message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReplicationLocation.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ReplicationLocation message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReplicationLocation.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.Workflow.ReplicationLocation(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + if (!(message.shards && message.shards.length)) + message.shards = []; + message.shards.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a ReplicationLocation message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReplicationLocation.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a ReplicationLocation message. + * @function verify + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReplicationLocation.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shards != null && message.hasOwnProperty("shards")) { + if (!Array.isArray(message.shards)) + return "shards: array expected"; + for (let i = 0; i < message.shards.length; ++i) + if (!$util.isString(message.shards[i])) + return "shards: string[] expected"; + } + return null; + }; + + /** + * Creates a ReplicationLocation message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow.ReplicationLocation} ReplicationLocation + */ + ReplicationLocation.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow.ReplicationLocation) + return object; + let message = new $root.vtctldata.Workflow.ReplicationLocation(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shards) { + if (!Array.isArray(object.shards)) + throw TypeError(".vtctldata.Workflow.ReplicationLocation.shards: array expected"); + message.shards = []; + for (let i = 0; i < object.shards.length; ++i) + message.shards[i] = String(object.shards[i]); + } + return message; + }; + + /** + * Creates a plain object from a ReplicationLocation message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {vtctldata.Workflow.ReplicationLocation} message ReplicationLocation + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ReplicationLocation.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.shards = []; + if (options.defaults) + object.keyspace = ""; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shards && message.shards.length) { + object.shards = []; + for (let j = 0; j < message.shards.length; ++j) + object.shards[j] = message.shards[j]; + } + return object; + }; + + /** + * Converts this ReplicationLocation to JSON. + * @function toJSON + * @memberof vtctldata.Workflow.ReplicationLocation + * @instance + * @returns {Object.} JSON object + */ + ReplicationLocation.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReplicationLocation + * @function getTypeUrl + * @memberof vtctldata.Workflow.ReplicationLocation + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReplicationLocation.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow.ReplicationLocation"; + }; + + return ReplicationLocation; + })(); + + Workflow.ShardStream = (function() { + + /** + * Properties of a ShardStream. + * @memberof vtctldata.Workflow + * @interface IShardStream + * @property {Array.|null} [streams] ShardStream streams + * @property {Array.|null} [tablet_controls] ShardStream tablet_controls + * @property {boolean|null} [is_primary_serving] ShardStream is_primary_serving + */ + + /** + * Constructs a new ShardStream. + * @memberof vtctldata.Workflow + * @classdesc Represents a ShardStream. + * @implements IShardStream + * @constructor + * @param {vtctldata.Workflow.IShardStream=} [properties] Properties to set + */ + function ShardStream(properties) { + this.streams = []; + this.tablet_controls = []; + 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]]; + } + + /** + * ShardStream streams. + * @member {Array.} streams + * @memberof vtctldata.Workflow.ShardStream + * @instance + */ + ShardStream.prototype.streams = $util.emptyArray; + + /** + * ShardStream tablet_controls. + * @member {Array.} tablet_controls + * @memberof vtctldata.Workflow.ShardStream + * @instance + */ + ShardStream.prototype.tablet_controls = $util.emptyArray; + + /** + * ShardStream is_primary_serving. + * @member {boolean} is_primary_serving + * @memberof vtctldata.Workflow.ShardStream + * @instance + */ + ShardStream.prototype.is_primary_serving = false; + + /** + * Creates a new ShardStream instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {vtctldata.Workflow.IShardStream=} [properties] Properties to set + * @returns {vtctldata.Workflow.ShardStream} ShardStream instance + */ + ShardStream.create = function create(properties) { + return new ShardStream(properties); + }; + + /** + * Encodes the specified ShardStream message. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {vtctldata.Workflow.IShardStream} message ShardStream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStream.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.streams != null && message.streams.length) + for (let i = 0; i < message.streams.length; ++i) + $root.vtctldata.Workflow.Stream.encode(message.streams[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.tablet_controls != null && message.tablet_controls.length) + for (let i = 0; i < message.tablet_controls.length; ++i) + $root.topodata.Shard.TabletControl.encode(message.tablet_controls[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.is_primary_serving != null && Object.hasOwnProperty.call(message, "is_primary_serving")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.is_primary_serving); + return writer; + }; + + /** + * Encodes the specified ShardStream message, length delimited. Does not implicitly {@link vtctldata.Workflow.ShardStream.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {vtctldata.Workflow.IShardStream} message ShardStream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStream.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ShardStream message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow.ShardStream} ShardStream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStream.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.Workflow.ShardStream(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.streams && message.streams.length)) + message.streams = []; + message.streams.push($root.vtctldata.Workflow.Stream.decode(reader, reader.uint32())); + break; + } + case 2: { + if (!(message.tablet_controls && message.tablet_controls.length)) + message.tablet_controls = []; + message.tablet_controls.push($root.topodata.Shard.TabletControl.decode(reader, reader.uint32())); + break; + } + case 3: { + message.is_primary_serving = reader.bool(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a ShardStream message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Workflow.ShardStream} ShardStream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStream.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a ShardStream message. + * @function verify + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ShardStream.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.streams != null && message.hasOwnProperty("streams")) { + if (!Array.isArray(message.streams)) + return "streams: array expected"; + for (let i = 0; i < message.streams.length; ++i) { + let error = $root.vtctldata.Workflow.Stream.verify(message.streams[i]); + if (error) + return "streams." + error; + } + } + if (message.tablet_controls != null && message.hasOwnProperty("tablet_controls")) { + if (!Array.isArray(message.tablet_controls)) + return "tablet_controls: array expected"; + for (let i = 0; i < message.tablet_controls.length; ++i) { + let error = $root.topodata.Shard.TabletControl.verify(message.tablet_controls[i]); + if (error) + return "tablet_controls." + error; + } + } + if (message.is_primary_serving != null && message.hasOwnProperty("is_primary_serving")) + if (typeof message.is_primary_serving !== "boolean") + return "is_primary_serving: boolean expected"; + return null; + }; + + /** + * Creates a ShardStream message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow.ShardStream} ShardStream + */ + ShardStream.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow.ShardStream) + return object; + let message = new $root.vtctldata.Workflow.ShardStream(); + if (object.streams) { + if (!Array.isArray(object.streams)) + throw TypeError(".vtctldata.Workflow.ShardStream.streams: array expected"); + message.streams = []; + for (let i = 0; i < object.streams.length; ++i) { + if (typeof object.streams[i] !== "object") + throw TypeError(".vtctldata.Workflow.ShardStream.streams: object expected"); + message.streams[i] = $root.vtctldata.Workflow.Stream.fromObject(object.streams[i]); + } + } + if (object.tablet_controls) { + if (!Array.isArray(object.tablet_controls)) + throw TypeError(".vtctldata.Workflow.ShardStream.tablet_controls: array expected"); + message.tablet_controls = []; + for (let i = 0; i < object.tablet_controls.length; ++i) { + if (typeof object.tablet_controls[i] !== "object") + throw TypeError(".vtctldata.Workflow.ShardStream.tablet_controls: object expected"); + message.tablet_controls[i] = $root.topodata.Shard.TabletControl.fromObject(object.tablet_controls[i]); + } + } + if (object.is_primary_serving != null) + message.is_primary_serving = Boolean(object.is_primary_serving); + return message; + }; + + /** + * Creates a plain object from a ShardStream message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {vtctldata.Workflow.ShardStream} message ShardStream + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ShardStream.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.streams = []; + object.tablet_controls = []; + } + if (options.defaults) + object.is_primary_serving = false; + if (message.streams && message.streams.length) { + object.streams = []; + for (let j = 0; j < message.streams.length; ++j) + object.streams[j] = $root.vtctldata.Workflow.Stream.toObject(message.streams[j], options); + } + if (message.tablet_controls && message.tablet_controls.length) { + object.tablet_controls = []; + for (let j = 0; j < message.tablet_controls.length; ++j) + object.tablet_controls[j] = $root.topodata.Shard.TabletControl.toObject(message.tablet_controls[j], options); + } + if (message.is_primary_serving != null && message.hasOwnProperty("is_primary_serving")) + object.is_primary_serving = message.is_primary_serving; + return object; + }; + + /** + * Converts this ShardStream to JSON. + * @function toJSON + * @memberof vtctldata.Workflow.ShardStream + * @instance + * @returns {Object.} JSON object + */ + ShardStream.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ShardStream + * @function getTypeUrl + * @memberof vtctldata.Workflow.ShardStream + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ShardStream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow.ShardStream"; + }; + + return ShardStream; + })(); + + Workflow.Stream = (function() { + + /** + * Properties of a Stream. + * @memberof vtctldata.Workflow + * @interface IStream + * @property {number|Long|null} [id] Stream id + * @property {string|null} [shard] Stream shard + * @property {topodata.ITabletAlias|null} [tablet] Stream tablet + * @property {binlogdata.IBinlogSource|null} [binlog_source] Stream binlog_source + * @property {string|null} [position] Stream position + * @property {string|null} [stop_position] Stream stop_position + * @property {string|null} [state] Stream state + * @property {string|null} [db_name] Stream db_name + * @property {vttime.ITime|null} [transaction_timestamp] Stream transaction_timestamp + * @property {vttime.ITime|null} [time_updated] Stream time_updated + * @property {string|null} [message] Stream message + * @property {Array.|null} [copy_states] Stream copy_states + * @property {Array.|null} [logs] Stream logs + * @property {string|null} [log_fetch_error] Stream log_fetch_error + * @property {Array.|null} [tags] Stream tags + */ + + /** + * Constructs a new Stream. + * @memberof vtctldata.Workflow + * @classdesc Represents a Stream. + * @implements IStream + * @constructor + * @param {vtctldata.Workflow.IStream=} [properties] Properties to set + */ + function Stream(properties) { + this.copy_states = []; + this.logs = []; + this.tags = []; + 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]]; + } + + /** + * Stream id. + * @member {number|Long} id + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream shard. + * @member {string} shard + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.shard = ""; + + /** + * Stream tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.tablet = null; + + /** + * Stream binlog_source. + * @member {binlogdata.IBinlogSource|null|undefined} binlog_source + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.binlog_source = null; + + /** + * Stream position. + * @member {string} position + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.position = ""; + + /** + * Stream stop_position. + * @member {string} stop_position + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.stop_position = ""; + + /** + * Stream state. + * @member {string} state + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.state = ""; + + /** + * Stream db_name. + * @member {string} db_name + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.db_name = ""; + + /** + * Stream transaction_timestamp. + * @member {vttime.ITime|null|undefined} transaction_timestamp + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.transaction_timestamp = null; + + /** + * Stream time_updated. + * @member {vttime.ITime|null|undefined} time_updated + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.time_updated = null; + + /** + * Stream message. + * @member {string} message + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.message = ""; + + /** + * Stream copy_states. + * @member {Array.} copy_states + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.copy_states = $util.emptyArray; + + /** + * Stream logs. + * @member {Array.} logs + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.logs = $util.emptyArray; + + /** + * Stream log_fetch_error. + * @member {string} log_fetch_error + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.log_fetch_error = ""; + + /** + * Stream tags. + * @member {Array.} tags + * @memberof vtctldata.Workflow.Stream + * @instance + */ + Stream.prototype.tags = $util.emptyArray; + + /** + * Creates a new Stream instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow.Stream + * @static + * @param {vtctldata.Workflow.IStream=} [properties] Properties to set + * @returns {vtctldata.Workflow.Stream} Stream instance + */ + Stream.create = function create(properties) { + return new Stream(properties); + }; + + /** + * Encodes the specified Stream message. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow.Stream + * @static + * @param {vtctldata.Workflow.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.binlog_source != null && Object.hasOwnProperty.call(message, "binlog_source")) + $root.binlogdata.BinlogSource.encode(message.binlog_source, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.position); + if (message.stop_position != null && Object.hasOwnProperty.call(message, "stop_position")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.stop_position); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.state); + if (message.db_name != null && Object.hasOwnProperty.call(message, "db_name")) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.db_name); + if (message.transaction_timestamp != null && Object.hasOwnProperty.call(message, "transaction_timestamp")) + $root.vttime.Time.encode(message.transaction_timestamp, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); + if (message.time_updated != null && Object.hasOwnProperty.call(message, "time_updated")) + $root.vttime.Time.encode(message.time_updated, writer.uint32(/* id 10, wireType 2 =*/82).fork()).ldelim(); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.message); + if (message.copy_states != null && message.copy_states.length) + for (let i = 0; i < message.copy_states.length; ++i) + $root.vtctldata.Workflow.Stream.CopyState.encode(message.copy_states[i], writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim(); + if (message.logs != null && message.logs.length) + for (let i = 0; i < message.logs.length; ++i) + $root.vtctldata.Workflow.Stream.Log.encode(message.logs[i], writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim(); + if (message.log_fetch_error != null && Object.hasOwnProperty.call(message, "log_fetch_error")) + writer.uint32(/* id 14, wireType 2 =*/114).string(message.log_fetch_error); + if (message.tags != null && message.tags.length) + for (let i = 0; i < message.tags.length; ++i) + writer.uint32(/* id 15, wireType 2 =*/122).string(message.tags[i]); + return writer; + }; + + /** + * Encodes the specified Stream message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow.Stream + * @static + * @param {vtctldata.Workflow.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Stream message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow.Stream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow.Stream} Stream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Stream.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.Workflow.Stream(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int64(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + message.binlog_source = $root.binlogdata.BinlogSource.decode(reader, reader.uint32()); + break; + } + case 5: { + message.position = reader.string(); + break; + } + case 6: { + message.stop_position = reader.string(); + break; + } + case 7: { + message.state = reader.string(); + break; + } + case 8: { + message.db_name = reader.string(); + break; + } + case 9: { + message.transaction_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 10: { + message.time_updated = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 11: { + message.message = reader.string(); + break; + } + case 12: { + if (!(message.copy_states && message.copy_states.length)) + message.copy_states = []; + message.copy_states.push($root.vtctldata.Workflow.Stream.CopyState.decode(reader, reader.uint32())); + break; + } + case 13: { + if (!(message.logs && message.logs.length)) + message.logs = []; + message.logs.push($root.vtctldata.Workflow.Stream.Log.decode(reader, reader.uint32())); + break; + } + case 14: { + message.log_fetch_error = reader.string(); + break; + } + case 15: { + if (!(message.tags && message.tags.length)) + message.tags = []; + message.tags.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** * Decodes a Stream message from the specified reader or buffer, length delimited. * @function decodeDelimited * @memberof vtctldata.Workflow.Stream @@ -101351,950 +104284,3849 @@ export const vtctldata = $root.vtctldata = (() => { return this.decode(reader, reader.uint32()); }; - /** - * Verifies a Stream message. - * @function verify - * @memberof vtctldata.Workflow.Stream - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Stream.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) - return "id: integer|Long expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.tablet != null && message.hasOwnProperty("tablet")) { - let error = $root.topodata.TabletAlias.verify(message.tablet); - if (error) - return "tablet." + error; - } - if (message.binlog_source != null && message.hasOwnProperty("binlog_source")) { - let error = $root.binlogdata.BinlogSource.verify(message.binlog_source); - if (error) - return "binlog_source." + error; - } - if (message.position != null && message.hasOwnProperty("position")) - if (!$util.isString(message.position)) - return "position: string expected"; - if (message.stop_position != null && message.hasOwnProperty("stop_position")) - if (!$util.isString(message.stop_position)) - return "stop_position: string expected"; - if (message.state != null && message.hasOwnProperty("state")) - if (!$util.isString(message.state)) - return "state: string expected"; - if (message.db_name != null && message.hasOwnProperty("db_name")) - if (!$util.isString(message.db_name)) - return "db_name: string expected"; - if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) { - let error = $root.vttime.Time.verify(message.transaction_timestamp); - if (error) - return "transaction_timestamp." + error; - } - if (message.time_updated != null && message.hasOwnProperty("time_updated")) { - let error = $root.vttime.Time.verify(message.time_updated); - if (error) - return "time_updated." + error; - } - if (message.message != null && message.hasOwnProperty("message")) - if (!$util.isString(message.message)) - return "message: string expected"; - if (message.copy_states != null && message.hasOwnProperty("copy_states")) { - if (!Array.isArray(message.copy_states)) - return "copy_states: array expected"; - for (let i = 0; i < message.copy_states.length; ++i) { - let error = $root.vtctldata.Workflow.Stream.CopyState.verify(message.copy_states[i]); - if (error) - return "copy_states." + error; - } - } - if (message.logs != null && message.hasOwnProperty("logs")) { - if (!Array.isArray(message.logs)) - return "logs: array expected"; - for (let i = 0; i < message.logs.length; ++i) { - let error = $root.vtctldata.Workflow.Stream.Log.verify(message.logs[i]); - if (error) - return "logs." + error; - } - } - if (message.log_fetch_error != null && message.hasOwnProperty("log_fetch_error")) - if (!$util.isString(message.log_fetch_error)) - return "log_fetch_error: string expected"; - if (message.tags != null && message.hasOwnProperty("tags")) { - if (!Array.isArray(message.tags)) - return "tags: array expected"; - for (let i = 0; i < message.tags.length; ++i) - if (!$util.isString(message.tags[i])) - return "tags: string[] expected"; - } - return null; - }; + /** + * Verifies a Stream message. + * @function verify + * @memberof vtctldata.Workflow.Stream + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Stream.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) + return "id: integer|Long expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } + if (message.binlog_source != null && message.hasOwnProperty("binlog_source")) { + let error = $root.binlogdata.BinlogSource.verify(message.binlog_source); + if (error) + return "binlog_source." + error; + } + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.stop_position != null && message.hasOwnProperty("stop_position")) + if (!$util.isString(message.stop_position)) + return "stop_position: string expected"; + if (message.state != null && message.hasOwnProperty("state")) + if (!$util.isString(message.state)) + return "state: string expected"; + if (message.db_name != null && message.hasOwnProperty("db_name")) + if (!$util.isString(message.db_name)) + return "db_name: string expected"; + if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) { + let error = $root.vttime.Time.verify(message.transaction_timestamp); + if (error) + return "transaction_timestamp." + error; + } + if (message.time_updated != null && message.hasOwnProperty("time_updated")) { + let error = $root.vttime.Time.verify(message.time_updated); + if (error) + return "time_updated." + error; + } + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.copy_states != null && message.hasOwnProperty("copy_states")) { + if (!Array.isArray(message.copy_states)) + return "copy_states: array expected"; + for (let i = 0; i < message.copy_states.length; ++i) { + let error = $root.vtctldata.Workflow.Stream.CopyState.verify(message.copy_states[i]); + if (error) + return "copy_states." + error; + } + } + if (message.logs != null && message.hasOwnProperty("logs")) { + if (!Array.isArray(message.logs)) + return "logs: array expected"; + for (let i = 0; i < message.logs.length; ++i) { + let error = $root.vtctldata.Workflow.Stream.Log.verify(message.logs[i]); + if (error) + return "logs." + error; + } + } + if (message.log_fetch_error != null && message.hasOwnProperty("log_fetch_error")) + if (!$util.isString(message.log_fetch_error)) + return "log_fetch_error: string expected"; + if (message.tags != null && message.hasOwnProperty("tags")) { + if (!Array.isArray(message.tags)) + return "tags: array expected"; + for (let i = 0; i < message.tags.length; ++i) + if (!$util.isString(message.tags[i])) + return "tags: string[] expected"; + } + return null; + }; + + /** + * Creates a Stream message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow.Stream + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow.Stream} Stream + */ + Stream.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow.Stream) + return object; + let message = new $root.vtctldata.Workflow.Stream(); + if (object.id != null) + if ($util.Long) + (message.id = $util.Long.fromValue(object.id)).unsigned = false; + else if (typeof object.id === "string") + message.id = parseInt(object.id, 10); + else if (typeof object.id === "number") + message.id = object.id; + else if (typeof object.id === "object") + message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); + if (object.shard != null) + message.shard = String(object.shard); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.Workflow.Stream.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } + if (object.binlog_source != null) { + if (typeof object.binlog_source !== "object") + throw TypeError(".vtctldata.Workflow.Stream.binlog_source: object expected"); + message.binlog_source = $root.binlogdata.BinlogSource.fromObject(object.binlog_source); + } + if (object.position != null) + message.position = String(object.position); + if (object.stop_position != null) + message.stop_position = String(object.stop_position); + if (object.state != null) + message.state = String(object.state); + if (object.db_name != null) + message.db_name = String(object.db_name); + if (object.transaction_timestamp != null) { + if (typeof object.transaction_timestamp !== "object") + throw TypeError(".vtctldata.Workflow.Stream.transaction_timestamp: object expected"); + message.transaction_timestamp = $root.vttime.Time.fromObject(object.transaction_timestamp); + } + if (object.time_updated != null) { + if (typeof object.time_updated !== "object") + throw TypeError(".vtctldata.Workflow.Stream.time_updated: object expected"); + message.time_updated = $root.vttime.Time.fromObject(object.time_updated); + } + if (object.message != null) + message.message = String(object.message); + if (object.copy_states) { + if (!Array.isArray(object.copy_states)) + throw TypeError(".vtctldata.Workflow.Stream.copy_states: array expected"); + message.copy_states = []; + for (let i = 0; i < object.copy_states.length; ++i) { + if (typeof object.copy_states[i] !== "object") + throw TypeError(".vtctldata.Workflow.Stream.copy_states: object expected"); + message.copy_states[i] = $root.vtctldata.Workflow.Stream.CopyState.fromObject(object.copy_states[i]); + } + } + if (object.logs) { + if (!Array.isArray(object.logs)) + throw TypeError(".vtctldata.Workflow.Stream.logs: array expected"); + message.logs = []; + for (let i = 0; i < object.logs.length; ++i) { + if (typeof object.logs[i] !== "object") + throw TypeError(".vtctldata.Workflow.Stream.logs: object expected"); + message.logs[i] = $root.vtctldata.Workflow.Stream.Log.fromObject(object.logs[i]); + } + } + if (object.log_fetch_error != null) + message.log_fetch_error = String(object.log_fetch_error); + if (object.tags) { + if (!Array.isArray(object.tags)) + throw TypeError(".vtctldata.Workflow.Stream.tags: array expected"); + message.tags = []; + for (let i = 0; i < object.tags.length; ++i) + message.tags[i] = String(object.tags[i]); + } + return message; + }; + + /** + * Creates a plain object from a Stream message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow.Stream + * @static + * @param {vtctldata.Workflow.Stream} message Stream + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Stream.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.copy_states = []; + object.logs = []; + object.tags = []; + } + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.id = options.longs === String ? "0" : 0; + object.shard = ""; + object.tablet = null; + object.binlog_source = null; + object.position = ""; + object.stop_position = ""; + object.state = ""; + object.db_name = ""; + object.transaction_timestamp = null; + object.time_updated = null; + object.message = ""; + object.log_fetch_error = ""; + } + if (message.id != null && message.hasOwnProperty("id")) + if (typeof message.id === "number") + object.id = options.longs === String ? String(message.id) : message.id; + else + object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (message.binlog_source != null && message.hasOwnProperty("binlog_source")) + object.binlog_source = $root.binlogdata.BinlogSource.toObject(message.binlog_source, options); + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.stop_position != null && message.hasOwnProperty("stop_position")) + object.stop_position = message.stop_position; + if (message.state != null && message.hasOwnProperty("state")) + object.state = message.state; + if (message.db_name != null && message.hasOwnProperty("db_name")) + object.db_name = message.db_name; + if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) + object.transaction_timestamp = $root.vttime.Time.toObject(message.transaction_timestamp, options); + if (message.time_updated != null && message.hasOwnProperty("time_updated")) + object.time_updated = $root.vttime.Time.toObject(message.time_updated, options); + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; + if (message.copy_states && message.copy_states.length) { + object.copy_states = []; + for (let j = 0; j < message.copy_states.length; ++j) + object.copy_states[j] = $root.vtctldata.Workflow.Stream.CopyState.toObject(message.copy_states[j], options); + } + if (message.logs && message.logs.length) { + object.logs = []; + for (let j = 0; j < message.logs.length; ++j) + object.logs[j] = $root.vtctldata.Workflow.Stream.Log.toObject(message.logs[j], options); + } + if (message.log_fetch_error != null && message.hasOwnProperty("log_fetch_error")) + object.log_fetch_error = message.log_fetch_error; + if (message.tags && message.tags.length) { + object.tags = []; + for (let j = 0; j < message.tags.length; ++j) + object.tags[j] = message.tags[j]; + } + return object; + }; + + /** + * Converts this Stream to JSON. + * @function toJSON + * @memberof vtctldata.Workflow.Stream + * @instance + * @returns {Object.} JSON object + */ + Stream.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Stream + * @function getTypeUrl + * @memberof vtctldata.Workflow.Stream + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Stream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow.Stream"; + }; + + Stream.CopyState = (function() { + + /** + * Properties of a CopyState. + * @memberof vtctldata.Workflow.Stream + * @interface ICopyState + * @property {string|null} [table] CopyState table + * @property {string|null} [last_pk] CopyState last_pk + */ + + /** + * Constructs a new CopyState. + * @memberof vtctldata.Workflow.Stream + * @classdesc Represents a CopyState. + * @implements ICopyState + * @constructor + * @param {vtctldata.Workflow.Stream.ICopyState=} [properties] Properties to set + */ + function CopyState(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]]; + } + + /** + * CopyState table. + * @member {string} table + * @memberof vtctldata.Workflow.Stream.CopyState + * @instance + */ + CopyState.prototype.table = ""; + + /** + * CopyState last_pk. + * @member {string} last_pk + * @memberof vtctldata.Workflow.Stream.CopyState + * @instance + */ + CopyState.prototype.last_pk = ""; + + /** + * Creates a new CopyState instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {vtctldata.Workflow.Stream.ICopyState=} [properties] Properties to set + * @returns {vtctldata.Workflow.Stream.CopyState} CopyState instance + */ + CopyState.create = function create(properties) { + return new CopyState(properties); + }; + + /** + * Encodes the specified CopyState message. Does not implicitly {@link vtctldata.Workflow.Stream.CopyState.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {vtctldata.Workflow.Stream.ICopyState} message CopyState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + CopyState.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.table != null && Object.hasOwnProperty.call(message, "table")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.table); + if (message.last_pk != null && Object.hasOwnProperty.call(message, "last_pk")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.last_pk); + return writer; + }; + + /** + * Encodes the specified CopyState message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.CopyState.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {vtctldata.Workflow.Stream.ICopyState} message CopyState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + CopyState.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a CopyState message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow.Stream.CopyState} CopyState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + CopyState.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.Workflow.Stream.CopyState(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.table = reader.string(); + break; + } + case 2: { + message.last_pk = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a CopyState message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Workflow.Stream.CopyState} CopyState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + CopyState.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a CopyState message. + * @function verify + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + CopyState.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.table != null && message.hasOwnProperty("table")) + if (!$util.isString(message.table)) + return "table: string expected"; + if (message.last_pk != null && message.hasOwnProperty("last_pk")) + if (!$util.isString(message.last_pk)) + return "last_pk: string expected"; + return null; + }; + + /** + * Creates a CopyState message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow.Stream.CopyState} CopyState + */ + CopyState.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow.Stream.CopyState) + return object; + let message = new $root.vtctldata.Workflow.Stream.CopyState(); + if (object.table != null) + message.table = String(object.table); + if (object.last_pk != null) + message.last_pk = String(object.last_pk); + return message; + }; + + /** + * Creates a plain object from a CopyState message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {vtctldata.Workflow.Stream.CopyState} message CopyState + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + CopyState.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.table = ""; + object.last_pk = ""; + } + if (message.table != null && message.hasOwnProperty("table")) + object.table = message.table; + if (message.last_pk != null && message.hasOwnProperty("last_pk")) + object.last_pk = message.last_pk; + return object; + }; + + /** + * Converts this CopyState to JSON. + * @function toJSON + * @memberof vtctldata.Workflow.Stream.CopyState + * @instance + * @returns {Object.} JSON object + */ + CopyState.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for CopyState + * @function getTypeUrl + * @memberof vtctldata.Workflow.Stream.CopyState + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + CopyState.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow.Stream.CopyState"; + }; + + return CopyState; + })(); + + Stream.Log = (function() { + + /** + * Properties of a Log. + * @memberof vtctldata.Workflow.Stream + * @interface ILog + * @property {number|Long|null} [id] Log id + * @property {number|Long|null} [stream_id] Log stream_id + * @property {string|null} [type] Log type + * @property {string|null} [state] Log state + * @property {vttime.ITime|null} [created_at] Log created_at + * @property {vttime.ITime|null} [updated_at] Log updated_at + * @property {string|null} [message] Log message + * @property {number|Long|null} [count] Log count + */ + + /** + * Constructs a new Log. + * @memberof vtctldata.Workflow.Stream + * @classdesc Represents a Log. + * @implements ILog + * @constructor + * @param {vtctldata.Workflow.Stream.ILog=} [properties] Properties to set + */ + function Log(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]]; + } + + /** + * Log id. + * @member {number|Long} id + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Log stream_id. + * @member {number|Long} stream_id + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.stream_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Log type. + * @member {string} type + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.type = ""; + + /** + * Log state. + * @member {string} state + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.state = ""; + + /** + * Log created_at. + * @member {vttime.ITime|null|undefined} created_at + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.created_at = null; + + /** + * Log updated_at. + * @member {vttime.ITime|null|undefined} updated_at + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.updated_at = null; + + /** + * Log message. + * @member {string} message + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.message = ""; + + /** + * Log count. + * @member {number|Long} count + * @memberof vtctldata.Workflow.Stream.Log + * @instance + */ + Log.prototype.count = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Creates a new Log instance using the specified properties. + * @function create + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {vtctldata.Workflow.Stream.ILog=} [properties] Properties to set + * @returns {vtctldata.Workflow.Stream.Log} Log instance + */ + Log.create = function create(properties) { + return new Log(properties); + }; + + /** + * Encodes the specified Log message. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. + * @function encode + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {vtctldata.Workflow.Stream.ILog} message Log message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Log.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); + if (message.stream_id != null && Object.hasOwnProperty.call(message, "stream_id")) + writer.uint32(/* id 2, wireType 0 =*/16).int64(message.stream_id); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.type); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.state); + if (message.created_at != null && Object.hasOwnProperty.call(message, "created_at")) + $root.vttime.Time.encode(message.created_at, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.updated_at != null && Object.hasOwnProperty.call(message, "updated_at")) + $root.vttime.Time.encode(message.updated_at, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.message); + if (message.count != null && Object.hasOwnProperty.call(message, "count")) + writer.uint32(/* id 8, wireType 0 =*/64).int64(message.count); + return writer; + }; + + /** + * Encodes the specified Log message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {vtctldata.Workflow.Stream.ILog} message Log message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Log.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Log message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.Workflow.Stream.Log} Log + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Log.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.Workflow.Stream.Log(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int64(); + break; + } + case 2: { + message.stream_id = reader.int64(); + break; + } + case 3: { + message.type = reader.string(); + break; + } + case 4: { + message.state = reader.string(); + break; + } + case 5: { + message.created_at = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 6: { + message.updated_at = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 7: { + message.message = reader.string(); + break; + } + case 8: { + message.count = reader.int64(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Log message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.Workflow.Stream.Log} Log + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Log.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Log message. + * @function verify + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Log.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) + return "id: integer|Long expected"; + if (message.stream_id != null && message.hasOwnProperty("stream_id")) + if (!$util.isInteger(message.stream_id) && !(message.stream_id && $util.isInteger(message.stream_id.low) && $util.isInteger(message.stream_id.high))) + return "stream_id: integer|Long expected"; + if (message.type != null && message.hasOwnProperty("type")) + if (!$util.isString(message.type)) + return "type: string expected"; + if (message.state != null && message.hasOwnProperty("state")) + if (!$util.isString(message.state)) + return "state: string expected"; + if (message.created_at != null && message.hasOwnProperty("created_at")) { + let error = $root.vttime.Time.verify(message.created_at); + if (error) + return "created_at." + error; + } + if (message.updated_at != null && message.hasOwnProperty("updated_at")) { + let error = $root.vttime.Time.verify(message.updated_at); + if (error) + return "updated_at." + error; + } + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.count != null && message.hasOwnProperty("count")) + if (!$util.isInteger(message.count) && !(message.count && $util.isInteger(message.count.low) && $util.isInteger(message.count.high))) + return "count: integer|Long expected"; + return null; + }; + + /** + * Creates a Log message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.Workflow.Stream.Log} Log + */ + Log.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.Workflow.Stream.Log) + return object; + let message = new $root.vtctldata.Workflow.Stream.Log(); + if (object.id != null) + if ($util.Long) + (message.id = $util.Long.fromValue(object.id)).unsigned = false; + else if (typeof object.id === "string") + message.id = parseInt(object.id, 10); + else if (typeof object.id === "number") + message.id = object.id; + else if (typeof object.id === "object") + message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); + if (object.stream_id != null) + if ($util.Long) + (message.stream_id = $util.Long.fromValue(object.stream_id)).unsigned = false; + else if (typeof object.stream_id === "string") + message.stream_id = parseInt(object.stream_id, 10); + else if (typeof object.stream_id === "number") + message.stream_id = object.stream_id; + else if (typeof object.stream_id === "object") + message.stream_id = new $util.LongBits(object.stream_id.low >>> 0, object.stream_id.high >>> 0).toNumber(); + if (object.type != null) + message.type = String(object.type); + if (object.state != null) + message.state = String(object.state); + if (object.created_at != null) { + if (typeof object.created_at !== "object") + throw TypeError(".vtctldata.Workflow.Stream.Log.created_at: object expected"); + message.created_at = $root.vttime.Time.fromObject(object.created_at); + } + if (object.updated_at != null) { + if (typeof object.updated_at !== "object") + throw TypeError(".vtctldata.Workflow.Stream.Log.updated_at: object expected"); + message.updated_at = $root.vttime.Time.fromObject(object.updated_at); + } + if (object.message != null) + message.message = String(object.message); + if (object.count != null) + if ($util.Long) + (message.count = $util.Long.fromValue(object.count)).unsigned = false; + else if (typeof object.count === "string") + message.count = parseInt(object.count, 10); + else if (typeof object.count === "number") + message.count = object.count; + else if (typeof object.count === "object") + message.count = new $util.LongBits(object.count.low >>> 0, object.count.high >>> 0).toNumber(); + return message; + }; + + /** + * Creates a plain object from a Log message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {vtctldata.Workflow.Stream.Log} message Log + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Log.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.id = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.stream_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.stream_id = options.longs === String ? "0" : 0; + object.type = ""; + object.state = ""; + object.created_at = null; + object.updated_at = null; + object.message = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.count = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.count = options.longs === String ? "0" : 0; + } + if (message.id != null && message.hasOwnProperty("id")) + if (typeof message.id === "number") + object.id = options.longs === String ? String(message.id) : message.id; + else + object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; + if (message.stream_id != null && message.hasOwnProperty("stream_id")) + if (typeof message.stream_id === "number") + object.stream_id = options.longs === String ? String(message.stream_id) : message.stream_id; + else + object.stream_id = options.longs === String ? $util.Long.prototype.toString.call(message.stream_id) : options.longs === Number ? new $util.LongBits(message.stream_id.low >>> 0, message.stream_id.high >>> 0).toNumber() : message.stream_id; + if (message.type != null && message.hasOwnProperty("type")) + object.type = message.type; + if (message.state != null && message.hasOwnProperty("state")) + object.state = message.state; + if (message.created_at != null && message.hasOwnProperty("created_at")) + object.created_at = $root.vttime.Time.toObject(message.created_at, options); + if (message.updated_at != null && message.hasOwnProperty("updated_at")) + object.updated_at = $root.vttime.Time.toObject(message.updated_at, options); + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; + if (message.count != null && message.hasOwnProperty("count")) + if (typeof message.count === "number") + object.count = options.longs === String ? String(message.count) : message.count; + else + object.count = options.longs === String ? $util.Long.prototype.toString.call(message.count) : options.longs === Number ? new $util.LongBits(message.count.low >>> 0, message.count.high >>> 0).toNumber() : message.count; + return object; + }; + + /** + * Converts this Log to JSON. + * @function toJSON + * @memberof vtctldata.Workflow.Stream.Log + * @instance + * @returns {Object.} JSON object + */ + Log.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Log + * @function getTypeUrl + * @memberof vtctldata.Workflow.Stream.Log + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Log.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.Workflow.Stream.Log"; + }; + + return Log; + })(); + + return Stream; + })(); + + return Workflow; + })(); + + vtctldata.AddCellInfoRequest = (function() { + + /** + * Properties of an AddCellInfoRequest. + * @memberof vtctldata + * @interface IAddCellInfoRequest + * @property {string|null} [name] AddCellInfoRequest name + * @property {topodata.ICellInfo|null} [cell_info] AddCellInfoRequest cell_info + */ + + /** + * Constructs a new AddCellInfoRequest. + * @memberof vtctldata + * @classdesc Represents an AddCellInfoRequest. + * @implements IAddCellInfoRequest + * @constructor + * @param {vtctldata.IAddCellInfoRequest=} [properties] Properties to set + */ + function AddCellInfoRequest(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]]; + } + + /** + * AddCellInfoRequest name. + * @member {string} name + * @memberof vtctldata.AddCellInfoRequest + * @instance + */ + AddCellInfoRequest.prototype.name = ""; + + /** + * AddCellInfoRequest cell_info. + * @member {topodata.ICellInfo|null|undefined} cell_info + * @memberof vtctldata.AddCellInfoRequest + * @instance + */ + AddCellInfoRequest.prototype.cell_info = null; + + /** + * Creates a new AddCellInfoRequest instance using the specified properties. + * @function create + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {vtctldata.IAddCellInfoRequest=} [properties] Properties to set + * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest instance + */ + AddCellInfoRequest.create = function create(properties) { + return new AddCellInfoRequest(properties); + }; + + /** + * Encodes the specified AddCellInfoRequest message. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {vtctldata.IAddCellInfoRequest} message AddCellInfoRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellInfoRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) + $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified AddCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {vtctldata.IAddCellInfoRequest} message AddCellInfoRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an AddCellInfoRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellInfoRequest.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.AddCellInfoRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an AddCellInfoRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an AddCellInfoRequest message. + * @function verify + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + AddCellInfoRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) { + let error = $root.topodata.CellInfo.verify(message.cell_info); + if (error) + return "cell_info." + error; + } + return null; + }; + + /** + * Creates an AddCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + */ + AddCellInfoRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.AddCellInfoRequest) + return object; + let message = new $root.vtctldata.AddCellInfoRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.cell_info != null) { + if (typeof object.cell_info !== "object") + throw TypeError(".vtctldata.AddCellInfoRequest.cell_info: object expected"); + message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + } + return message; + }; + + /** + * Creates a plain object from an AddCellInfoRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {vtctldata.AddCellInfoRequest} message AddCellInfoRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + AddCellInfoRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.name = ""; + object.cell_info = null; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) + object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); + return object; + }; + + /** + * Converts this AddCellInfoRequest to JSON. + * @function toJSON + * @memberof vtctldata.AddCellInfoRequest + * @instance + * @returns {Object.} JSON object + */ + AddCellInfoRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for AddCellInfoRequest + * @function getTypeUrl + * @memberof vtctldata.AddCellInfoRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + AddCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.AddCellInfoRequest"; + }; + + return AddCellInfoRequest; + })(); + + vtctldata.AddCellInfoResponse = (function() { + + /** + * Properties of an AddCellInfoResponse. + * @memberof vtctldata + * @interface IAddCellInfoResponse + */ + + /** + * Constructs a new AddCellInfoResponse. + * @memberof vtctldata + * @classdesc Represents an AddCellInfoResponse. + * @implements IAddCellInfoResponse + * @constructor + * @param {vtctldata.IAddCellInfoResponse=} [properties] Properties to set + */ + function AddCellInfoResponse(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 AddCellInfoResponse instance using the specified properties. + * @function create + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {vtctldata.IAddCellInfoResponse=} [properties] Properties to set + * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse instance + */ + AddCellInfoResponse.create = function create(properties) { + return new AddCellInfoResponse(properties); + }; + + /** + * Encodes the specified AddCellInfoResponse message. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {vtctldata.IAddCellInfoResponse} message AddCellInfoResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellInfoResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified AddCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {vtctldata.IAddCellInfoResponse} message AddCellInfoResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an AddCellInfoResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellInfoResponse.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.AddCellInfoResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an AddCellInfoResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an AddCellInfoResponse message. + * @function verify + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + AddCellInfoResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates an AddCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + */ + AddCellInfoResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.AddCellInfoResponse) + return object; + return new $root.vtctldata.AddCellInfoResponse(); + }; + + /** + * Creates a plain object from an AddCellInfoResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {vtctldata.AddCellInfoResponse} message AddCellInfoResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + AddCellInfoResponse.toObject = function toObject() { + return {}; + }; + + /** + * Converts this AddCellInfoResponse to JSON. + * @function toJSON + * @memberof vtctldata.AddCellInfoResponse + * @instance + * @returns {Object.} JSON object + */ + AddCellInfoResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for AddCellInfoResponse + * @function getTypeUrl + * @memberof vtctldata.AddCellInfoResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + AddCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.AddCellInfoResponse"; + }; + + return AddCellInfoResponse; + })(); + + vtctldata.AddCellsAliasRequest = (function() { + + /** + * Properties of an AddCellsAliasRequest. + * @memberof vtctldata + * @interface IAddCellsAliasRequest + * @property {string|null} [name] AddCellsAliasRequest name + * @property {Array.|null} [cells] AddCellsAliasRequest cells + */ + + /** + * Constructs a new AddCellsAliasRequest. + * @memberof vtctldata + * @classdesc Represents an AddCellsAliasRequest. + * @implements IAddCellsAliasRequest + * @constructor + * @param {vtctldata.IAddCellsAliasRequest=} [properties] Properties to set + */ + function AddCellsAliasRequest(properties) { + this.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]]; + } + + /** + * AddCellsAliasRequest name. + * @member {string} name + * @memberof vtctldata.AddCellsAliasRequest + * @instance + */ + AddCellsAliasRequest.prototype.name = ""; + + /** + * AddCellsAliasRequest cells. + * @member {Array.} cells + * @memberof vtctldata.AddCellsAliasRequest + * @instance + */ + AddCellsAliasRequest.prototype.cells = $util.emptyArray; + + /** + * Creates a new AddCellsAliasRequest instance using the specified properties. + * @function create + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {vtctldata.IAddCellsAliasRequest=} [properties] Properties to set + * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest instance + */ + AddCellsAliasRequest.create = function create(properties) { + return new AddCellsAliasRequest(properties); + }; + + /** + * Encodes the specified AddCellsAliasRequest message. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {vtctldata.IAddCellsAliasRequest} message AddCellsAliasRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellsAliasRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + return writer; + }; + + /** + * Encodes the specified AddCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {vtctldata.IAddCellsAliasRequest} message AddCellsAliasRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an AddCellsAliasRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellsAliasRequest.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.AddCellsAliasRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an AddCellsAliasRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an AddCellsAliasRequest message. + * @function verify + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + AddCellsAliasRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + return null; + }; + + /** + * Creates an AddCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + */ + AddCellsAliasRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.AddCellsAliasRequest) + return object; + let message = new $root.vtctldata.AddCellsAliasRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.AddCellsAliasRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + return message; + }; + + /** + * Creates a plain object from an AddCellsAliasRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {vtctldata.AddCellsAliasRequest} message AddCellsAliasRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + AddCellsAliasRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.cells = []; + if (options.defaults) + object.name = ""; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + return object; + }; + + /** + * Converts this AddCellsAliasRequest to JSON. + * @function toJSON + * @memberof vtctldata.AddCellsAliasRequest + * @instance + * @returns {Object.} JSON object + */ + AddCellsAliasRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for AddCellsAliasRequest + * @function getTypeUrl + * @memberof vtctldata.AddCellsAliasRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + AddCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.AddCellsAliasRequest"; + }; + + return AddCellsAliasRequest; + })(); + + vtctldata.AddCellsAliasResponse = (function() { + + /** + * Properties of an AddCellsAliasResponse. + * @memberof vtctldata + * @interface IAddCellsAliasResponse + */ + + /** + * Constructs a new AddCellsAliasResponse. + * @memberof vtctldata + * @classdesc Represents an AddCellsAliasResponse. + * @implements IAddCellsAliasResponse + * @constructor + * @param {vtctldata.IAddCellsAliasResponse=} [properties] Properties to set + */ + function AddCellsAliasResponse(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 AddCellsAliasResponse instance using the specified properties. + * @function create + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {vtctldata.IAddCellsAliasResponse=} [properties] Properties to set + * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse instance + */ + AddCellsAliasResponse.create = function create(properties) { + return new AddCellsAliasResponse(properties); + }; + + /** + * Encodes the specified AddCellsAliasResponse message. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {vtctldata.IAddCellsAliasResponse} message AddCellsAliasResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellsAliasResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified AddCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {vtctldata.IAddCellsAliasResponse} message AddCellsAliasResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + AddCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an AddCellsAliasResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellsAliasResponse.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.AddCellsAliasResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an AddCellsAliasResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + AddCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an AddCellsAliasResponse message. + * @function verify + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + AddCellsAliasResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates an AddCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + */ + AddCellsAliasResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.AddCellsAliasResponse) + return object; + return new $root.vtctldata.AddCellsAliasResponse(); + }; + + /** + * Creates a plain object from an AddCellsAliasResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {vtctldata.AddCellsAliasResponse} message AddCellsAliasResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + AddCellsAliasResponse.toObject = function toObject() { + return {}; + }; + + /** + * Converts this AddCellsAliasResponse to JSON. + * @function toJSON + * @memberof vtctldata.AddCellsAliasResponse + * @instance + * @returns {Object.} JSON object + */ + AddCellsAliasResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for AddCellsAliasResponse + * @function getTypeUrl + * @memberof vtctldata.AddCellsAliasResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + AddCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.AddCellsAliasResponse"; + }; + + return AddCellsAliasResponse; + })(); + + vtctldata.ApplyRoutingRulesRequest = (function() { + + /** + * Properties of an ApplyRoutingRulesRequest. + * @memberof vtctldata + * @interface IApplyRoutingRulesRequest + * @property {vschema.IRoutingRules|null} [routing_rules] ApplyRoutingRulesRequest routing_rules + * @property {boolean|null} [skip_rebuild] ApplyRoutingRulesRequest skip_rebuild + * @property {Array.|null} [rebuild_cells] ApplyRoutingRulesRequest rebuild_cells + */ + + /** + * Constructs a new ApplyRoutingRulesRequest. + * @memberof vtctldata + * @classdesc Represents an ApplyRoutingRulesRequest. + * @implements IApplyRoutingRulesRequest + * @constructor + * @param {vtctldata.IApplyRoutingRulesRequest=} [properties] Properties to set + */ + function ApplyRoutingRulesRequest(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]]; + } + + /** + * ApplyRoutingRulesRequest routing_rules. + * @member {vschema.IRoutingRules|null|undefined} routing_rules + * @memberof vtctldata.ApplyRoutingRulesRequest + * @instance + */ + ApplyRoutingRulesRequest.prototype.routing_rules = null; + + /** + * ApplyRoutingRulesRequest skip_rebuild. + * @member {boolean} skip_rebuild + * @memberof vtctldata.ApplyRoutingRulesRequest + * @instance + */ + ApplyRoutingRulesRequest.prototype.skip_rebuild = false; + + /** + * ApplyRoutingRulesRequest rebuild_cells. + * @member {Array.} rebuild_cells + * @memberof vtctldata.ApplyRoutingRulesRequest + * @instance + */ + ApplyRoutingRulesRequest.prototype.rebuild_cells = $util.emptyArray; + + /** + * Creates a new ApplyRoutingRulesRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {vtctldata.IApplyRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest instance + */ + ApplyRoutingRulesRequest.create = function create(properties) { + return new ApplyRoutingRulesRequest(properties); + }; + + /** + * Encodes the specified ApplyRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {vtctldata.IApplyRoutingRulesRequest} message ApplyRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyRoutingRulesRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) + $root.vschema.RoutingRules.encode(message.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 ApplyRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {vtctldata.IApplyRoutingRulesRequest} message ApplyRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyRoutingRulesRequest.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.ApplyRoutingRulesRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.routing_rules = $root.vschema.RoutingRules.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 ApplyRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyRoutingRulesRequest message. + * @function verify + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyRoutingRulesRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { + let error = $root.vschema.RoutingRules.verify(message.routing_rules); + if (error) + return "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 ApplyRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + */ + ApplyRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyRoutingRulesRequest) + return object; + let message = new $root.vtctldata.ApplyRoutingRulesRequest(); + if (object.routing_rules != null) { + if (typeof object.routing_rules !== "object") + throw TypeError(".vtctldata.ApplyRoutingRulesRequest.routing_rules: object expected"); + message.routing_rules = $root.vschema.RoutingRules.fromObject(object.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.ApplyRoutingRulesRequest.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 ApplyRoutingRulesRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {vtctldata.ApplyRoutingRulesRequest} message ApplyRoutingRulesRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyRoutingRulesRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.rebuild_cells = []; + if (options.defaults) { + object.routing_rules = null; + object.skip_rebuild = false; + } + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) + object.routing_rules = $root.vschema.RoutingRules.toObject(message.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 ApplyRoutingRulesRequest to JSON. + * @function toJSON + * @memberof vtctldata.ApplyRoutingRulesRequest + * @instance + * @returns {Object.} JSON object + */ + ApplyRoutingRulesRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyRoutingRulesRequest + * @function getTypeUrl + * @memberof vtctldata.ApplyRoutingRulesRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyRoutingRulesRequest"; + }; + + return ApplyRoutingRulesRequest; + })(); + + vtctldata.ApplyRoutingRulesResponse = (function() { + + /** + * Properties of an ApplyRoutingRulesResponse. + * @memberof vtctldata + * @interface IApplyRoutingRulesResponse + */ + + /** + * Constructs a new ApplyRoutingRulesResponse. + * @memberof vtctldata + * @classdesc Represents an ApplyRoutingRulesResponse. + * @implements IApplyRoutingRulesResponse + * @constructor + * @param {vtctldata.IApplyRoutingRulesResponse=} [properties] Properties to set + */ + function ApplyRoutingRulesResponse(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 ApplyRoutingRulesResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {vtctldata.IApplyRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse instance + */ + ApplyRoutingRulesResponse.create = function create(properties) { + return new ApplyRoutingRulesResponse(properties); + }; + + /** + * Encodes the specified ApplyRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {vtctldata.IApplyRoutingRulesResponse} message ApplyRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyRoutingRulesResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified ApplyRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {vtctldata.IApplyRoutingRulesResponse} message ApplyRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyRoutingRulesResponse.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.ApplyRoutingRulesResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyRoutingRulesResponse message. + * @function verify + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyRoutingRulesResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates an ApplyRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + */ + ApplyRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyRoutingRulesResponse) + return object; + return new $root.vtctldata.ApplyRoutingRulesResponse(); + }; + + /** + * Creates a plain object from an ApplyRoutingRulesResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {vtctldata.ApplyRoutingRulesResponse} message ApplyRoutingRulesResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyRoutingRulesResponse.toObject = function toObject() { + return {}; + }; + + /** + * Converts this ApplyRoutingRulesResponse to JSON. + * @function toJSON + * @memberof vtctldata.ApplyRoutingRulesResponse + * @instance + * @returns {Object.} JSON object + */ + ApplyRoutingRulesResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyRoutingRulesResponse + * @function getTypeUrl + * @memberof vtctldata.ApplyRoutingRulesResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyRoutingRulesResponse"; + }; + + return ApplyRoutingRulesResponse; + })(); + + vtctldata.ApplyShardRoutingRulesRequest = (function() { + + /** + * Properties of an ApplyShardRoutingRulesRequest. + * @memberof vtctldata + * @interface IApplyShardRoutingRulesRequest + * @property {vschema.IShardRoutingRules|null} [shard_routing_rules] ApplyShardRoutingRulesRequest shard_routing_rules + * @property {boolean|null} [skip_rebuild] ApplyShardRoutingRulesRequest skip_rebuild + * @property {Array.|null} [rebuild_cells] ApplyShardRoutingRulesRequest rebuild_cells + */ + + /** + * Constructs a new ApplyShardRoutingRulesRequest. + * @memberof vtctldata + * @classdesc Represents an ApplyShardRoutingRulesRequest. + * @implements IApplyShardRoutingRulesRequest + * @constructor + * @param {vtctldata.IApplyShardRoutingRulesRequest=} [properties] Properties to set + */ + function ApplyShardRoutingRulesRequest(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]]; + } + + /** + * ApplyShardRoutingRulesRequest shard_routing_rules. + * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @instance + */ + ApplyShardRoutingRulesRequest.prototype.shard_routing_rules = null; + + /** + * ApplyShardRoutingRulesRequest skip_rebuild. + * @member {boolean} skip_rebuild + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @instance + */ + ApplyShardRoutingRulesRequest.prototype.skip_rebuild = false; + + /** + * ApplyShardRoutingRulesRequest rebuild_cells. + * @member {Array.} rebuild_cells + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @instance + */ + ApplyShardRoutingRulesRequest.prototype.rebuild_cells = $util.emptyArray; + + /** + * Creates a new ApplyShardRoutingRulesRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {vtctldata.IApplyShardRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest instance + */ + ApplyShardRoutingRulesRequest.create = function create(properties) { + return new ApplyShardRoutingRulesRequest(properties); + }; + + /** + * Encodes the specified ApplyShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {vtctldata.IApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyShardRoutingRulesRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.shard_routing_rules != null && Object.hasOwnProperty.call(message, "shard_routing_rules")) + $root.vschema.ShardRoutingRules.encode(message.shard_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 ApplyShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {vtctldata.IApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyShardRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyShardRoutingRulesRequest.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.ApplyShardRoutingRulesRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.shard_routing_rules = $root.vschema.ShardRoutingRules.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 ApplyShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyShardRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyShardRoutingRulesRequest message. + * @function verify + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyShardRoutingRulesRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { + let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); + if (error) + return "shard_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 ApplyShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + */ + ApplyShardRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyShardRoutingRulesRequest) + return object; + let message = new $root.vtctldata.ApplyShardRoutingRulesRequest(); + if (object.shard_routing_rules != null) { + if (typeof object.shard_routing_rules !== "object") + throw TypeError(".vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules: object expected"); + message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_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.ApplyShardRoutingRulesRequest.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 ApplyShardRoutingRulesRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {vtctldata.ApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyShardRoutingRulesRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.rebuild_cells = []; + if (options.defaults) { + object.shard_routing_rules = null; + object.skip_rebuild = false; + } + 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.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 ApplyShardRoutingRulesRequest to JSON. + * @function toJSON + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @instance + * @returns {Object.} JSON object + */ + ApplyShardRoutingRulesRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyShardRoutingRulesRequest + * @function getTypeUrl + * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyShardRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyShardRoutingRulesRequest"; + }; + + return ApplyShardRoutingRulesRequest; + })(); + + vtctldata.ApplyShardRoutingRulesResponse = (function() { + + /** + * Properties of an ApplyShardRoutingRulesResponse. + * @memberof vtctldata + * @interface IApplyShardRoutingRulesResponse + */ + + /** + * Constructs a new ApplyShardRoutingRulesResponse. + * @memberof vtctldata + * @classdesc Represents an ApplyShardRoutingRulesResponse. + * @implements IApplyShardRoutingRulesResponse + * @constructor + * @param {vtctldata.IApplyShardRoutingRulesResponse=} [properties] Properties to set + */ + function ApplyShardRoutingRulesResponse(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 ApplyShardRoutingRulesResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {vtctldata.IApplyShardRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse instance + */ + ApplyShardRoutingRulesResponse.create = function create(properties) { + return new ApplyShardRoutingRulesResponse(properties); + }; + + /** + * Encodes the specified ApplyShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {vtctldata.IApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyShardRoutingRulesResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified ApplyShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {vtctldata.IApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyShardRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyShardRoutingRulesResponse.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.ApplyShardRoutingRulesResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyShardRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyShardRoutingRulesResponse message. + * @function verify + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyShardRoutingRulesResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates an ApplyShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + */ + ApplyShardRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyShardRoutingRulesResponse) + return object; + return new $root.vtctldata.ApplyShardRoutingRulesResponse(); + }; + + /** + * Creates a plain object from an ApplyShardRoutingRulesResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {vtctldata.ApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyShardRoutingRulesResponse.toObject = function toObject() { + return {}; + }; + + /** + * Converts this ApplyShardRoutingRulesResponse to JSON. + * @function toJSON + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @instance + * @returns {Object.} JSON object + */ + ApplyShardRoutingRulesResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyShardRoutingRulesResponse + * @function getTypeUrl + * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyShardRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyShardRoutingRulesResponse"; + }; + + return ApplyShardRoutingRulesResponse; + })(); + + vtctldata.ApplySchemaRequest = (function() { + + /** + * Properties of an ApplySchemaRequest. + * @memberof vtctldata + * @interface IApplySchemaRequest + * @property {string|null} [keyspace] ApplySchemaRequest keyspace + * @property {Array.|null} [sql] ApplySchemaRequest sql + * @property {string|null} [ddl_strategy] ApplySchemaRequest ddl_strategy + * @property {Array.|null} [uuid_list] ApplySchemaRequest uuid_list + * @property {string|null} [migration_context] ApplySchemaRequest migration_context + * @property {vttime.IDuration|null} [wait_replicas_timeout] ApplySchemaRequest wait_replicas_timeout + * @property {boolean|null} [skip_preflight] ApplySchemaRequest skip_preflight + * @property {vtrpc.ICallerID|null} [caller_id] ApplySchemaRequest caller_id + */ + + /** + * Constructs a new ApplySchemaRequest. + * @memberof vtctldata + * @classdesc Represents an ApplySchemaRequest. + * @implements IApplySchemaRequest + * @constructor + * @param {vtctldata.IApplySchemaRequest=} [properties] Properties to set + */ + function ApplySchemaRequest(properties) { + this.sql = []; + this.uuid_list = []; + 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]]; + } + + /** + * ApplySchemaRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.keyspace = ""; + + /** + * ApplySchemaRequest sql. + * @member {Array.} sql + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.sql = $util.emptyArray; + + /** + * ApplySchemaRequest ddl_strategy. + * @member {string} ddl_strategy + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.ddl_strategy = ""; + + /** + * ApplySchemaRequest uuid_list. + * @member {Array.} uuid_list + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.uuid_list = $util.emptyArray; + + /** + * ApplySchemaRequest migration_context. + * @member {string} migration_context + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.migration_context = ""; + + /** + * ApplySchemaRequest wait_replicas_timeout. + * @member {vttime.IDuration|null|undefined} wait_replicas_timeout + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.wait_replicas_timeout = null; + + /** + * ApplySchemaRequest skip_preflight. + * @member {boolean} skip_preflight + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.skip_preflight = false; + + /** + * ApplySchemaRequest caller_id. + * @member {vtrpc.ICallerID|null|undefined} caller_id + * @memberof vtctldata.ApplySchemaRequest + * @instance + */ + ApplySchemaRequest.prototype.caller_id = null; + + /** + * Creates a new ApplySchemaRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {vtctldata.IApplySchemaRequest=} [properties] Properties to set + * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest instance + */ + ApplySchemaRequest.create = function create(properties) { + return new ApplySchemaRequest(properties); + }; + + /** + * Encodes the specified ApplySchemaRequest message. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {vtctldata.IApplySchemaRequest} message ApplySchemaRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplySchemaRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.sql != null && message.sql.length) + for (let i = 0; i < message.sql.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.sql[i]); + if (message.ddl_strategy != null && Object.hasOwnProperty.call(message, "ddl_strategy")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.ddl_strategy); + if (message.uuid_list != null && message.uuid_list.length) + for (let i = 0; i < message.uuid_list.length; ++i) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.uuid_list[i]); + if (message.migration_context != null && Object.hasOwnProperty.call(message, "migration_context")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.migration_context); + if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) + $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); + if (message.skip_preflight != null && Object.hasOwnProperty.call(message, "skip_preflight")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.skip_preflight); + if (message.caller_id != null && Object.hasOwnProperty.call(message, "caller_id")) + $root.vtrpc.CallerID.encode(message.caller_id, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified ApplySchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {vtctldata.IApplySchemaRequest} message ApplySchemaRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplySchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplySchemaRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplySchemaRequest.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.ApplySchemaRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 3: { + if (!(message.sql && message.sql.length)) + message.sql = []; + message.sql.push(reader.string()); + break; + } + case 4: { + message.ddl_strategy = reader.string(); + break; + } + case 5: { + if (!(message.uuid_list && message.uuid_list.length)) + message.uuid_list = []; + message.uuid_list.push(reader.string()); + break; + } + case 6: { + message.migration_context = reader.string(); + break; + } + case 7: { + message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + break; + } + case 8: { + message.skip_preflight = reader.bool(); + break; + } + case 9: { + message.caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an ApplySchemaRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplySchemaRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplySchemaRequest message. + * @function verify + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplySchemaRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.sql != null && message.hasOwnProperty("sql")) { + if (!Array.isArray(message.sql)) + return "sql: array expected"; + for (let i = 0; i < message.sql.length; ++i) + if (!$util.isString(message.sql[i])) + return "sql: string[] expected"; + } + if (message.ddl_strategy != null && message.hasOwnProperty("ddl_strategy")) + if (!$util.isString(message.ddl_strategy)) + return "ddl_strategy: string expected"; + if (message.uuid_list != null && message.hasOwnProperty("uuid_list")) { + if (!Array.isArray(message.uuid_list)) + return "uuid_list: array expected"; + for (let i = 0; i < message.uuid_list.length; ++i) + if (!$util.isString(message.uuid_list[i])) + return "uuid_list: string[] expected"; + } + if (message.migration_context != null && message.hasOwnProperty("migration_context")) + if (!$util.isString(message.migration_context)) + return "migration_context: string expected"; + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { + let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); + if (error) + return "wait_replicas_timeout." + error; + } + if (message.skip_preflight != null && message.hasOwnProperty("skip_preflight")) + if (typeof message.skip_preflight !== "boolean") + return "skip_preflight: boolean expected"; + if (message.caller_id != null && message.hasOwnProperty("caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.caller_id); + if (error) + return "caller_id." + error; + } + return null; + }; + + /** + * Creates an ApplySchemaRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + */ + ApplySchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplySchemaRequest) + return object; + let message = new $root.vtctldata.ApplySchemaRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.sql) { + if (!Array.isArray(object.sql)) + throw TypeError(".vtctldata.ApplySchemaRequest.sql: array expected"); + message.sql = []; + for (let i = 0; i < object.sql.length; ++i) + message.sql[i] = String(object.sql[i]); + } + if (object.ddl_strategy != null) + message.ddl_strategy = String(object.ddl_strategy); + if (object.uuid_list) { + if (!Array.isArray(object.uuid_list)) + throw TypeError(".vtctldata.ApplySchemaRequest.uuid_list: array expected"); + message.uuid_list = []; + for (let i = 0; i < object.uuid_list.length; ++i) + message.uuid_list[i] = String(object.uuid_list[i]); + } + if (object.migration_context != null) + message.migration_context = String(object.migration_context); + if (object.wait_replicas_timeout != null) { + if (typeof object.wait_replicas_timeout !== "object") + throw TypeError(".vtctldata.ApplySchemaRequest.wait_replicas_timeout: object expected"); + message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + } + if (object.skip_preflight != null) + message.skip_preflight = Boolean(object.skip_preflight); + if (object.caller_id != null) { + if (typeof object.caller_id !== "object") + throw TypeError(".vtctldata.ApplySchemaRequest.caller_id: object expected"); + message.caller_id = $root.vtrpc.CallerID.fromObject(object.caller_id); + } + return message; + }; + + /** + * Creates a plain object from an ApplySchemaRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {vtctldata.ApplySchemaRequest} message ApplySchemaRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplySchemaRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.sql = []; + object.uuid_list = []; + } + if (options.defaults) { + object.keyspace = ""; + object.ddl_strategy = ""; + object.migration_context = ""; + object.wait_replicas_timeout = null; + object.skip_preflight = false; + object.caller_id = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.sql && message.sql.length) { + object.sql = []; + for (let j = 0; j < message.sql.length; ++j) + object.sql[j] = message.sql[j]; + } + if (message.ddl_strategy != null && message.hasOwnProperty("ddl_strategy")) + object.ddl_strategy = message.ddl_strategy; + if (message.uuid_list && message.uuid_list.length) { + object.uuid_list = []; + for (let j = 0; j < message.uuid_list.length; ++j) + object.uuid_list[j] = message.uuid_list[j]; + } + if (message.migration_context != null && message.hasOwnProperty("migration_context")) + object.migration_context = message.migration_context; + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) + object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); + if (message.skip_preflight != null && message.hasOwnProperty("skip_preflight")) + object.skip_preflight = message.skip_preflight; + if (message.caller_id != null && message.hasOwnProperty("caller_id")) + object.caller_id = $root.vtrpc.CallerID.toObject(message.caller_id, options); + return object; + }; + + /** + * Converts this ApplySchemaRequest to JSON. + * @function toJSON + * @memberof vtctldata.ApplySchemaRequest + * @instance + * @returns {Object.} JSON object + */ + ApplySchemaRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplySchemaRequest + * @function getTypeUrl + * @memberof vtctldata.ApplySchemaRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplySchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplySchemaRequest"; + }; - /** - * Creates a Stream message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.Workflow.Stream - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.Workflow.Stream} Stream - */ - Stream.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow.Stream) - return object; - let message = new $root.vtctldata.Workflow.Stream(); - if (object.id != null) - if ($util.Long) - (message.id = $util.Long.fromValue(object.id)).unsigned = false; - else if (typeof object.id === "string") - message.id = parseInt(object.id, 10); - else if (typeof object.id === "number") - message.id = object.id; - else if (typeof object.id === "object") - message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); - if (object.shard != null) - message.shard = String(object.shard); - if (object.tablet != null) { - if (typeof object.tablet !== "object") - throw TypeError(".vtctldata.Workflow.Stream.tablet: object expected"); - message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); - } - if (object.binlog_source != null) { - if (typeof object.binlog_source !== "object") - throw TypeError(".vtctldata.Workflow.Stream.binlog_source: object expected"); - message.binlog_source = $root.binlogdata.BinlogSource.fromObject(object.binlog_source); - } - if (object.position != null) - message.position = String(object.position); - if (object.stop_position != null) - message.stop_position = String(object.stop_position); - if (object.state != null) - message.state = String(object.state); - if (object.db_name != null) - message.db_name = String(object.db_name); - if (object.transaction_timestamp != null) { - if (typeof object.transaction_timestamp !== "object") - throw TypeError(".vtctldata.Workflow.Stream.transaction_timestamp: object expected"); - message.transaction_timestamp = $root.vttime.Time.fromObject(object.transaction_timestamp); - } - if (object.time_updated != null) { - if (typeof object.time_updated !== "object") - throw TypeError(".vtctldata.Workflow.Stream.time_updated: object expected"); - message.time_updated = $root.vttime.Time.fromObject(object.time_updated); - } - if (object.message != null) - message.message = String(object.message); - if (object.copy_states) { - if (!Array.isArray(object.copy_states)) - throw TypeError(".vtctldata.Workflow.Stream.copy_states: array expected"); - message.copy_states = []; - for (let i = 0; i < object.copy_states.length; ++i) { - if (typeof object.copy_states[i] !== "object") - throw TypeError(".vtctldata.Workflow.Stream.copy_states: object expected"); - message.copy_states[i] = $root.vtctldata.Workflow.Stream.CopyState.fromObject(object.copy_states[i]); - } - } - if (object.logs) { - if (!Array.isArray(object.logs)) - throw TypeError(".vtctldata.Workflow.Stream.logs: array expected"); - message.logs = []; - for (let i = 0; i < object.logs.length; ++i) { - if (typeof object.logs[i] !== "object") - throw TypeError(".vtctldata.Workflow.Stream.logs: object expected"); - message.logs[i] = $root.vtctldata.Workflow.Stream.Log.fromObject(object.logs[i]); - } - } - if (object.log_fetch_error != null) - message.log_fetch_error = String(object.log_fetch_error); - if (object.tags) { - if (!Array.isArray(object.tags)) - throw TypeError(".vtctldata.Workflow.Stream.tags: array expected"); - message.tags = []; - for (let i = 0; i < object.tags.length; ++i) - message.tags[i] = String(object.tags[i]); - } - return message; - }; + return ApplySchemaRequest; + })(); - /** - * Creates a plain object from a Stream message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.Workflow.Stream - * @static - * @param {vtctldata.Workflow.Stream} message Stream - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Stream.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) { - object.copy_states = []; - object.logs = []; - object.tags = []; - } - if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.id = options.longs === String ? "0" : 0; - object.shard = ""; - object.tablet = null; - object.binlog_source = null; - object.position = ""; - object.stop_position = ""; - object.state = ""; - object.db_name = ""; - object.transaction_timestamp = null; - object.time_updated = null; - object.message = ""; - object.log_fetch_error = ""; - } - if (message.id != null && message.hasOwnProperty("id")) - if (typeof message.id === "number") - object.id = options.longs === String ? String(message.id) : message.id; - else - object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.tablet != null && message.hasOwnProperty("tablet")) - object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); - if (message.binlog_source != null && message.hasOwnProperty("binlog_source")) - object.binlog_source = $root.binlogdata.BinlogSource.toObject(message.binlog_source, options); - if (message.position != null && message.hasOwnProperty("position")) - object.position = message.position; - if (message.stop_position != null && message.hasOwnProperty("stop_position")) - object.stop_position = message.stop_position; - if (message.state != null && message.hasOwnProperty("state")) - object.state = message.state; - if (message.db_name != null && message.hasOwnProperty("db_name")) - object.db_name = message.db_name; - if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) - object.transaction_timestamp = $root.vttime.Time.toObject(message.transaction_timestamp, options); - if (message.time_updated != null && message.hasOwnProperty("time_updated")) - object.time_updated = $root.vttime.Time.toObject(message.time_updated, options); - if (message.message != null && message.hasOwnProperty("message")) - object.message = message.message; - if (message.copy_states && message.copy_states.length) { - object.copy_states = []; - for (let j = 0; j < message.copy_states.length; ++j) - object.copy_states[j] = $root.vtctldata.Workflow.Stream.CopyState.toObject(message.copy_states[j], options); - } - if (message.logs && message.logs.length) { - object.logs = []; - for (let j = 0; j < message.logs.length; ++j) - object.logs[j] = $root.vtctldata.Workflow.Stream.Log.toObject(message.logs[j], options); - } - if (message.log_fetch_error != null && message.hasOwnProperty("log_fetch_error")) - object.log_fetch_error = message.log_fetch_error; - if (message.tags && message.tags.length) { - object.tags = []; - for (let j = 0; j < message.tags.length; ++j) - object.tags[j] = message.tags[j]; - } - return object; - }; + vtctldata.ApplySchemaResponse = (function() { - /** - * Converts this Stream to JSON. - * @function toJSON - * @memberof vtctldata.Workflow.Stream - * @instance - * @returns {Object.} JSON object - */ - Stream.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Properties of an ApplySchemaResponse. + * @memberof vtctldata + * @interface IApplySchemaResponse + * @property {Array.|null} [uuid_list] ApplySchemaResponse uuid_list + */ - /** - * Gets the default type url for Stream - * @function getTypeUrl - * @memberof vtctldata.Workflow.Stream - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Stream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.Workflow.Stream"; - }; + /** + * Constructs a new ApplySchemaResponse. + * @memberof vtctldata + * @classdesc Represents an ApplySchemaResponse. + * @implements IApplySchemaResponse + * @constructor + * @param {vtctldata.IApplySchemaResponse=} [properties] Properties to set + */ + function ApplySchemaResponse(properties) { + this.uuid_list = []; + 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]]; + } - Stream.CopyState = (function() { + /** + * ApplySchemaResponse uuid_list. + * @member {Array.} uuid_list + * @memberof vtctldata.ApplySchemaResponse + * @instance + */ + ApplySchemaResponse.prototype.uuid_list = $util.emptyArray; - /** - * Properties of a CopyState. - * @memberof vtctldata.Workflow.Stream - * @interface ICopyState - * @property {string|null} [table] CopyState table - * @property {string|null} [last_pk] CopyState last_pk - */ + /** + * Creates a new ApplySchemaResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {vtctldata.IApplySchemaResponse=} [properties] Properties to set + * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse instance + */ + ApplySchemaResponse.create = function create(properties) { + return new ApplySchemaResponse(properties); + }; - /** - * Constructs a new CopyState. - * @memberof vtctldata.Workflow.Stream - * @classdesc Represents a CopyState. - * @implements ICopyState - * @constructor - * @param {vtctldata.Workflow.Stream.ICopyState=} [properties] Properties to set - */ - function CopyState(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]]; + /** + * Encodes the specified ApplySchemaResponse message. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {vtctldata.IApplySchemaResponse} message ApplySchemaResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplySchemaResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.uuid_list != null && message.uuid_list.length) + for (let i = 0; i < message.uuid_list.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.uuid_list[i]); + return writer; + }; + + /** + * Encodes the specified ApplySchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {vtctldata.IApplySchemaResponse} message ApplySchemaResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplySchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplySchemaResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplySchemaResponse.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.ApplySchemaResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.uuid_list && message.uuid_list.length)) + message.uuid_list = []; + message.uuid_list.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; } + } + return message; + }; - /** - * CopyState table. - * @member {string} table - * @memberof vtctldata.Workflow.Stream.CopyState - * @instance - */ - CopyState.prototype.table = ""; + /** + * Decodes an ApplySchemaResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplySchemaResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplySchemaResponse message. + * @function verify + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplySchemaResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.uuid_list != null && message.hasOwnProperty("uuid_list")) { + if (!Array.isArray(message.uuid_list)) + return "uuid_list: array expected"; + for (let i = 0; i < message.uuid_list.length; ++i) + if (!$util.isString(message.uuid_list[i])) + return "uuid_list: string[] expected"; + } + return null; + }; + + /** + * Creates an ApplySchemaResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + */ + ApplySchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplySchemaResponse) + return object; + let message = new $root.vtctldata.ApplySchemaResponse(); + if (object.uuid_list) { + if (!Array.isArray(object.uuid_list)) + throw TypeError(".vtctldata.ApplySchemaResponse.uuid_list: array expected"); + message.uuid_list = []; + for (let i = 0; i < object.uuid_list.length; ++i) + message.uuid_list[i] = String(object.uuid_list[i]); + } + return message; + }; + + /** + * Creates a plain object from an ApplySchemaResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {vtctldata.ApplySchemaResponse} message ApplySchemaResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplySchemaResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.uuid_list = []; + if (message.uuid_list && message.uuid_list.length) { + object.uuid_list = []; + for (let j = 0; j < message.uuid_list.length; ++j) + object.uuid_list[j] = message.uuid_list[j]; + } + return object; + }; + + /** + * Converts this ApplySchemaResponse to JSON. + * @function toJSON + * @memberof vtctldata.ApplySchemaResponse + * @instance + * @returns {Object.} JSON object + */ + ApplySchemaResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * CopyState last_pk. - * @member {string} last_pk - * @memberof vtctldata.Workflow.Stream.CopyState - * @instance - */ - CopyState.prototype.last_pk = ""; + /** + * Gets the default type url for ApplySchemaResponse + * @function getTypeUrl + * @memberof vtctldata.ApplySchemaResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplySchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplySchemaResponse"; + }; - /** - * Creates a new CopyState instance using the specified properties. - * @function create - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {vtctldata.Workflow.Stream.ICopyState=} [properties] Properties to set - * @returns {vtctldata.Workflow.Stream.CopyState} CopyState instance - */ - CopyState.create = function create(properties) { - return new CopyState(properties); - }; + return ApplySchemaResponse; + })(); - /** - * Encodes the specified CopyState message. Does not implicitly {@link vtctldata.Workflow.Stream.CopyState.verify|verify} messages. - * @function encode - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {vtctldata.Workflow.Stream.ICopyState} message CopyState message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - CopyState.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.table != null && Object.hasOwnProperty.call(message, "table")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.table); - if (message.last_pk != null && Object.hasOwnProperty.call(message, "last_pk")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.last_pk); - return writer; - }; + vtctldata.ApplyVSchemaRequest = (function() { - /** - * Encodes the specified CopyState message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.CopyState.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {vtctldata.Workflow.Stream.ICopyState} message CopyState message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - CopyState.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * Properties of an ApplyVSchemaRequest. + * @memberof vtctldata + * @interface IApplyVSchemaRequest + * @property {string|null} [keyspace] ApplyVSchemaRequest keyspace + * @property {boolean|null} [skip_rebuild] ApplyVSchemaRequest skip_rebuild + * @property {boolean|null} [dry_run] ApplyVSchemaRequest dry_run + * @property {Array.|null} [cells] ApplyVSchemaRequest cells + * @property {vschema.IKeyspace|null} [v_schema] ApplyVSchemaRequest v_schema + * @property {string|null} [sql] ApplyVSchemaRequest sql + */ - /** - * Decodes a CopyState message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow.Stream.CopyState} CopyState - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - CopyState.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.Workflow.Stream.CopyState(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.table = reader.string(); - break; - } - case 2: { - message.last_pk = reader.string(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; + /** + * Constructs a new ApplyVSchemaRequest. + * @memberof vtctldata + * @classdesc Represents an ApplyVSchemaRequest. + * @implements IApplyVSchemaRequest + * @constructor + * @param {vtctldata.IApplyVSchemaRequest=} [properties] Properties to set + */ + function ApplyVSchemaRequest(properties) { + this.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]]; + } - /** - * Decodes a CopyState message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Workflow.Stream.CopyState} CopyState - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - CopyState.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + /** + * ApplyVSchemaRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.keyspace = ""; - /** - * Verifies a CopyState message. - * @function verify - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - CopyState.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.table != null && message.hasOwnProperty("table")) - if (!$util.isString(message.table)) - return "table: string expected"; - if (message.last_pk != null && message.hasOwnProperty("last_pk")) - if (!$util.isString(message.last_pk)) - return "last_pk: string expected"; - return null; - }; + /** + * ApplyVSchemaRequest skip_rebuild. + * @member {boolean} skip_rebuild + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.skip_rebuild = false; - /** - * Creates a CopyState message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.Workflow.Stream.CopyState} CopyState - */ - CopyState.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow.Stream.CopyState) - return object; - let message = new $root.vtctldata.Workflow.Stream.CopyState(); - if (object.table != null) - message.table = String(object.table); - if (object.last_pk != null) - message.last_pk = String(object.last_pk); - return message; - }; + /** + * ApplyVSchemaRequest dry_run. + * @member {boolean} dry_run + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.dry_run = false; - /** - * Creates a plain object from a CopyState message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {vtctldata.Workflow.Stream.CopyState} message CopyState - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - CopyState.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.table = ""; - object.last_pk = ""; - } - if (message.table != null && message.hasOwnProperty("table")) - object.table = message.table; - if (message.last_pk != null && message.hasOwnProperty("last_pk")) - object.last_pk = message.last_pk; - return object; - }; + /** + * ApplyVSchemaRequest cells. + * @member {Array.} cells + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.cells = $util.emptyArray; - /** - * Converts this CopyState to JSON. - * @function toJSON - * @memberof vtctldata.Workflow.Stream.CopyState - * @instance - * @returns {Object.} JSON object - */ - CopyState.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * ApplyVSchemaRequest v_schema. + * @member {vschema.IKeyspace|null|undefined} v_schema + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.v_schema = null; - /** - * Gets the default type url for CopyState - * @function getTypeUrl - * @memberof vtctldata.Workflow.Stream.CopyState - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - CopyState.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.Workflow.Stream.CopyState"; - }; + /** + * ApplyVSchemaRequest sql. + * @member {string} sql + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + */ + ApplyVSchemaRequest.prototype.sql = ""; - return CopyState; - })(); + /** + * Creates a new ApplyVSchemaRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {vtctldata.IApplyVSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest instance + */ + ApplyVSchemaRequest.create = function create(properties) { + return new ApplyVSchemaRequest(properties); + }; - Stream.Log = (function() { + /** + * Encodes the specified ApplyVSchemaRequest message. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {vtctldata.IApplyVSchemaRequest} message ApplyVSchemaRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyVSchemaRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.skip_rebuild != null && Object.hasOwnProperty.call(message, "skip_rebuild")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.skip_rebuild); + if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.dry_run); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.cells[i]); + if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) + $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.sql); + return writer; + }; - /** - * Properties of a Log. - * @memberof vtctldata.Workflow.Stream - * @interface ILog - * @property {number|Long|null} [id] Log id - * @property {number|Long|null} [stream_id] Log stream_id - * @property {string|null} [type] Log type - * @property {string|null} [state] Log state - * @property {vttime.ITime|null} [created_at] Log created_at - * @property {vttime.ITime|null} [updated_at] Log updated_at - * @property {string|null} [message] Log message - * @property {number|Long|null} [count] Log count - */ + /** + * Encodes the specified ApplyVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {vtctldata.IApplyVSchemaRequest} message ApplyVSchemaRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * Constructs a new Log. - * @memberof vtctldata.Workflow.Stream - * @classdesc Represents a Log. - * @implements ILog - * @constructor - * @param {vtctldata.Workflow.Stream.ILog=} [properties] Properties to set - */ - function Log(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]]; + /** + * Decodes an ApplyVSchemaRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyVSchemaRequest.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.ApplyVSchemaRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.skip_rebuild = reader.bool(); + break; + } + case 3: { + message.dry_run = reader.bool(); + break; + } + case 4: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 5: { + message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); + break; + } + case 6: { + message.sql = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; } + } + return message; + }; - /** - * Log id. - * @member {number|Long} id - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * Decodes an ApplyVSchemaRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Log stream_id. - * @member {number|Long} stream_id - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.stream_id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * Verifies an ApplyVSchemaRequest message. + * @function verify + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyVSchemaRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) + if (typeof message.skip_rebuild !== "boolean") + return "skip_rebuild: boolean expected"; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + if (typeof message.dry_run !== "boolean") + return "dry_run: boolean expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.v_schema != null && message.hasOwnProperty("v_schema")) { + let error = $root.vschema.Keyspace.verify(message.v_schema); + if (error) + return "v_schema." + error; + } + if (message.sql != null && message.hasOwnProperty("sql")) + if (!$util.isString(message.sql)) + return "sql: string expected"; + return null; + }; - /** - * Log type. - * @member {string} type - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.type = ""; + /** + * Creates an ApplyVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + */ + ApplyVSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyVSchemaRequest) + return object; + let message = new $root.vtctldata.ApplyVSchemaRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.skip_rebuild != null) + message.skip_rebuild = Boolean(object.skip_rebuild); + if (object.dry_run != null) + message.dry_run = Boolean(object.dry_run); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.ApplyVSchemaRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.v_schema != null) { + if (typeof object.v_schema !== "object") + throw TypeError(".vtctldata.ApplyVSchemaRequest.v_schema: object expected"); + message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); + } + if (object.sql != null) + message.sql = String(object.sql); + return message; + }; - /** - * Log state. - * @member {string} state - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.state = ""; + /** + * Creates a plain object from an ApplyVSchemaRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {vtctldata.ApplyVSchemaRequest} message ApplyVSchemaRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyVSchemaRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.cells = []; + if (options.defaults) { + object.keyspace = ""; + object.skip_rebuild = false; + object.dry_run = false; + object.v_schema = null; + object.sql = ""; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) + object.skip_rebuild = message.skip_rebuild; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + object.dry_run = message.dry_run; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.v_schema != null && message.hasOwnProperty("v_schema")) + object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); + if (message.sql != null && message.hasOwnProperty("sql")) + object.sql = message.sql; + return object; + }; - /** - * Log created_at. - * @member {vttime.ITime|null|undefined} created_at - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.created_at = null; + /** + * Converts this ApplyVSchemaRequest to JSON. + * @function toJSON + * @memberof vtctldata.ApplyVSchemaRequest + * @instance + * @returns {Object.} JSON object + */ + ApplyVSchemaRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * Log updated_at. - * @member {vttime.ITime|null|undefined} updated_at - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.updated_at = null; + /** + * Gets the default type url for ApplyVSchemaRequest + * @function getTypeUrl + * @memberof vtctldata.ApplyVSchemaRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyVSchemaRequest"; + }; - /** - * Log message. - * @member {string} message - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.message = ""; + return ApplyVSchemaRequest; + })(); - /** - * Log count. - * @member {number|Long} count - * @memberof vtctldata.Workflow.Stream.Log - * @instance - */ - Log.prototype.count = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + vtctldata.ApplyVSchemaResponse = (function() { - /** - * Creates a new Log instance using the specified properties. - * @function create - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {vtctldata.Workflow.Stream.ILog=} [properties] Properties to set - * @returns {vtctldata.Workflow.Stream.Log} Log instance - */ - Log.create = function create(properties) { - return new Log(properties); - }; + /** + * Properties of an ApplyVSchemaResponse. + * @memberof vtctldata + * @interface IApplyVSchemaResponse + * @property {vschema.IKeyspace|null} [v_schema] ApplyVSchemaResponse v_schema + */ - /** - * Encodes the specified Log message. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. - * @function encode - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {vtctldata.Workflow.Stream.ILog} message Log message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Log.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); - if (message.stream_id != null && Object.hasOwnProperty.call(message, "stream_id")) - writer.uint32(/* id 2, wireType 0 =*/16).int64(message.stream_id); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.type); - if (message.state != null && Object.hasOwnProperty.call(message, "state")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.state); - if (message.created_at != null && Object.hasOwnProperty.call(message, "created_at")) - $root.vttime.Time.encode(message.created_at, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.updated_at != null && Object.hasOwnProperty.call(message, "updated_at")) - $root.vttime.Time.encode(message.updated_at, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.message != null && Object.hasOwnProperty.call(message, "message")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.message); - if (message.count != null && Object.hasOwnProperty.call(message, "count")) - writer.uint32(/* id 8, wireType 0 =*/64).int64(message.count); - return writer; - }; + /** + * Constructs a new ApplyVSchemaResponse. + * @memberof vtctldata + * @classdesc Represents an ApplyVSchemaResponse. + * @implements IApplyVSchemaResponse + * @constructor + * @param {vtctldata.IApplyVSchemaResponse=} [properties] Properties to set + */ + function ApplyVSchemaResponse(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]]; + } - /** - * Encodes the specified Log message, length delimited. Does not implicitly {@link vtctldata.Workflow.Stream.Log.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {vtctldata.Workflow.Stream.ILog} message Log message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Log.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * ApplyVSchemaResponse v_schema. + * @member {vschema.IKeyspace|null|undefined} v_schema + * @memberof vtctldata.ApplyVSchemaResponse + * @instance + */ + ApplyVSchemaResponse.prototype.v_schema = null; - /** - * Decodes a Log message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.Workflow.Stream.Log} Log - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Log.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.Workflow.Stream.Log(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.id = reader.int64(); - break; - } - case 2: { - message.stream_id = reader.int64(); - break; - } - case 3: { - message.type = reader.string(); - break; - } - case 4: { - message.state = reader.string(); - break; - } - case 5: { - message.created_at = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 6: { - message.updated_at = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 7: { - message.message = reader.string(); - break; - } - case 8: { - message.count = reader.int64(); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; + /** + * Creates a new ApplyVSchemaResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {vtctldata.IApplyVSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse instance + */ + ApplyVSchemaResponse.create = function create(properties) { + return new ApplyVSchemaResponse(properties); + }; - /** - * Decodes a Log message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.Workflow.Stream.Log} Log - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Log.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + /** + * Encodes the specified ApplyVSchemaResponse message. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {vtctldata.IApplyVSchemaResponse} message ApplyVSchemaResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyVSchemaResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) + $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; - /** - * Verifies a Log message. - * @function verify - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Log.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) - return "id: integer|Long expected"; - if (message.stream_id != null && message.hasOwnProperty("stream_id")) - if (!$util.isInteger(message.stream_id) && !(message.stream_id && $util.isInteger(message.stream_id.low) && $util.isInteger(message.stream_id.high))) - return "stream_id: integer|Long expected"; - if (message.type != null && message.hasOwnProperty("type")) - if (!$util.isString(message.type)) - return "type: string expected"; - if (message.state != null && message.hasOwnProperty("state")) - if (!$util.isString(message.state)) - return "state: string expected"; - if (message.created_at != null && message.hasOwnProperty("created_at")) { - let error = $root.vttime.Time.verify(message.created_at); - if (error) - return "created_at." + error; - } - if (message.updated_at != null && message.hasOwnProperty("updated_at")) { - let error = $root.vttime.Time.verify(message.updated_at); - if (error) - return "updated_at." + error; - } - if (message.message != null && message.hasOwnProperty("message")) - if (!$util.isString(message.message)) - return "message: string expected"; - if (message.count != null && message.hasOwnProperty("count")) - if (!$util.isInteger(message.count) && !(message.count && $util.isInteger(message.count.low) && $util.isInteger(message.count.high))) - return "count: integer|Long expected"; - return null; - }; + /** + * Encodes the specified ApplyVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {vtctldata.IApplyVSchemaResponse} message ApplyVSchemaResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * Creates a Log message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.Workflow.Stream.Log} Log - */ - Log.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.Workflow.Stream.Log) - return object; - let message = new $root.vtctldata.Workflow.Stream.Log(); - if (object.id != null) - if ($util.Long) - (message.id = $util.Long.fromValue(object.id)).unsigned = false; - else if (typeof object.id === "string") - message.id = parseInt(object.id, 10); - else if (typeof object.id === "number") - message.id = object.id; - else if (typeof object.id === "object") - message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); - if (object.stream_id != null) - if ($util.Long) - (message.stream_id = $util.Long.fromValue(object.stream_id)).unsigned = false; - else if (typeof object.stream_id === "string") - message.stream_id = parseInt(object.stream_id, 10); - else if (typeof object.stream_id === "number") - message.stream_id = object.stream_id; - else if (typeof object.stream_id === "object") - message.stream_id = new $util.LongBits(object.stream_id.low >>> 0, object.stream_id.high >>> 0).toNumber(); - if (object.type != null) - message.type = String(object.type); - if (object.state != null) - message.state = String(object.state); - if (object.created_at != null) { - if (typeof object.created_at !== "object") - throw TypeError(".vtctldata.Workflow.Stream.Log.created_at: object expected"); - message.created_at = $root.vttime.Time.fromObject(object.created_at); - } - if (object.updated_at != null) { - if (typeof object.updated_at !== "object") - throw TypeError(".vtctldata.Workflow.Stream.Log.updated_at: object expected"); - message.updated_at = $root.vttime.Time.fromObject(object.updated_at); + /** + * Decodes an ApplyVSchemaResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyVSchemaResponse.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.ApplyVSchemaResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); + break; } - if (object.message != null) - message.message = String(object.message); - if (object.count != null) - if ($util.Long) - (message.count = $util.Long.fromValue(object.count)).unsigned = false; - else if (typeof object.count === "string") - message.count = parseInt(object.count, 10); - else if (typeof object.count === "number") - message.count = object.count; - else if (typeof object.count === "object") - message.count = new $util.LongBits(object.count.low >>> 0, object.count.high >>> 0).toNumber(); - return message; - }; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; - /** - * Creates a plain object from a Log message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {vtctldata.Workflow.Stream.Log} message Log - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Log.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.id = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.stream_id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.stream_id = options.longs === String ? "0" : 0; - object.type = ""; - object.state = ""; - object.created_at = null; - object.updated_at = null; - object.message = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.count = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.count = options.longs === String ? "0" : 0; - } - if (message.id != null && message.hasOwnProperty("id")) - if (typeof message.id === "number") - object.id = options.longs === String ? String(message.id) : message.id; - else - object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; - if (message.stream_id != null && message.hasOwnProperty("stream_id")) - if (typeof message.stream_id === "number") - object.stream_id = options.longs === String ? String(message.stream_id) : message.stream_id; - else - object.stream_id = options.longs === String ? $util.Long.prototype.toString.call(message.stream_id) : options.longs === Number ? new $util.LongBits(message.stream_id.low >>> 0, message.stream_id.high >>> 0).toNumber() : message.stream_id; - if (message.type != null && message.hasOwnProperty("type")) - object.type = message.type; - if (message.state != null && message.hasOwnProperty("state")) - object.state = message.state; - if (message.created_at != null && message.hasOwnProperty("created_at")) - object.created_at = $root.vttime.Time.toObject(message.created_at, options); - if (message.updated_at != null && message.hasOwnProperty("updated_at")) - object.updated_at = $root.vttime.Time.toObject(message.updated_at, options); - if (message.message != null && message.hasOwnProperty("message")) - object.message = message.message; - if (message.count != null && message.hasOwnProperty("count")) - if (typeof message.count === "number") - object.count = options.longs === String ? String(message.count) : message.count; - else - object.count = options.longs === String ? $util.Long.prototype.toString.call(message.count) : options.longs === Number ? new $util.LongBits(message.count.low >>> 0, message.count.high >>> 0).toNumber() : message.count; - return object; - }; + /** + * Decodes an ApplyVSchemaResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Converts this Log to JSON. - * @function toJSON - * @memberof vtctldata.Workflow.Stream.Log - * @instance - * @returns {Object.} JSON object - */ - Log.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Verifies an ApplyVSchemaResponse message. + * @function verify + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyVSchemaResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.v_schema != null && message.hasOwnProperty("v_schema")) { + let error = $root.vschema.Keyspace.verify(message.v_schema); + if (error) + return "v_schema." + error; + } + return null; + }; - /** - * Gets the default type url for Log - * @function getTypeUrl - * @memberof vtctldata.Workflow.Stream.Log - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Log.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.Workflow.Stream.Log"; - }; + /** + * Creates an ApplyVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + */ + ApplyVSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyVSchemaResponse) + return object; + let message = new $root.vtctldata.ApplyVSchemaResponse(); + if (object.v_schema != null) { + if (typeof object.v_schema !== "object") + throw TypeError(".vtctldata.ApplyVSchemaResponse.v_schema: object expected"); + message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); + } + return message; + }; - return Log; - })(); + /** + * Creates a plain object from an ApplyVSchemaResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {vtctldata.ApplyVSchemaResponse} message ApplyVSchemaResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyVSchemaResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.v_schema = null; + if (message.v_schema != null && message.hasOwnProperty("v_schema")) + object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); + return object; + }; - return Stream; - })(); + /** + * Converts this ApplyVSchemaResponse to JSON. + * @function toJSON + * @memberof vtctldata.ApplyVSchemaResponse + * @instance + * @returns {Object.} JSON object + */ + ApplyVSchemaResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - return Workflow; + /** + * Gets the default type url for ApplyVSchemaResponse + * @function getTypeUrl + * @memberof vtctldata.ApplyVSchemaResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyVSchemaResponse"; + }; + + return ApplyVSchemaResponse; })(); - vtctldata.AddCellInfoRequest = (function() { + vtctldata.BackupRequest = (function() { /** - * Properties of an AddCellInfoRequest. + * Properties of a BackupRequest. * @memberof vtctldata - * @interface IAddCellInfoRequest - * @property {string|null} [name] AddCellInfoRequest name - * @property {topodata.ICellInfo|null} [cell_info] AddCellInfoRequest cell_info + * @interface IBackupRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] BackupRequest tablet_alias + * @property {boolean|null} [allow_primary] BackupRequest allow_primary + * @property {number|Long|null} [concurrency] BackupRequest concurrency + * @property {string|null} [incremental_from_pos] BackupRequest incremental_from_pos + * @property {boolean|null} [upgrade_safe] BackupRequest upgrade_safe */ /** - * Constructs a new AddCellInfoRequest. + * Constructs a new BackupRequest. * @memberof vtctldata - * @classdesc Represents an AddCellInfoRequest. - * @implements IAddCellInfoRequest + * @classdesc Represents a BackupRequest. + * @implements IBackupRequest * @constructor - * @param {vtctldata.IAddCellInfoRequest=} [properties] Properties to set + * @param {vtctldata.IBackupRequest=} [properties] Properties to set */ - function AddCellInfoRequest(properties) { + function BackupRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -102302,89 +108134,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * AddCellInfoRequest name. - * @member {string} name - * @memberof vtctldata.AddCellInfoRequest + * BackupRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.BackupRequest * @instance */ - AddCellInfoRequest.prototype.name = ""; + BackupRequest.prototype.tablet_alias = null; /** - * AddCellInfoRequest cell_info. - * @member {topodata.ICellInfo|null|undefined} cell_info - * @memberof vtctldata.AddCellInfoRequest + * BackupRequest allow_primary. + * @member {boolean} allow_primary + * @memberof vtctldata.BackupRequest * @instance */ - AddCellInfoRequest.prototype.cell_info = null; + BackupRequest.prototype.allow_primary = false; /** - * Creates a new AddCellInfoRequest instance using the specified properties. + * BackupRequest concurrency. + * @member {number|Long} concurrency + * @memberof vtctldata.BackupRequest + * @instance + */ + BackupRequest.prototype.concurrency = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + + /** + * BackupRequest incremental_from_pos. + * @member {string} incremental_from_pos + * @memberof vtctldata.BackupRequest + * @instance + */ + BackupRequest.prototype.incremental_from_pos = ""; + + /** + * BackupRequest upgrade_safe. + * @member {boolean} upgrade_safe + * @memberof vtctldata.BackupRequest + * @instance + */ + BackupRequest.prototype.upgrade_safe = false; + + /** + * Creates a new BackupRequest instance using the specified properties. * @function create - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static - * @param {vtctldata.IAddCellInfoRequest=} [properties] Properties to set - * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest instance + * @param {vtctldata.IBackupRequest=} [properties] Properties to set + * @returns {vtctldata.BackupRequest} BackupRequest instance */ - AddCellInfoRequest.create = function create(properties) { - return new AddCellInfoRequest(properties); + BackupRequest.create = function create(properties) { + return new BackupRequest(properties); }; /** - * Encodes the specified AddCellInfoRequest message. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * Encodes the specified BackupRequest message. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static - * @param {vtctldata.IAddCellInfoRequest} message AddCellInfoRequest message or plain object to encode + * @param {vtctldata.IBackupRequest} message BackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellInfoRequest.encode = function encode(message, writer) { + BackupRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) - $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.allow_primary); + if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) + writer.uint32(/* id 3, wireType 0 =*/24).uint64(message.concurrency); + if (message.incremental_from_pos != null && Object.hasOwnProperty.call(message, "incremental_from_pos")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.incremental_from_pos); + if (message.upgrade_safe != null && Object.hasOwnProperty.call(message, "upgrade_safe")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.upgrade_safe); return writer; }; /** - * Encodes the specified AddCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoRequest.verify|verify} messages. + * Encodes the specified BackupRequest message, length delimited. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static - * @param {vtctldata.IAddCellInfoRequest} message AddCellInfoRequest message or plain object to encode + * @param {vtctldata.IBackupRequest} message BackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + BackupRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an AddCellInfoRequest message from the specified reader or buffer. + * Decodes a BackupRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + * @returns {vtctldata.BackupRequest} BackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellInfoRequest.decode = function decode(reader, length) { + BackupRequest.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.AddCellInfoRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.BackupRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); + message.allow_primary = reader.bool(); + break; + } + case 3: { + message.concurrency = reader.uint64(); + break; + } + case 4: { + message.incremental_from_pos = reader.string(); + break; + } + case 5: { + message.upgrade_safe = reader.bool(); break; } default: @@ -102396,135 +108270,177 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an AddCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a BackupRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + * @returns {vtctldata.BackupRequest} BackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { + BackupRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an AddCellInfoRequest message. + * Verifies a BackupRequest message. * @function verify - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - AddCellInfoRequest.verify = function verify(message) { + BackupRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) { - let error = $root.topodata.CellInfo.verify(message.cell_info); + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) - return "cell_info." + error; + return "tablet_alias." + error; } + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + if (typeof message.allow_primary !== "boolean") + return "allow_primary: boolean expected"; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (!$util.isInteger(message.concurrency) && !(message.concurrency && $util.isInteger(message.concurrency.low) && $util.isInteger(message.concurrency.high))) + return "concurrency: integer|Long expected"; + if (message.incremental_from_pos != null && message.hasOwnProperty("incremental_from_pos")) + if (!$util.isString(message.incremental_from_pos)) + return "incremental_from_pos: string expected"; + if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) + if (typeof message.upgrade_safe !== "boolean") + return "upgrade_safe: boolean expected"; return null; }; /** - * Creates an AddCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BackupRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.AddCellInfoRequest} AddCellInfoRequest + * @returns {vtctldata.BackupRequest} BackupRequest */ - AddCellInfoRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.AddCellInfoRequest) + BackupRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.BackupRequest) return object; - let message = new $root.vtctldata.AddCellInfoRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.cell_info != null) { - if (typeof object.cell_info !== "object") - throw TypeError(".vtctldata.AddCellInfoRequest.cell_info: object expected"); - message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + let message = new $root.vtctldata.BackupRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.BackupRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.allow_primary != null) + message.allow_primary = Boolean(object.allow_primary); + if (object.concurrency != null) + if ($util.Long) + (message.concurrency = $util.Long.fromValue(object.concurrency)).unsigned = true; + else if (typeof object.concurrency === "string") + message.concurrency = parseInt(object.concurrency, 10); + else if (typeof object.concurrency === "number") + message.concurrency = object.concurrency; + else if (typeof object.concurrency === "object") + message.concurrency = new $util.LongBits(object.concurrency.low >>> 0, object.concurrency.high >>> 0).toNumber(true); + if (object.incremental_from_pos != null) + message.incremental_from_pos = String(object.incremental_from_pos); + if (object.upgrade_safe != null) + message.upgrade_safe = Boolean(object.upgrade_safe); return message; }; /** - * Creates a plain object from an AddCellInfoRequest message. Also converts values to other types if specified. + * Creates a plain object from a BackupRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static - * @param {vtctldata.AddCellInfoRequest} message AddCellInfoRequest + * @param {vtctldata.BackupRequest} message BackupRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - AddCellInfoRequest.toObject = function toObject(message, options) { + BackupRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.name = ""; - object.cell_info = null; + object.tablet_alias = null; + object.allow_primary = false; + if ($util.Long) { + let long = new $util.Long(0, 0, true); + object.concurrency = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.concurrency = options.longs === String ? "0" : 0; + object.incremental_from_pos = ""; + object.upgrade_safe = false; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) - object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + object.allow_primary = message.allow_primary; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (typeof message.concurrency === "number") + object.concurrency = options.longs === String ? String(message.concurrency) : message.concurrency; + else + object.concurrency = options.longs === String ? $util.Long.prototype.toString.call(message.concurrency) : options.longs === Number ? new $util.LongBits(message.concurrency.low >>> 0, message.concurrency.high >>> 0).toNumber(true) : message.concurrency; + if (message.incremental_from_pos != null && message.hasOwnProperty("incremental_from_pos")) + object.incremental_from_pos = message.incremental_from_pos; + if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) + object.upgrade_safe = message.upgrade_safe; return object; }; /** - * Converts this AddCellInfoRequest to JSON. + * Converts this BackupRequest to JSON. * @function toJSON - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @instance * @returns {Object.} JSON object */ - AddCellInfoRequest.prototype.toJSON = function toJSON() { + BackupRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for AddCellInfoRequest + * Gets the default type url for BackupRequest * @function getTypeUrl - * @memberof vtctldata.AddCellInfoRequest + * @memberof vtctldata.BackupRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - AddCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.AddCellInfoRequest"; + return typeUrlPrefix + "/vtctldata.BackupRequest"; }; - return AddCellInfoRequest; + return BackupRequest; })(); - vtctldata.AddCellInfoResponse = (function() { + vtctldata.BackupResponse = (function() { /** - * Properties of an AddCellInfoResponse. + * Properties of a BackupResponse. * @memberof vtctldata - * @interface IAddCellInfoResponse + * @interface IBackupResponse + * @property {topodata.ITabletAlias|null} [tablet_alias] BackupResponse tablet_alias + * @property {string|null} [keyspace] BackupResponse keyspace + * @property {string|null} [shard] BackupResponse shard + * @property {logutil.IEvent|null} [event] BackupResponse event */ /** - * Constructs a new AddCellInfoResponse. + * Constructs a new BackupResponse. * @memberof vtctldata - * @classdesc Represents an AddCellInfoResponse. - * @implements IAddCellInfoResponse + * @classdesc Represents a BackupResponse. + * @implements IBackupResponse * @constructor - * @param {vtctldata.IAddCellInfoResponse=} [properties] Properties to set + * @param {vtctldata.IBackupResponse=} [properties] Properties to set */ - function AddCellInfoResponse(properties) { + function BackupResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -102532,63 +108448,119 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new AddCellInfoResponse instance using the specified properties. + * BackupResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.BackupResponse + * @instance + */ + BackupResponse.prototype.tablet_alias = null; + + /** + * BackupResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.BackupResponse + * @instance + */ + BackupResponse.prototype.keyspace = ""; + + /** + * BackupResponse shard. + * @member {string} shard + * @memberof vtctldata.BackupResponse + * @instance + */ + BackupResponse.prototype.shard = ""; + + /** + * BackupResponse event. + * @member {logutil.IEvent|null|undefined} event + * @memberof vtctldata.BackupResponse + * @instance + */ + BackupResponse.prototype.event = null; + + /** + * Creates a new BackupResponse instance using the specified properties. * @function create - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static - * @param {vtctldata.IAddCellInfoResponse=} [properties] Properties to set - * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse instance + * @param {vtctldata.IBackupResponse=} [properties] Properties to set + * @returns {vtctldata.BackupResponse} BackupResponse instance */ - AddCellInfoResponse.create = function create(properties) { - return new AddCellInfoResponse(properties); + BackupResponse.create = function create(properties) { + return new BackupResponse(properties); }; /** - * Encodes the specified AddCellInfoResponse message. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * Encodes the specified BackupResponse message. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static - * @param {vtctldata.IAddCellInfoResponse} message AddCellInfoResponse message or plain object to encode + * @param {vtctldata.IBackupResponse} message BackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellInfoResponse.encode = function encode(message, writer) { + BackupResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); + if (message.event != null && Object.hasOwnProperty.call(message, "event")) + $root.logutil.Event.encode(message.event, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified AddCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellInfoResponse.verify|verify} messages. + * Encodes the specified BackupResponse message, length delimited. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static - * @param {vtctldata.IAddCellInfoResponse} message AddCellInfoResponse message or plain object to encode + * @param {vtctldata.IBackupResponse} message BackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + BackupResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an AddCellInfoResponse message from the specified reader or buffer. + * Decodes a BackupResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + * @returns {vtctldata.BackupResponse} BackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellInfoResponse.decode = function decode(reader, length) { + BackupResponse.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.AddCellInfoResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.BackupResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.keyspace = reader.string(); + break; + } + case 3: { + message.shard = reader.string(); + break; + } + case 4: { + message.event = $root.logutil.Event.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -102598,111 +108570,161 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an AddCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a BackupResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + * @returns {vtctldata.BackupResponse} BackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { + BackupResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an AddCellInfoResponse message. + * Verifies a BackupResponse message. * @function verify - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - AddCellInfoResponse.verify = function verify(message) { + BackupResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.event != null && message.hasOwnProperty("event")) { + let error = $root.logutil.Event.verify(message.event); + if (error) + return "event." + error; + } return null; }; /** - * Creates an AddCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a BackupResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.AddCellInfoResponse} AddCellInfoResponse + * @returns {vtctldata.BackupResponse} BackupResponse */ - AddCellInfoResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.AddCellInfoResponse) + BackupResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.BackupResponse) return object; - return new $root.vtctldata.AddCellInfoResponse(); + let message = new $root.vtctldata.BackupResponse(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.BackupResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.event != null) { + if (typeof object.event !== "object") + throw TypeError(".vtctldata.BackupResponse.event: object expected"); + message.event = $root.logutil.Event.fromObject(object.event); + } + return message; }; /** - * Creates a plain object from an AddCellInfoResponse message. Also converts values to other types if specified. + * Creates a plain object from a BackupResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static - * @param {vtctldata.AddCellInfoResponse} message AddCellInfoResponse + * @param {vtctldata.BackupResponse} message BackupResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - AddCellInfoResponse.toObject = function toObject() { - return {}; + BackupResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet_alias = null; + object.keyspace = ""; + object.shard = ""; + object.event = null; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.event != null && message.hasOwnProperty("event")) + object.event = $root.logutil.Event.toObject(message.event, options); + return object; }; /** - * Converts this AddCellInfoResponse to JSON. + * Converts this BackupResponse to JSON. * @function toJSON - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @instance * @returns {Object.} JSON object */ - AddCellInfoResponse.prototype.toJSON = function toJSON() { + BackupResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for AddCellInfoResponse + * Gets the default type url for BackupResponse * @function getTypeUrl - * @memberof vtctldata.AddCellInfoResponse + * @memberof vtctldata.BackupResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - AddCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.AddCellInfoResponse"; + return typeUrlPrefix + "/vtctldata.BackupResponse"; }; - return AddCellInfoResponse; + return BackupResponse; })(); - vtctldata.AddCellsAliasRequest = (function() { + vtctldata.BackupShardRequest = (function() { /** - * Properties of an AddCellsAliasRequest. + * Properties of a BackupShardRequest. * @memberof vtctldata - * @interface IAddCellsAliasRequest - * @property {string|null} [name] AddCellsAliasRequest name - * @property {Array.|null} [cells] AddCellsAliasRequest cells + * @interface IBackupShardRequest + * @property {string|null} [keyspace] BackupShardRequest keyspace + * @property {string|null} [shard] BackupShardRequest shard + * @property {boolean|null} [allow_primary] BackupShardRequest allow_primary + * @property {number|Long|null} [concurrency] BackupShardRequest concurrency + * @property {boolean|null} [upgrade_safe] BackupShardRequest upgrade_safe */ /** - * Constructs a new AddCellsAliasRequest. + * Constructs a new BackupShardRequest. * @memberof vtctldata - * @classdesc Represents an AddCellsAliasRequest. - * @implements IAddCellsAliasRequest + * @classdesc Represents a BackupShardRequest. + * @implements IBackupShardRequest * @constructor - * @param {vtctldata.IAddCellsAliasRequest=} [properties] Properties to set + * @param {vtctldata.IBackupShardRequest=} [properties] Properties to set */ - function AddCellsAliasRequest(properties) { - this.cells = []; + function BackupShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -102710,92 +108732,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * AddCellsAliasRequest name. - * @member {string} name - * @memberof vtctldata.AddCellsAliasRequest + * BackupShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.BackupShardRequest * @instance */ - AddCellsAliasRequest.prototype.name = ""; + BackupShardRequest.prototype.keyspace = ""; /** - * AddCellsAliasRequest cells. - * @member {Array.} cells - * @memberof vtctldata.AddCellsAliasRequest + * BackupShardRequest shard. + * @member {string} shard + * @memberof vtctldata.BackupShardRequest * @instance */ - AddCellsAliasRequest.prototype.cells = $util.emptyArray; + BackupShardRequest.prototype.shard = ""; /** - * Creates a new AddCellsAliasRequest instance using the specified properties. + * BackupShardRequest allow_primary. + * @member {boolean} allow_primary + * @memberof vtctldata.BackupShardRequest + * @instance + */ + BackupShardRequest.prototype.allow_primary = false; + + /** + * BackupShardRequest concurrency. + * @member {number|Long} concurrency + * @memberof vtctldata.BackupShardRequest + * @instance + */ + BackupShardRequest.prototype.concurrency = $util.Long ? $util.Long.fromBits(0,0,true) : 0; + + /** + * BackupShardRequest upgrade_safe. + * @member {boolean} upgrade_safe + * @memberof vtctldata.BackupShardRequest + * @instance + */ + BackupShardRequest.prototype.upgrade_safe = false; + + /** + * Creates a new BackupShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static - * @param {vtctldata.IAddCellsAliasRequest=} [properties] Properties to set - * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest instance + * @param {vtctldata.IBackupShardRequest=} [properties] Properties to set + * @returns {vtctldata.BackupShardRequest} BackupShardRequest instance */ - AddCellsAliasRequest.create = function create(properties) { - return new AddCellsAliasRequest(properties); + BackupShardRequest.create = function create(properties) { + return new BackupShardRequest(properties); }; /** - * Encodes the specified AddCellsAliasRequest message. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * Encodes the specified BackupShardRequest message. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static - * @param {vtctldata.IAddCellsAliasRequest} message AddCellsAliasRequest message or plain object to encode + * @param {vtctldata.IBackupShardRequest} message BackupShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellsAliasRequest.encode = function encode(message, writer) { + BackupShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_primary); + if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) + writer.uint32(/* id 4, wireType 0 =*/32).uint64(message.concurrency); + if (message.upgrade_safe != null && Object.hasOwnProperty.call(message, "upgrade_safe")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.upgrade_safe); return writer; }; /** - * Encodes the specified AddCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasRequest.verify|verify} messages. + * Encodes the specified BackupShardRequest message, length delimited. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static - * @param {vtctldata.IAddCellsAliasRequest} message AddCellsAliasRequest message or plain object to encode + * @param {vtctldata.IBackupShardRequest} message BackupShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { + BackupShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an AddCellsAliasRequest message from the specified reader or buffer. + * Decodes a BackupShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + * @returns {vtctldata.BackupShardRequest} BackupShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellsAliasRequest.decode = function decode(reader, length) { + BackupShardRequest.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.AddCellsAliasRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.BackupShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.keyspace = reader.string(); break; } case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + message.shard = reader.string(); + break; + } + case 3: { + message.allow_primary = reader.bool(); + break; + } + case 4: { + message.concurrency = reader.uint64(); + break; + } + case 5: { + message.upgrade_safe = reader.bool(); break; } default: @@ -102807,142 +108868,171 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an AddCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes a BackupShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + * @returns {vtctldata.BackupShardRequest} BackupShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { + BackupShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an AddCellsAliasRequest message. + * Verifies a BackupShardRequest message. * @function verify - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - AddCellsAliasRequest.verify = function verify(message) { + BackupShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + if (typeof message.allow_primary !== "boolean") + return "allow_primary: boolean expected"; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (!$util.isInteger(message.concurrency) && !(message.concurrency && $util.isInteger(message.concurrency.low) && $util.isInteger(message.concurrency.high))) + return "concurrency: integer|Long expected"; + if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) + if (typeof message.upgrade_safe !== "boolean") + return "upgrade_safe: boolean expected"; return null; }; /** - * Creates an AddCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a BackupShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.AddCellsAliasRequest} AddCellsAliasRequest + * @returns {vtctldata.BackupShardRequest} BackupShardRequest */ - AddCellsAliasRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.AddCellsAliasRequest) + BackupShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.BackupShardRequest) return object; - let message = new $root.vtctldata.AddCellsAliasRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.AddCellsAliasRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } + let message = new $root.vtctldata.BackupShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.allow_primary != null) + message.allow_primary = Boolean(object.allow_primary); + if (object.concurrency != null) + if ($util.Long) + (message.concurrency = $util.Long.fromValue(object.concurrency)).unsigned = true; + else if (typeof object.concurrency === "string") + message.concurrency = parseInt(object.concurrency, 10); + else if (typeof object.concurrency === "number") + message.concurrency = object.concurrency; + else if (typeof object.concurrency === "object") + message.concurrency = new $util.LongBits(object.concurrency.low >>> 0, object.concurrency.high >>> 0).toNumber(true); + if (object.upgrade_safe != null) + message.upgrade_safe = Boolean(object.upgrade_safe); return message; }; /** - * Creates a plain object from an AddCellsAliasRequest message. Also converts values to other types if specified. + * Creates a plain object from a BackupShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static - * @param {vtctldata.AddCellsAliasRequest} message AddCellsAliasRequest + * @param {vtctldata.BackupShardRequest} message BackupShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - AddCellsAliasRequest.toObject = function toObject(message, options) { + BackupShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; - if (options.defaults) - object.name = ""; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.allow_primary = false; + if ($util.Long) { + let long = new $util.Long(0, 0, true); + object.concurrency = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.concurrency = options.longs === String ? "0" : 0; + object.upgrade_safe = false; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + object.allow_primary = message.allow_primary; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (typeof message.concurrency === "number") + object.concurrency = options.longs === String ? String(message.concurrency) : message.concurrency; + else + object.concurrency = options.longs === String ? $util.Long.prototype.toString.call(message.concurrency) : options.longs === Number ? new $util.LongBits(message.concurrency.low >>> 0, message.concurrency.high >>> 0).toNumber(true) : message.concurrency; + if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) + object.upgrade_safe = message.upgrade_safe; return object; }; /** - * Converts this AddCellsAliasRequest to JSON. + * Converts this BackupShardRequest to JSON. * @function toJSON - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @instance * @returns {Object.} JSON object */ - AddCellsAliasRequest.prototype.toJSON = function toJSON() { + BackupShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for AddCellsAliasRequest + * Gets the default type url for BackupShardRequest * @function getTypeUrl - * @memberof vtctldata.AddCellsAliasRequest + * @memberof vtctldata.BackupShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - AddCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + BackupShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.AddCellsAliasRequest"; + return typeUrlPrefix + "/vtctldata.BackupShardRequest"; }; - return AddCellsAliasRequest; + return BackupShardRequest; })(); - vtctldata.AddCellsAliasResponse = (function() { + vtctldata.ChangeTabletTypeRequest = (function() { /** - * Properties of an AddCellsAliasResponse. + * Properties of a ChangeTabletTypeRequest. * @memberof vtctldata - * @interface IAddCellsAliasResponse + * @interface IChangeTabletTypeRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] ChangeTabletTypeRequest tablet_alias + * @property {topodata.TabletType|null} [db_type] ChangeTabletTypeRequest db_type + * @property {boolean|null} [dry_run] ChangeTabletTypeRequest dry_run */ /** - * Constructs a new AddCellsAliasResponse. + * Constructs a new ChangeTabletTypeRequest. * @memberof vtctldata - * @classdesc Represents an AddCellsAliasResponse. - * @implements IAddCellsAliasResponse + * @classdesc Represents a ChangeTabletTypeRequest. + * @implements IChangeTabletTypeRequest * @constructor - * @param {vtctldata.IAddCellsAliasResponse=} [properties] Properties to set + * @param {vtctldata.IChangeTabletTypeRequest=} [properties] Properties to set */ - function AddCellsAliasResponse(properties) { + function ChangeTabletTypeRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -102950,63 +109040,105 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new AddCellsAliasResponse instance using the specified properties. + * ChangeTabletTypeRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ChangeTabletTypeRequest + * @instance + */ + ChangeTabletTypeRequest.prototype.tablet_alias = null; + + /** + * ChangeTabletTypeRequest db_type. + * @member {topodata.TabletType} db_type + * @memberof vtctldata.ChangeTabletTypeRequest + * @instance + */ + ChangeTabletTypeRequest.prototype.db_type = 0; + + /** + * ChangeTabletTypeRequest dry_run. + * @member {boolean} dry_run + * @memberof vtctldata.ChangeTabletTypeRequest + * @instance + */ + ChangeTabletTypeRequest.prototype.dry_run = false; + + /** + * Creates a new ChangeTabletTypeRequest instance using the specified properties. * @function create - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static - * @param {vtctldata.IAddCellsAliasResponse=} [properties] Properties to set - * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse instance + * @param {vtctldata.IChangeTabletTypeRequest=} [properties] Properties to set + * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest instance */ - AddCellsAliasResponse.create = function create(properties) { - return new AddCellsAliasResponse(properties); + ChangeTabletTypeRequest.create = function create(properties) { + return new ChangeTabletTypeRequest(properties); }; /** - * Encodes the specified AddCellsAliasResponse message. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * Encodes the specified ChangeTabletTypeRequest message. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static - * @param {vtctldata.IAddCellsAliasResponse} message AddCellsAliasResponse message or plain object to encode + * @param {vtctldata.IChangeTabletTypeRequest} message ChangeTabletTypeRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellsAliasResponse.encode = function encode(message, writer) { + ChangeTabletTypeRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.db_type != null && Object.hasOwnProperty.call(message, "db_type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.db_type); + if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.dry_run); return writer; }; /** - * Encodes the specified AddCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.AddCellsAliasResponse.verify|verify} messages. + * Encodes the specified ChangeTabletTypeRequest message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static - * @param {vtctldata.IAddCellsAliasResponse} message AddCellsAliasResponse message or plain object to encode + * @param {vtctldata.IChangeTabletTypeRequest} message ChangeTabletTypeRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - AddCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { + ChangeTabletTypeRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an AddCellsAliasResponse message from the specified reader or buffer. + * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellsAliasResponse.decode = function decode(reader, length) { + ChangeTabletTypeRequest.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.AddCellsAliasResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ChangeTabletTypeRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.db_type = reader.int32(); + break; + } + case 3: { + message.dry_run = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -103016,112 +109148,210 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an AddCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - AddCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { + ChangeTabletTypeRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an AddCellsAliasResponse message. + * Verifies a ChangeTabletTypeRequest message. * @function verify - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - AddCellsAliasResponse.verify = function verify(message) { + ChangeTabletTypeRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.db_type != null && message.hasOwnProperty("db_type")) + switch (message.db_type) { + default: + return "db_type: enum value expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + if (typeof message.dry_run !== "boolean") + return "dry_run: boolean expected"; return null; }; /** - * Creates an AddCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ChangeTabletTypeRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.AddCellsAliasResponse} AddCellsAliasResponse + * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest */ - AddCellsAliasResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.AddCellsAliasResponse) + ChangeTabletTypeRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ChangeTabletTypeRequest) return object; - return new $root.vtctldata.AddCellsAliasResponse(); + let message = new $root.vtctldata.ChangeTabletTypeRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ChangeTabletTypeRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + switch (object.db_type) { + default: + if (typeof object.db_type === "number") { + message.db_type = object.db_type; + break; + } + break; + case "UNKNOWN": + case 0: + message.db_type = 0; + break; + case "PRIMARY": + case 1: + message.db_type = 1; + break; + case "MASTER": + case 1: + message.db_type = 1; + break; + case "REPLICA": + case 2: + message.db_type = 2; + break; + case "RDONLY": + case 3: + message.db_type = 3; + break; + case "BATCH": + case 3: + message.db_type = 3; + break; + case "SPARE": + case 4: + message.db_type = 4; + break; + case "EXPERIMENTAL": + case 5: + message.db_type = 5; + break; + case "BACKUP": + case 6: + message.db_type = 6; + break; + case "RESTORE": + case 7: + message.db_type = 7; + break; + case "DRAINED": + case 8: + message.db_type = 8; + break; + } + if (object.dry_run != null) + message.dry_run = Boolean(object.dry_run); + return message; }; /** - * Creates a plain object from an AddCellsAliasResponse message. Also converts values to other types if specified. + * Creates a plain object from a ChangeTabletTypeRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static - * @param {vtctldata.AddCellsAliasResponse} message AddCellsAliasResponse + * @param {vtctldata.ChangeTabletTypeRequest} message ChangeTabletTypeRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - AddCellsAliasResponse.toObject = function toObject() { - return {}; + ChangeTabletTypeRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet_alias = null; + object.db_type = options.enums === String ? "UNKNOWN" : 0; + object.dry_run = false; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.db_type != null && message.hasOwnProperty("db_type")) + object.db_type = options.enums === String ? $root.topodata.TabletType[message.db_type] === undefined ? message.db_type : $root.topodata.TabletType[message.db_type] : message.db_type; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + object.dry_run = message.dry_run; + return object; }; /** - * Converts this AddCellsAliasResponse to JSON. + * Converts this ChangeTabletTypeRequest to JSON. * @function toJSON - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @instance * @returns {Object.} JSON object */ - AddCellsAliasResponse.prototype.toJSON = function toJSON() { + ChangeTabletTypeRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for AddCellsAliasResponse + * Gets the default type url for ChangeTabletTypeRequest * @function getTypeUrl - * @memberof vtctldata.AddCellsAliasResponse + * @memberof vtctldata.ChangeTabletTypeRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - AddCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ChangeTabletTypeRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.AddCellsAliasResponse"; + return typeUrlPrefix + "/vtctldata.ChangeTabletTypeRequest"; }; - return AddCellsAliasResponse; + return ChangeTabletTypeRequest; })(); - vtctldata.ApplyRoutingRulesRequest = (function() { + vtctldata.ChangeTabletTypeResponse = (function() { /** - * Properties of an ApplyRoutingRulesRequest. + * Properties of a ChangeTabletTypeResponse. * @memberof vtctldata - * @interface IApplyRoutingRulesRequest - * @property {vschema.IRoutingRules|null} [routing_rules] ApplyRoutingRulesRequest routing_rules - * @property {boolean|null} [skip_rebuild] ApplyRoutingRulesRequest skip_rebuild - * @property {Array.|null} [rebuild_cells] ApplyRoutingRulesRequest rebuild_cells + * @interface IChangeTabletTypeResponse + * @property {topodata.ITablet|null} [before_tablet] ChangeTabletTypeResponse before_tablet + * @property {topodata.ITablet|null} [after_tablet] ChangeTabletTypeResponse after_tablet + * @property {boolean|null} [was_dry_run] ChangeTabletTypeResponse was_dry_run */ /** - * Constructs a new ApplyRoutingRulesRequest. + * Constructs a new ChangeTabletTypeResponse. * @memberof vtctldata - * @classdesc Represents an ApplyRoutingRulesRequest. - * @implements IApplyRoutingRulesRequest + * @classdesc Represents a ChangeTabletTypeResponse. + * @implements IChangeTabletTypeResponse * @constructor - * @param {vtctldata.IApplyRoutingRulesRequest=} [properties] Properties to set + * @param {vtctldata.IChangeTabletTypeResponse=} [properties] Properties to set */ - function ApplyRoutingRulesRequest(properties) { - this.rebuild_cells = []; + function ChangeTabletTypeResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -103129,106 +109359,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplyRoutingRulesRequest routing_rules. - * @member {vschema.IRoutingRules|null|undefined} routing_rules - * @memberof vtctldata.ApplyRoutingRulesRequest + * ChangeTabletTypeResponse before_tablet. + * @member {topodata.ITablet|null|undefined} before_tablet + * @memberof vtctldata.ChangeTabletTypeResponse * @instance */ - ApplyRoutingRulesRequest.prototype.routing_rules = null; + ChangeTabletTypeResponse.prototype.before_tablet = null; /** - * ApplyRoutingRulesRequest skip_rebuild. - * @member {boolean} skip_rebuild - * @memberof vtctldata.ApplyRoutingRulesRequest + * ChangeTabletTypeResponse after_tablet. + * @member {topodata.ITablet|null|undefined} after_tablet + * @memberof vtctldata.ChangeTabletTypeResponse * @instance */ - ApplyRoutingRulesRequest.prototype.skip_rebuild = false; + ChangeTabletTypeResponse.prototype.after_tablet = null; /** - * ApplyRoutingRulesRequest rebuild_cells. - * @member {Array.} rebuild_cells - * @memberof vtctldata.ApplyRoutingRulesRequest + * ChangeTabletTypeResponse was_dry_run. + * @member {boolean} was_dry_run + * @memberof vtctldata.ChangeTabletTypeResponse * @instance */ - ApplyRoutingRulesRequest.prototype.rebuild_cells = $util.emptyArray; + ChangeTabletTypeResponse.prototype.was_dry_run = false; /** - * Creates a new ApplyRoutingRulesRequest instance using the specified properties. + * Creates a new ChangeTabletTypeResponse instance using the specified properties. * @function create - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static - * @param {vtctldata.IApplyRoutingRulesRequest=} [properties] Properties to set - * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest instance + * @param {vtctldata.IChangeTabletTypeResponse=} [properties] Properties to set + * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse instance */ - ApplyRoutingRulesRequest.create = function create(properties) { - return new ApplyRoutingRulesRequest(properties); + ChangeTabletTypeResponse.create = function create(properties) { + return new ChangeTabletTypeResponse(properties); }; /** - * Encodes the specified ApplyRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * Encodes the specified ChangeTabletTypeResponse message. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static - * @param {vtctldata.IApplyRoutingRulesRequest} message ApplyRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IChangeTabletTypeResponse} message ChangeTabletTypeResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyRoutingRulesRequest.encode = function encode(message, writer) { + ChangeTabletTypeResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) - $root.vschema.RoutingRules.encode(message.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]); + if (message.before_tablet != null && Object.hasOwnProperty.call(message, "before_tablet")) + $root.topodata.Tablet.encode(message.before_tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.after_tablet != null && Object.hasOwnProperty.call(message, "after_tablet")) + $root.topodata.Tablet.encode(message.after_tablet, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.was_dry_run != null && Object.hasOwnProperty.call(message, "was_dry_run")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.was_dry_run); return writer; }; /** - * Encodes the specified ApplyRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesRequest.verify|verify} messages. + * Encodes the specified ChangeTabletTypeResponse message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static - * @param {vtctldata.IApplyRoutingRulesRequest} message ApplyRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IChangeTabletTypeResponse} message ChangeTabletTypeResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + ChangeTabletTypeResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer. + * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyRoutingRulesRequest.decode = function decode(reader, length) { + ChangeTabletTypeResponse.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.ApplyRoutingRulesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ChangeTabletTypeResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.routing_rules = $root.vschema.RoutingRules.decode(reader, reader.uint32()); + message.before_tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); break; } case 2: { - message.skip_rebuild = reader.bool(); + message.after_tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); break; } case 3: { - if (!(message.rebuild_cells && message.rebuild_cells.length)) - message.rebuild_cells = []; - message.rebuild_cells.push(reader.string()); + message.was_dry_run = reader.bool(); break; } default: @@ -103240,156 +109467,158 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + ChangeTabletTypeResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyRoutingRulesRequest message. + * Verifies a ChangeTabletTypeResponse message. * @function verify - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyRoutingRulesRequest.verify = function verify(message) { + ChangeTabletTypeResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { - let error = $root.vschema.RoutingRules.verify(message.routing_rules); + if (message.before_tablet != null && message.hasOwnProperty("before_tablet")) { + let error = $root.topodata.Tablet.verify(message.before_tablet); if (error) - return "routing_rules." + error; + return "before_tablet." + 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"; + if (message.after_tablet != null && message.hasOwnProperty("after_tablet")) { + let error = $root.topodata.Tablet.verify(message.after_tablet); + if (error) + return "after_tablet." + error; } + if (message.was_dry_run != null && message.hasOwnProperty("was_dry_run")) + if (typeof message.was_dry_run !== "boolean") + return "was_dry_run: boolean expected"; return null; }; /** - * Creates an ApplyRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ChangeTabletTypeResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyRoutingRulesRequest} ApplyRoutingRulesRequest + * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse */ - ApplyRoutingRulesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyRoutingRulesRequest) + ChangeTabletTypeResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ChangeTabletTypeResponse) return object; - let message = new $root.vtctldata.ApplyRoutingRulesRequest(); - if (object.routing_rules != null) { - if (typeof object.routing_rules !== "object") - throw TypeError(".vtctldata.ApplyRoutingRulesRequest.routing_rules: object expected"); - message.routing_rules = $root.vschema.RoutingRules.fromObject(object.routing_rules); + let message = new $root.vtctldata.ChangeTabletTypeResponse(); + if (object.before_tablet != null) { + if (typeof object.before_tablet !== "object") + throw TypeError(".vtctldata.ChangeTabletTypeResponse.before_tablet: object expected"); + message.before_tablet = $root.topodata.Tablet.fromObject(object.before_tablet); } - 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.ApplyRoutingRulesRequest.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]); + if (object.after_tablet != null) { + if (typeof object.after_tablet !== "object") + throw TypeError(".vtctldata.ChangeTabletTypeResponse.after_tablet: object expected"); + message.after_tablet = $root.topodata.Tablet.fromObject(object.after_tablet); } + if (object.was_dry_run != null) + message.was_dry_run = Boolean(object.was_dry_run); return message; }; /** - * Creates a plain object from an ApplyRoutingRulesRequest message. Also converts values to other types if specified. + * Creates a plain object from a ChangeTabletTypeResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static - * @param {vtctldata.ApplyRoutingRulesRequest} message ApplyRoutingRulesRequest + * @param {vtctldata.ChangeTabletTypeResponse} message ChangeTabletTypeResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyRoutingRulesRequest.toObject = function toObject(message, options) { + ChangeTabletTypeResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.rebuild_cells = []; if (options.defaults) { - object.routing_rules = null; - object.skip_rebuild = false; - } - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) - object.routing_rules = $root.vschema.RoutingRules.toObject(message.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]; + object.before_tablet = null; + object.after_tablet = null; + object.was_dry_run = false; } + if (message.before_tablet != null && message.hasOwnProperty("before_tablet")) + object.before_tablet = $root.topodata.Tablet.toObject(message.before_tablet, options); + if (message.after_tablet != null && message.hasOwnProperty("after_tablet")) + object.after_tablet = $root.topodata.Tablet.toObject(message.after_tablet, options); + if (message.was_dry_run != null && message.hasOwnProperty("was_dry_run")) + object.was_dry_run = message.was_dry_run; return object; }; /** - * Converts this ApplyRoutingRulesRequest to JSON. + * Converts this ChangeTabletTypeResponse to JSON. * @function toJSON - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @instance * @returns {Object.} JSON object */ - ApplyRoutingRulesRequest.prototype.toJSON = function toJSON() { + ChangeTabletTypeResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyRoutingRulesRequest + * Gets the default type url for ChangeTabletTypeResponse * @function getTypeUrl - * @memberof vtctldata.ApplyRoutingRulesRequest + * @memberof vtctldata.ChangeTabletTypeResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ChangeTabletTypeResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyRoutingRulesRequest"; + return typeUrlPrefix + "/vtctldata.ChangeTabletTypeResponse"; }; - return ApplyRoutingRulesRequest; + return ChangeTabletTypeResponse; })(); - vtctldata.ApplyRoutingRulesResponse = (function() { + vtctldata.CreateKeyspaceRequest = (function() { /** - * Properties of an ApplyRoutingRulesResponse. + * Properties of a CreateKeyspaceRequest. * @memberof vtctldata - * @interface IApplyRoutingRulesResponse + * @interface ICreateKeyspaceRequest + * @property {string|null} [name] CreateKeyspaceRequest name + * @property {boolean|null} [force] CreateKeyspaceRequest force + * @property {boolean|null} [allow_empty_v_schema] CreateKeyspaceRequest allow_empty_v_schema + * @property {Array.|null} [served_froms] CreateKeyspaceRequest served_froms + * @property {topodata.KeyspaceType|null} [type] CreateKeyspaceRequest type + * @property {string|null} [base_keyspace] CreateKeyspaceRequest base_keyspace + * @property {vttime.ITime|null} [snapshot_time] CreateKeyspaceRequest snapshot_time + * @property {string|null} [durability_policy] CreateKeyspaceRequest durability_policy + * @property {string|null} [sidecar_db_name] CreateKeyspaceRequest sidecar_db_name */ /** - * Constructs a new ApplyRoutingRulesResponse. + * Constructs a new CreateKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents an ApplyRoutingRulesResponse. - * @implements IApplyRoutingRulesResponse + * @classdesc Represents a CreateKeyspaceRequest. + * @implements ICreateKeyspaceRequest * @constructor - * @param {vtctldata.IApplyRoutingRulesResponse=} [properties] Properties to set + * @param {vtctldata.ICreateKeyspaceRequest=} [properties] Properties to set */ - function ApplyRoutingRulesResponse(properties) { + function CreateKeyspaceRequest(properties) { + this.served_froms = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -103397,63 +109626,192 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new ApplyRoutingRulesResponse instance using the specified properties. + * CreateKeyspaceRequest name. + * @member {string} name + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.name = ""; + + /** + * CreateKeyspaceRequest force. + * @member {boolean} force + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.force = false; + + /** + * CreateKeyspaceRequest allow_empty_v_schema. + * @member {boolean} allow_empty_v_schema + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.allow_empty_v_schema = false; + + /** + * CreateKeyspaceRequest served_froms. + * @member {Array.} served_froms + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.served_froms = $util.emptyArray; + + /** + * CreateKeyspaceRequest type. + * @member {topodata.KeyspaceType} type + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.type = 0; + + /** + * CreateKeyspaceRequest base_keyspace. + * @member {string} base_keyspace + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.base_keyspace = ""; + + /** + * CreateKeyspaceRequest snapshot_time. + * @member {vttime.ITime|null|undefined} snapshot_time + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.snapshot_time = null; + + /** + * CreateKeyspaceRequest durability_policy. + * @member {string} durability_policy + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.durability_policy = ""; + + /** + * CreateKeyspaceRequest sidecar_db_name. + * @member {string} sidecar_db_name + * @memberof vtctldata.CreateKeyspaceRequest + * @instance + */ + CreateKeyspaceRequest.prototype.sidecar_db_name = ""; + + /** + * Creates a new CreateKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static - * @param {vtctldata.IApplyRoutingRulesResponse=} [properties] Properties to set - * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse instance + * @param {vtctldata.ICreateKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest instance */ - ApplyRoutingRulesResponse.create = function create(properties) { - return new ApplyRoutingRulesResponse(properties); + CreateKeyspaceRequest.create = function create(properties) { + return new CreateKeyspaceRequest(properties); }; /** - * Encodes the specified ApplyRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * Encodes the specified CreateKeyspaceRequest message. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static - * @param {vtctldata.IApplyRoutingRulesResponse} message ApplyRoutingRulesResponse message or plain object to encode + * @param {vtctldata.ICreateKeyspaceRequest} message CreateKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyRoutingRulesResponse.encode = function encode(message, writer) { + CreateKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.force); + if (message.allow_empty_v_schema != null && Object.hasOwnProperty.call(message, "allow_empty_v_schema")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_empty_v_schema); + if (message.served_froms != null && message.served_froms.length) + for (let i = 0; i < message.served_froms.length; ++i) + $root.topodata.Keyspace.ServedFrom.encode(message.served_froms[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.type != null && Object.hasOwnProperty.call(message, "type")) + writer.uint32(/* id 7, wireType 0 =*/56).int32(message.type); + if (message.base_keyspace != null && Object.hasOwnProperty.call(message, "base_keyspace")) + writer.uint32(/* id 8, wireType 2 =*/66).string(message.base_keyspace); + if (message.snapshot_time != null && Object.hasOwnProperty.call(message, "snapshot_time")) + $root.vttime.Time.encode(message.snapshot_time, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); + if (message.durability_policy != null && Object.hasOwnProperty.call(message, "durability_policy")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.durability_policy); + if (message.sidecar_db_name != null && Object.hasOwnProperty.call(message, "sidecar_db_name")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.sidecar_db_name); return writer; }; /** - * Encodes the specified ApplyRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyRoutingRulesResponse.verify|verify} messages. + * Encodes the specified CreateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static - * @param {vtctldata.IApplyRoutingRulesResponse} message ApplyRoutingRulesResponse message or plain object to encode + * @param {vtctldata.ICreateKeyspaceRequest} message CreateKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + CreateKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer. + * Decodes a CreateKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyRoutingRulesResponse.decode = function decode(reader, length) { + CreateKeyspaceRequest.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.ApplyRoutingRulesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.CreateKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + message.force = reader.bool(); + break; + } + case 3: { + message.allow_empty_v_schema = reader.bool(); + break; + } + case 6: { + if (!(message.served_froms && message.served_froms.length)) + message.served_froms = []; + message.served_froms.push($root.topodata.Keyspace.ServedFrom.decode(reader, reader.uint32())); + break; + } + case 7: { + message.type = reader.int32(); + break; + } + case 8: { + message.base_keyspace = reader.string(); + break; + } + case 9: { + message.snapshot_time = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 10: { + message.durability_policy = reader.string(); + break; + } + case 11: { + message.sidecar_db_name = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -103463,112 +109821,229 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + CreateKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyRoutingRulesResponse message. + * Verifies a CreateKeyspaceRequest message. * @function verify - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyRoutingRulesResponse.verify = function verify(message) { + CreateKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; + if (message.allow_empty_v_schema != null && message.hasOwnProperty("allow_empty_v_schema")) + if (typeof message.allow_empty_v_schema !== "boolean") + return "allow_empty_v_schema: boolean expected"; + if (message.served_froms != null && message.hasOwnProperty("served_froms")) { + if (!Array.isArray(message.served_froms)) + return "served_froms: array expected"; + for (let i = 0; i < message.served_froms.length; ++i) { + let error = $root.topodata.Keyspace.ServedFrom.verify(message.served_froms[i]); + if (error) + return "served_froms." + error; + } + } + if (message.type != null && message.hasOwnProperty("type")) + switch (message.type) { + default: + return "type: enum value expected"; + case 0: + case 1: + break; + } + if (message.base_keyspace != null && message.hasOwnProperty("base_keyspace")) + if (!$util.isString(message.base_keyspace)) + return "base_keyspace: string expected"; + if (message.snapshot_time != null && message.hasOwnProperty("snapshot_time")) { + let error = $root.vttime.Time.verify(message.snapshot_time); + if (error) + return "snapshot_time." + error; + } + if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) + if (!$util.isString(message.durability_policy)) + return "durability_policy: string expected"; + if (message.sidecar_db_name != null && message.hasOwnProperty("sidecar_db_name")) + if (!$util.isString(message.sidecar_db_name)) + return "sidecar_db_name: string expected"; return null; }; /** - * Creates an ApplyRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyRoutingRulesResponse} ApplyRoutingRulesResponse + * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest */ - ApplyRoutingRulesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyRoutingRulesResponse) + CreateKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.CreateKeyspaceRequest) return object; - return new $root.vtctldata.ApplyRoutingRulesResponse(); + let message = new $root.vtctldata.CreateKeyspaceRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.force != null) + message.force = Boolean(object.force); + if (object.allow_empty_v_schema != null) + message.allow_empty_v_schema = Boolean(object.allow_empty_v_schema); + if (object.served_froms) { + if (!Array.isArray(object.served_froms)) + throw TypeError(".vtctldata.CreateKeyspaceRequest.served_froms: array expected"); + message.served_froms = []; + for (let i = 0; i < object.served_froms.length; ++i) { + if (typeof object.served_froms[i] !== "object") + throw TypeError(".vtctldata.CreateKeyspaceRequest.served_froms: object expected"); + message.served_froms[i] = $root.topodata.Keyspace.ServedFrom.fromObject(object.served_froms[i]); + } + } + switch (object.type) { + default: + if (typeof object.type === "number") { + message.type = object.type; + break; + } + break; + case "NORMAL": + case 0: + message.type = 0; + break; + case "SNAPSHOT": + case 1: + message.type = 1; + break; + } + if (object.base_keyspace != null) + message.base_keyspace = String(object.base_keyspace); + if (object.snapshot_time != null) { + if (typeof object.snapshot_time !== "object") + throw TypeError(".vtctldata.CreateKeyspaceRequest.snapshot_time: object expected"); + message.snapshot_time = $root.vttime.Time.fromObject(object.snapshot_time); + } + if (object.durability_policy != null) + message.durability_policy = String(object.durability_policy); + if (object.sidecar_db_name != null) + message.sidecar_db_name = String(object.sidecar_db_name); + return message; }; /** - * Creates a plain object from an ApplyRoutingRulesResponse message. Also converts values to other types if specified. + * Creates a plain object from a CreateKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static - * @param {vtctldata.ApplyRoutingRulesResponse} message ApplyRoutingRulesResponse + * @param {vtctldata.CreateKeyspaceRequest} message CreateKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyRoutingRulesResponse.toObject = function toObject() { - return {}; + CreateKeyspaceRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.served_froms = []; + if (options.defaults) { + object.name = ""; + object.force = false; + object.allow_empty_v_schema = false; + object.type = options.enums === String ? "NORMAL" : 0; + object.base_keyspace = ""; + object.snapshot_time = null; + object.durability_policy = ""; + object.sidecar_db_name = ""; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; + if (message.allow_empty_v_schema != null && message.hasOwnProperty("allow_empty_v_schema")) + object.allow_empty_v_schema = message.allow_empty_v_schema; + if (message.served_froms && message.served_froms.length) { + object.served_froms = []; + for (let j = 0; j < message.served_froms.length; ++j) + object.served_froms[j] = $root.topodata.Keyspace.ServedFrom.toObject(message.served_froms[j], options); + } + if (message.type != null && message.hasOwnProperty("type")) + object.type = options.enums === String ? $root.topodata.KeyspaceType[message.type] === undefined ? message.type : $root.topodata.KeyspaceType[message.type] : message.type; + if (message.base_keyspace != null && message.hasOwnProperty("base_keyspace")) + object.base_keyspace = message.base_keyspace; + if (message.snapshot_time != null && message.hasOwnProperty("snapshot_time")) + object.snapshot_time = $root.vttime.Time.toObject(message.snapshot_time, options); + if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) + object.durability_policy = message.durability_policy; + if (message.sidecar_db_name != null && message.hasOwnProperty("sidecar_db_name")) + object.sidecar_db_name = message.sidecar_db_name; + return object; }; /** - * Converts this ApplyRoutingRulesResponse to JSON. + * Converts this CreateKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @instance * @returns {Object.} JSON object */ - ApplyRoutingRulesResponse.prototype.toJSON = function toJSON() { + CreateKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyRoutingRulesResponse + * Gets the default type url for CreateKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.ApplyRoutingRulesResponse + * @memberof vtctldata.CreateKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyRoutingRulesResponse"; + return typeUrlPrefix + "/vtctldata.CreateKeyspaceRequest"; }; - return ApplyRoutingRulesResponse; + return CreateKeyspaceRequest; })(); - vtctldata.ApplyShardRoutingRulesRequest = (function() { + vtctldata.CreateKeyspaceResponse = (function() { /** - * Properties of an ApplyShardRoutingRulesRequest. + * Properties of a CreateKeyspaceResponse. * @memberof vtctldata - * @interface IApplyShardRoutingRulesRequest - * @property {vschema.IShardRoutingRules|null} [shard_routing_rules] ApplyShardRoutingRulesRequest shard_routing_rules - * @property {boolean|null} [skip_rebuild] ApplyShardRoutingRulesRequest skip_rebuild - * @property {Array.|null} [rebuild_cells] ApplyShardRoutingRulesRequest rebuild_cells + * @interface ICreateKeyspaceResponse + * @property {vtctldata.IKeyspace|null} [keyspace] CreateKeyspaceResponse keyspace */ /** - * Constructs a new ApplyShardRoutingRulesRequest. + * Constructs a new CreateKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents an ApplyShardRoutingRulesRequest. - * @implements IApplyShardRoutingRulesRequest + * @classdesc Represents a CreateKeyspaceResponse. + * @implements ICreateKeyspaceResponse * @constructor - * @param {vtctldata.IApplyShardRoutingRulesRequest=} [properties] Properties to set + * @param {vtctldata.ICreateKeyspaceResponse=} [properties] Properties to set */ - function ApplyShardRoutingRulesRequest(properties) { - this.rebuild_cells = []; + function CreateKeyspaceResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -103576,106 +110051,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplyShardRoutingRulesRequest shard_routing_rules. - * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules - * @memberof vtctldata.ApplyShardRoutingRulesRequest - * @instance - */ - ApplyShardRoutingRulesRequest.prototype.shard_routing_rules = null; - - /** - * ApplyShardRoutingRulesRequest skip_rebuild. - * @member {boolean} skip_rebuild - * @memberof vtctldata.ApplyShardRoutingRulesRequest - * @instance - */ - ApplyShardRoutingRulesRequest.prototype.skip_rebuild = false; - - /** - * ApplyShardRoutingRulesRequest rebuild_cells. - * @member {Array.} rebuild_cells - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * CreateKeyspaceResponse keyspace. + * @member {vtctldata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.CreateKeyspaceResponse * @instance */ - ApplyShardRoutingRulesRequest.prototype.rebuild_cells = $util.emptyArray; + CreateKeyspaceResponse.prototype.keyspace = null; /** - * Creates a new ApplyShardRoutingRulesRequest instance using the specified properties. + * Creates a new CreateKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static - * @param {vtctldata.IApplyShardRoutingRulesRequest=} [properties] Properties to set - * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest instance + * @param {vtctldata.ICreateKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse instance */ - ApplyShardRoutingRulesRequest.create = function create(properties) { - return new ApplyShardRoutingRulesRequest(properties); + CreateKeyspaceResponse.create = function create(properties) { + return new CreateKeyspaceResponse(properties); }; /** - * Encodes the specified ApplyShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * Encodes the specified CreateKeyspaceResponse message. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static - * @param {vtctldata.IApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest message or plain object to encode + * @param {vtctldata.ICreateKeyspaceResponse} message CreateKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyShardRoutingRulesRequest.encode = function encode(message, writer) { + CreateKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard_routing_rules != null && Object.hasOwnProperty.call(message, "shard_routing_rules")) - $root.vschema.ShardRoutingRules.encode(message.shard_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]); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ApplyShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesRequest.verify|verify} messages. + * Encodes the specified CreateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static - * @param {vtctldata.IApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest message or plain object to encode + * @param {vtctldata.ICreateKeyspaceResponse} message CreateKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyShardRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + CreateKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer. + * Decodes a CreateKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyShardRoutingRulesRequest.decode = function decode(reader, length) { + CreateKeyspaceResponse.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.ApplyShardRoutingRulesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.CreateKeyspaceResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard_routing_rules = $root.vschema.ShardRoutingRules.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()); + message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); break; } default: @@ -103687,156 +110131,130 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyShardRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + CreateKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyShardRoutingRulesRequest message. + * Verifies a CreateKeyspaceResponse message. * @function verify - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyShardRoutingRulesRequest.verify = function verify(message) { + CreateKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { - let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.vtctldata.Keyspace.verify(message.keyspace); if (error) - return "shard_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 "keyspace." + error; } return null; }; /** - * Creates an ApplyShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyShardRoutingRulesRequest} ApplyShardRoutingRulesRequest + * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse */ - ApplyShardRoutingRulesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyShardRoutingRulesRequest) + CreateKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.CreateKeyspaceResponse) return object; - let message = new $root.vtctldata.ApplyShardRoutingRulesRequest(); - if (object.shard_routing_rules != null) { - if (typeof object.shard_routing_rules !== "object") - throw TypeError(".vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules: object expected"); - message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_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.ApplyShardRoutingRulesRequest.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]); + let message = new $root.vtctldata.CreateKeyspaceResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.CreateKeyspaceResponse.keyspace: object expected"); + message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); } return message; }; /** - * Creates a plain object from an ApplyShardRoutingRulesRequest message. Also converts values to other types if specified. + * Creates a plain object from a CreateKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static - * @param {vtctldata.ApplyShardRoutingRulesRequest} message ApplyShardRoutingRulesRequest + * @param {vtctldata.CreateKeyspaceResponse} message CreateKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyShardRoutingRulesRequest.toObject = function toObject(message, options) { + CreateKeyspaceResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.rebuild_cells = []; - if (options.defaults) { - object.shard_routing_rules = null; - object.skip_rebuild = false; - } - 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.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]; - } + if (options.defaults) + object.keyspace = null; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); return object; }; /** - * Converts this ApplyShardRoutingRulesRequest to JSON. + * Converts this CreateKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @instance * @returns {Object.} JSON object */ - ApplyShardRoutingRulesRequest.prototype.toJSON = function toJSON() { + CreateKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyShardRoutingRulesRequest + * Gets the default type url for CreateKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.ApplyShardRoutingRulesRequest + * @memberof vtctldata.CreateKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyShardRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyShardRoutingRulesRequest"; + return typeUrlPrefix + "/vtctldata.CreateKeyspaceResponse"; }; - return ApplyShardRoutingRulesRequest; + return CreateKeyspaceResponse; })(); - vtctldata.ApplyShardRoutingRulesResponse = (function() { + vtctldata.CreateShardRequest = (function() { /** - * Properties of an ApplyShardRoutingRulesResponse. + * Properties of a CreateShardRequest. * @memberof vtctldata - * @interface IApplyShardRoutingRulesResponse + * @interface ICreateShardRequest + * @property {string|null} [keyspace] CreateShardRequest keyspace + * @property {string|null} [shard_name] CreateShardRequest shard_name + * @property {boolean|null} [force] CreateShardRequest force + * @property {boolean|null} [include_parent] CreateShardRequest include_parent */ /** - * Constructs a new ApplyShardRoutingRulesResponse. + * Constructs a new CreateShardRequest. * @memberof vtctldata - * @classdesc Represents an ApplyShardRoutingRulesResponse. - * @implements IApplyShardRoutingRulesResponse + * @classdesc Represents a CreateShardRequest. + * @implements ICreateShardRequest * @constructor - * @param {vtctldata.IApplyShardRoutingRulesResponse=} [properties] Properties to set + * @param {vtctldata.ICreateShardRequest=} [properties] Properties to set */ - function ApplyShardRoutingRulesResponse(properties) { + function CreateShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -103844,63 +110262,119 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new ApplyShardRoutingRulesResponse instance using the specified properties. + * CreateShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.CreateShardRequest + * @instance + */ + CreateShardRequest.prototype.keyspace = ""; + + /** + * CreateShardRequest shard_name. + * @member {string} shard_name + * @memberof vtctldata.CreateShardRequest + * @instance + */ + CreateShardRequest.prototype.shard_name = ""; + + /** + * CreateShardRequest force. + * @member {boolean} force + * @memberof vtctldata.CreateShardRequest + * @instance + */ + CreateShardRequest.prototype.force = false; + + /** + * CreateShardRequest include_parent. + * @member {boolean} include_parent + * @memberof vtctldata.CreateShardRequest + * @instance + */ + CreateShardRequest.prototype.include_parent = false; + + /** + * Creates a new CreateShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static - * @param {vtctldata.IApplyShardRoutingRulesResponse=} [properties] Properties to set - * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse instance + * @param {vtctldata.ICreateShardRequest=} [properties] Properties to set + * @returns {vtctldata.CreateShardRequest} CreateShardRequest instance */ - ApplyShardRoutingRulesResponse.create = function create(properties) { - return new ApplyShardRoutingRulesResponse(properties); + CreateShardRequest.create = function create(properties) { + return new CreateShardRequest(properties); }; /** - * Encodes the specified ApplyShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * Encodes the specified CreateShardRequest message. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static - * @param {vtctldata.IApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse message or plain object to encode + * @param {vtctldata.ICreateShardRequest} message CreateShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyShardRoutingRulesResponse.encode = function encode(message, writer) { + CreateShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); + if (message.include_parent != null && Object.hasOwnProperty.call(message, "include_parent")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_parent); return writer; }; /** - * Encodes the specified ApplyShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyShardRoutingRulesResponse.verify|verify} messages. + * Encodes the specified CreateShardRequest message, length delimited. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static - * @param {vtctldata.IApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse message or plain object to encode + * @param {vtctldata.ICreateShardRequest} message CreateShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyShardRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + CreateShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer. + * Decodes a CreateShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + * @returns {vtctldata.CreateShardRequest} CreateShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyShardRoutingRulesResponse.decode = function decode(reader, length) { + CreateShardRequest.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.ApplyShardRoutingRulesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.CreateShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard_name = reader.string(); + break; + } + case 3: { + message.force = reader.bool(); + break; + } + case 4: { + message.include_parent = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -103910,118 +110384,149 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a CreateShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + * @returns {vtctldata.CreateShardRequest} CreateShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyShardRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + CreateShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyShardRoutingRulesResponse message. + * Verifies a CreateShardRequest message. * @function verify - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyShardRoutingRulesResponse.verify = function verify(message) { + CreateShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + if (!$util.isString(message.shard_name)) + return "shard_name: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; + if (message.include_parent != null && message.hasOwnProperty("include_parent")) + if (typeof message.include_parent !== "boolean") + return "include_parent: boolean expected"; return null; }; /** - * Creates an ApplyShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a CreateShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyShardRoutingRulesResponse} ApplyShardRoutingRulesResponse + * @returns {vtctldata.CreateShardRequest} CreateShardRequest */ - ApplyShardRoutingRulesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyShardRoutingRulesResponse) + CreateShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.CreateShardRequest) return object; - return new $root.vtctldata.ApplyShardRoutingRulesResponse(); + let message = new $root.vtctldata.CreateShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard_name != null) + message.shard_name = String(object.shard_name); + if (object.force != null) + message.force = Boolean(object.force); + if (object.include_parent != null) + message.include_parent = Boolean(object.include_parent); + return message; }; /** - * Creates a plain object from an ApplyShardRoutingRulesResponse message. Also converts values to other types if specified. + * Creates a plain object from a CreateShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static - * @param {vtctldata.ApplyShardRoutingRulesResponse} message ApplyShardRoutingRulesResponse + * @param {vtctldata.CreateShardRequest} message CreateShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyShardRoutingRulesResponse.toObject = function toObject() { - return {}; + CreateShardRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.keyspace = ""; + object.shard_name = ""; + object.force = false; + object.include_parent = false; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + object.shard_name = message.shard_name; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; + if (message.include_parent != null && message.hasOwnProperty("include_parent")) + object.include_parent = message.include_parent; + return object; }; /** - * Converts this ApplyShardRoutingRulesResponse to JSON. + * Converts this CreateShardRequest to JSON. * @function toJSON - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @instance * @returns {Object.} JSON object */ - ApplyShardRoutingRulesResponse.prototype.toJSON = function toJSON() { + CreateShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyShardRoutingRulesResponse + * Gets the default type url for CreateShardRequest * @function getTypeUrl - * @memberof vtctldata.ApplyShardRoutingRulesResponse + * @memberof vtctldata.CreateShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyShardRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyShardRoutingRulesResponse"; + return typeUrlPrefix + "/vtctldata.CreateShardRequest"; }; - return ApplyShardRoutingRulesResponse; + return CreateShardRequest; })(); - vtctldata.ApplySchemaRequest = (function() { + vtctldata.CreateShardResponse = (function() { /** - * Properties of an ApplySchemaRequest. + * Properties of a CreateShardResponse. * @memberof vtctldata - * @interface IApplySchemaRequest - * @property {string|null} [keyspace] ApplySchemaRequest keyspace - * @property {Array.|null} [sql] ApplySchemaRequest sql - * @property {string|null} [ddl_strategy] ApplySchemaRequest ddl_strategy - * @property {Array.|null} [uuid_list] ApplySchemaRequest uuid_list - * @property {string|null} [migration_context] ApplySchemaRequest migration_context - * @property {vttime.IDuration|null} [wait_replicas_timeout] ApplySchemaRequest wait_replicas_timeout - * @property {boolean|null} [skip_preflight] ApplySchemaRequest skip_preflight - * @property {vtrpc.ICallerID|null} [caller_id] ApplySchemaRequest caller_id + * @interface ICreateShardResponse + * @property {vtctldata.IKeyspace|null} [keyspace] CreateShardResponse keyspace + * @property {vtctldata.IShard|null} [shard] CreateShardResponse shard + * @property {boolean|null} [shard_already_exists] CreateShardResponse shard_already_exists */ /** - * Constructs a new ApplySchemaRequest. + * Constructs a new CreateShardResponse. * @memberof vtctldata - * @classdesc Represents an ApplySchemaRequest. - * @implements IApplySchemaRequest + * @classdesc Represents a CreateShardResponse. + * @implements ICreateShardResponse * @constructor - * @param {vtctldata.IApplySchemaRequest=} [properties] Properties to set + * @param {vtctldata.ICreateShardResponse=} [properties] Properties to set */ - function ApplySchemaRequest(properties) { - this.sql = []; - this.uuid_list = []; + function CreateShardResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -104029,179 +110534,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplySchemaRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ApplySchemaRequest - * @instance - */ - ApplySchemaRequest.prototype.keyspace = ""; - - /** - * ApplySchemaRequest sql. - * @member {Array.} sql - * @memberof vtctldata.ApplySchemaRequest - * @instance - */ - ApplySchemaRequest.prototype.sql = $util.emptyArray; - - /** - * ApplySchemaRequest ddl_strategy. - * @member {string} ddl_strategy - * @memberof vtctldata.ApplySchemaRequest - * @instance - */ - ApplySchemaRequest.prototype.ddl_strategy = ""; - - /** - * ApplySchemaRequest uuid_list. - * @member {Array.} uuid_list - * @memberof vtctldata.ApplySchemaRequest - * @instance - */ - ApplySchemaRequest.prototype.uuid_list = $util.emptyArray; - - /** - * ApplySchemaRequest migration_context. - * @member {string} migration_context - * @memberof vtctldata.ApplySchemaRequest - * @instance - */ - ApplySchemaRequest.prototype.migration_context = ""; - - /** - * ApplySchemaRequest wait_replicas_timeout. - * @member {vttime.IDuration|null|undefined} wait_replicas_timeout - * @memberof vtctldata.ApplySchemaRequest + * CreateShardResponse keyspace. + * @member {vtctldata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.CreateShardResponse * @instance */ - ApplySchemaRequest.prototype.wait_replicas_timeout = null; + CreateShardResponse.prototype.keyspace = null; /** - * ApplySchemaRequest skip_preflight. - * @member {boolean} skip_preflight - * @memberof vtctldata.ApplySchemaRequest + * CreateShardResponse shard. + * @member {vtctldata.IShard|null|undefined} shard + * @memberof vtctldata.CreateShardResponse * @instance */ - ApplySchemaRequest.prototype.skip_preflight = false; + CreateShardResponse.prototype.shard = null; /** - * ApplySchemaRequest caller_id. - * @member {vtrpc.ICallerID|null|undefined} caller_id - * @memberof vtctldata.ApplySchemaRequest + * CreateShardResponse shard_already_exists. + * @member {boolean} shard_already_exists + * @memberof vtctldata.CreateShardResponse * @instance */ - ApplySchemaRequest.prototype.caller_id = null; + CreateShardResponse.prototype.shard_already_exists = false; /** - * Creates a new ApplySchemaRequest instance using the specified properties. + * Creates a new CreateShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static - * @param {vtctldata.IApplySchemaRequest=} [properties] Properties to set - * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest instance + * @param {vtctldata.ICreateShardResponse=} [properties] Properties to set + * @returns {vtctldata.CreateShardResponse} CreateShardResponse instance */ - ApplySchemaRequest.create = function create(properties) { - return new ApplySchemaRequest(properties); + CreateShardResponse.create = function create(properties) { + return new CreateShardResponse(properties); }; /** - * Encodes the specified ApplySchemaRequest message. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * Encodes the specified CreateShardResponse message. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static - * @param {vtctldata.IApplySchemaRequest} message ApplySchemaRequest message or plain object to encode + * @param {vtctldata.ICreateShardResponse} message CreateShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplySchemaRequest.encode = function encode(message, writer) { + CreateShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.sql != null && message.sql.length) - for (let i = 0; i < message.sql.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.sql[i]); - if (message.ddl_strategy != null && Object.hasOwnProperty.call(message, "ddl_strategy")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.ddl_strategy); - if (message.uuid_list != null && message.uuid_list.length) - for (let i = 0; i < message.uuid_list.length; ++i) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.uuid_list[i]); - if (message.migration_context != null && Object.hasOwnProperty.call(message, "migration_context")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.migration_context); - if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) - $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); - if (message.skip_preflight != null && Object.hasOwnProperty.call(message, "skip_preflight")) - writer.uint32(/* id 8, wireType 0 =*/64).bool(message.skip_preflight); - if (message.caller_id != null && Object.hasOwnProperty.call(message, "caller_id")) - $root.vtrpc.CallerID.encode(message.caller_id, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); + $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.vtctldata.Shard.encode(message.shard, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.shard_already_exists != null && Object.hasOwnProperty.call(message, "shard_already_exists")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.shard_already_exists); return writer; }; /** - * Encodes the specified ApplySchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaRequest.verify|verify} messages. + * Encodes the specified CreateShardResponse message, length delimited. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static - * @param {vtctldata.IApplySchemaRequest} message ApplySchemaRequest message or plain object to encode + * @param {vtctldata.ICreateShardResponse} message CreateShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplySchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + CreateShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplySchemaRequest message from the specified reader or buffer. + * Decodes a CreateShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + * @returns {vtctldata.CreateShardResponse} CreateShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplySchemaRequest.decode = function decode(reader, length) { + CreateShardResponse.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.ApplySchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.CreateShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 3: { - if (!(message.sql && message.sql.length)) - message.sql = []; - message.sql.push(reader.string()); - break; - } - case 4: { - message.ddl_strategy = reader.string(); - break; - } - case 5: { - if (!(message.uuid_list && message.uuid_list.length)) - message.uuid_list = []; - message.uuid_list.push(reader.string()); - break; - } - case 6: { - message.migration_context = reader.string(); - break; - } - case 7: { - message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); break; } - case 8: { - message.skip_preflight = reader.bool(); + case 2: { + message.shard = $root.vtctldata.Shard.decode(reader, reader.uint32()); break; } - case 9: { - message.caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + case 3: { + message.shard_already_exists = reader.bool(); break; } default: @@ -104213,216 +110642,150 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplySchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a CreateShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + * @returns {vtctldata.CreateShardResponse} CreateShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplySchemaRequest.decodeDelimited = function decodeDelimited(reader) { + CreateShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplySchemaRequest message. + * Verifies a CreateShardResponse message. * @function verify - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplySchemaRequest.verify = function verify(message) { + CreateShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.sql != null && message.hasOwnProperty("sql")) { - if (!Array.isArray(message.sql)) - return "sql: array expected"; - for (let i = 0; i < message.sql.length; ++i) - if (!$util.isString(message.sql[i])) - return "sql: string[] expected"; - } - if (message.ddl_strategy != null && message.hasOwnProperty("ddl_strategy")) - if (!$util.isString(message.ddl_strategy)) - return "ddl_strategy: string expected"; - if (message.uuid_list != null && message.hasOwnProperty("uuid_list")) { - if (!Array.isArray(message.uuid_list)) - return "uuid_list: array expected"; - for (let i = 0; i < message.uuid_list.length; ++i) - if (!$util.isString(message.uuid_list[i])) - return "uuid_list: string[] expected"; - } - if (message.migration_context != null && message.hasOwnProperty("migration_context")) - if (!$util.isString(message.migration_context)) - return "migration_context: string expected"; - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { - let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.vtctldata.Keyspace.verify(message.keyspace); if (error) - return "wait_replicas_timeout." + error; + return "keyspace." + error; } - if (message.skip_preflight != null && message.hasOwnProperty("skip_preflight")) - if (typeof message.skip_preflight !== "boolean") - return "skip_preflight: boolean expected"; - if (message.caller_id != null && message.hasOwnProperty("caller_id")) { - let error = $root.vtrpc.CallerID.verify(message.caller_id); + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.vtctldata.Shard.verify(message.shard); if (error) - return "caller_id." + error; + return "shard." + error; } + if (message.shard_already_exists != null && message.hasOwnProperty("shard_already_exists")) + if (typeof message.shard_already_exists !== "boolean") + return "shard_already_exists: boolean expected"; return null; }; /** - * Creates an ApplySchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a CreateShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplySchemaRequest} ApplySchemaRequest + * @returns {vtctldata.CreateShardResponse} CreateShardResponse */ - ApplySchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplySchemaRequest) + CreateShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.CreateShardResponse) return object; - let message = new $root.vtctldata.ApplySchemaRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.sql) { - if (!Array.isArray(object.sql)) - throw TypeError(".vtctldata.ApplySchemaRequest.sql: array expected"); - message.sql = []; - for (let i = 0; i < object.sql.length; ++i) - message.sql[i] = String(object.sql[i]); - } - if (object.ddl_strategy != null) - message.ddl_strategy = String(object.ddl_strategy); - if (object.uuid_list) { - if (!Array.isArray(object.uuid_list)) - throw TypeError(".vtctldata.ApplySchemaRequest.uuid_list: array expected"); - message.uuid_list = []; - for (let i = 0; i < object.uuid_list.length; ++i) - message.uuid_list[i] = String(object.uuid_list[i]); - } - if (object.migration_context != null) - message.migration_context = String(object.migration_context); - if (object.wait_replicas_timeout != null) { - if (typeof object.wait_replicas_timeout !== "object") - throw TypeError(".vtctldata.ApplySchemaRequest.wait_replicas_timeout: object expected"); - message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + let message = new $root.vtctldata.CreateShardResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.CreateShardResponse.keyspace: object expected"); + message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); } - if (object.skip_preflight != null) - message.skip_preflight = Boolean(object.skip_preflight); - if (object.caller_id != null) { - if (typeof object.caller_id !== "object") - throw TypeError(".vtctldata.ApplySchemaRequest.caller_id: object expected"); - message.caller_id = $root.vtrpc.CallerID.fromObject(object.caller_id); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.CreateShardResponse.shard: object expected"); + message.shard = $root.vtctldata.Shard.fromObject(object.shard); } + if (object.shard_already_exists != null) + message.shard_already_exists = Boolean(object.shard_already_exists); return message; }; /** - * Creates a plain object from an ApplySchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a CreateShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static - * @param {vtctldata.ApplySchemaRequest} message ApplySchemaRequest + * @param {vtctldata.CreateShardResponse} message CreateShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplySchemaRequest.toObject = function toObject(message, options) { + CreateShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.sql = []; - object.uuid_list = []; - } if (options.defaults) { - object.keyspace = ""; - object.ddl_strategy = ""; - object.migration_context = ""; - object.wait_replicas_timeout = null; - object.skip_preflight = false; - object.caller_id = null; + object.keyspace = null; + object.shard = null; + object.shard_already_exists = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.sql && message.sql.length) { - object.sql = []; - for (let j = 0; j < message.sql.length; ++j) - object.sql[j] = message.sql[j]; - } - if (message.ddl_strategy != null && message.hasOwnProperty("ddl_strategy")) - object.ddl_strategy = message.ddl_strategy; - if (message.uuid_list && message.uuid_list.length) { - object.uuid_list = []; - for (let j = 0; j < message.uuid_list.length; ++j) - object.uuid_list[j] = message.uuid_list[j]; - } - if (message.migration_context != null && message.hasOwnProperty("migration_context")) - object.migration_context = message.migration_context; - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) - object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); - if (message.skip_preflight != null && message.hasOwnProperty("skip_preflight")) - object.skip_preflight = message.skip_preflight; - if (message.caller_id != null && message.hasOwnProperty("caller_id")) - object.caller_id = $root.vtrpc.CallerID.toObject(message.caller_id, options); + object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.vtctldata.Shard.toObject(message.shard, options); + if (message.shard_already_exists != null && message.hasOwnProperty("shard_already_exists")) + object.shard_already_exists = message.shard_already_exists; return object; }; /** - * Converts this ApplySchemaRequest to JSON. + * Converts this CreateShardResponse to JSON. * @function toJSON - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @instance * @returns {Object.} JSON object */ - ApplySchemaRequest.prototype.toJSON = function toJSON() { + CreateShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplySchemaRequest + * Gets the default type url for CreateShardResponse * @function getTypeUrl - * @memberof vtctldata.ApplySchemaRequest + * @memberof vtctldata.CreateShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplySchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + CreateShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplySchemaRequest"; + return typeUrlPrefix + "/vtctldata.CreateShardResponse"; }; - return ApplySchemaRequest; + return CreateShardResponse; })(); - vtctldata.ApplySchemaResponse = (function() { + vtctldata.DeleteCellInfoRequest = (function() { /** - * Properties of an ApplySchemaResponse. + * Properties of a DeleteCellInfoRequest. * @memberof vtctldata - * @interface IApplySchemaResponse - * @property {Array.|null} [uuid_list] ApplySchemaResponse uuid_list + * @interface IDeleteCellInfoRequest + * @property {string|null} [name] DeleteCellInfoRequest name + * @property {boolean|null} [force] DeleteCellInfoRequest force */ /** - * Constructs a new ApplySchemaResponse. + * Constructs a new DeleteCellInfoRequest. * @memberof vtctldata - * @classdesc Represents an ApplySchemaResponse. - * @implements IApplySchemaResponse + * @classdesc Represents a DeleteCellInfoRequest. + * @implements IDeleteCellInfoRequest * @constructor - * @param {vtctldata.IApplySchemaResponse=} [properties] Properties to set + * @param {vtctldata.IDeleteCellInfoRequest=} [properties] Properties to set */ - function ApplySchemaResponse(properties) { - this.uuid_list = []; + function DeleteCellInfoRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -104430,78 +110793,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplySchemaResponse uuid_list. - * @member {Array.} uuid_list - * @memberof vtctldata.ApplySchemaResponse + * DeleteCellInfoRequest name. + * @member {string} name + * @memberof vtctldata.DeleteCellInfoRequest * @instance */ - ApplySchemaResponse.prototype.uuid_list = $util.emptyArray; + DeleteCellInfoRequest.prototype.name = ""; /** - * Creates a new ApplySchemaResponse instance using the specified properties. + * DeleteCellInfoRequest force. + * @member {boolean} force + * @memberof vtctldata.DeleteCellInfoRequest + * @instance + */ + DeleteCellInfoRequest.prototype.force = false; + + /** + * Creates a new DeleteCellInfoRequest instance using the specified properties. * @function create - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static - * @param {vtctldata.IApplySchemaResponse=} [properties] Properties to set - * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse instance + * @param {vtctldata.IDeleteCellInfoRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest instance */ - ApplySchemaResponse.create = function create(properties) { - return new ApplySchemaResponse(properties); + DeleteCellInfoRequest.create = function create(properties) { + return new DeleteCellInfoRequest(properties); }; /** - * Encodes the specified ApplySchemaResponse message. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * Encodes the specified DeleteCellInfoRequest message. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static - * @param {vtctldata.IApplySchemaResponse} message ApplySchemaResponse message or plain object to encode + * @param {vtctldata.IDeleteCellInfoRequest} message DeleteCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplySchemaResponse.encode = function encode(message, writer) { + DeleteCellInfoRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.uuid_list != null && message.uuid_list.length) - for (let i = 0; i < message.uuid_list.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.uuid_list[i]); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.force); return writer; }; /** - * Encodes the specified ApplySchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplySchemaResponse.verify|verify} messages. + * Encodes the specified DeleteCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static - * @param {vtctldata.IApplySchemaResponse} message ApplySchemaResponse message or plain object to encode + * @param {vtctldata.IDeleteCellInfoRequest} message DeleteCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplySchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplySchemaResponse message from the specified reader or buffer. + * Decodes a DeleteCellInfoRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplySchemaResponse.decode = function decode(reader, length) { + DeleteCellInfoRequest.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.ApplySchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteCellInfoRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.uuid_list && message.uuid_list.length)) - message.uuid_list = []; - message.uuid_list.push(reader.string()); + message.name = reader.string(); + break; + } + case 2: { + message.force = reader.bool(); break; } default: @@ -104513,140 +110887,130 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplySchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellInfoRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplySchemaResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplySchemaResponse message. + * Verifies a DeleteCellInfoRequest message. * @function verify - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplySchemaResponse.verify = function verify(message) { + DeleteCellInfoRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.uuid_list != null && message.hasOwnProperty("uuid_list")) { - if (!Array.isArray(message.uuid_list)) - return "uuid_list: array expected"; - for (let i = 0; i < message.uuid_list.length; ++i) - if (!$util.isString(message.uuid_list[i])) - return "uuid_list: string[] expected"; - } + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; return null; }; /** - * Creates an ApplySchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplySchemaResponse} ApplySchemaResponse + * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest */ - ApplySchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplySchemaResponse) - return object; - let message = new $root.vtctldata.ApplySchemaResponse(); - if (object.uuid_list) { - if (!Array.isArray(object.uuid_list)) - throw TypeError(".vtctldata.ApplySchemaResponse.uuid_list: array expected"); - message.uuid_list = []; - for (let i = 0; i < object.uuid_list.length; ++i) - message.uuid_list[i] = String(object.uuid_list[i]); - } + DeleteCellInfoRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteCellInfoRequest) + return object; + let message = new $root.vtctldata.DeleteCellInfoRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.force != null) + message.force = Boolean(object.force); return message; }; /** - * Creates a plain object from an ApplySchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteCellInfoRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static - * @param {vtctldata.ApplySchemaResponse} message ApplySchemaResponse + * @param {vtctldata.DeleteCellInfoRequest} message DeleteCellInfoRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplySchemaResponse.toObject = function toObject(message, options) { + DeleteCellInfoRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.uuid_list = []; - if (message.uuid_list && message.uuid_list.length) { - object.uuid_list = []; - for (let j = 0; j < message.uuid_list.length; ++j) - object.uuid_list[j] = message.uuid_list[j]; + if (options.defaults) { + object.name = ""; + object.force = false; } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; return object; }; /** - * Converts this ApplySchemaResponse to JSON. + * Converts this DeleteCellInfoRequest to JSON. * @function toJSON - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @instance * @returns {Object.} JSON object */ - ApplySchemaResponse.prototype.toJSON = function toJSON() { + DeleteCellInfoRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplySchemaResponse + * Gets the default type url for DeleteCellInfoRequest * @function getTypeUrl - * @memberof vtctldata.ApplySchemaResponse + * @memberof vtctldata.DeleteCellInfoRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplySchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplySchemaResponse"; + return typeUrlPrefix + "/vtctldata.DeleteCellInfoRequest"; }; - return ApplySchemaResponse; + return DeleteCellInfoRequest; })(); - vtctldata.ApplyVSchemaRequest = (function() { + vtctldata.DeleteCellInfoResponse = (function() { /** - * Properties of an ApplyVSchemaRequest. + * Properties of a DeleteCellInfoResponse. * @memberof vtctldata - * @interface IApplyVSchemaRequest - * @property {string|null} [keyspace] ApplyVSchemaRequest keyspace - * @property {boolean|null} [skip_rebuild] ApplyVSchemaRequest skip_rebuild - * @property {boolean|null} [dry_run] ApplyVSchemaRequest dry_run - * @property {Array.|null} [cells] ApplyVSchemaRequest cells - * @property {vschema.IKeyspace|null} [v_schema] ApplyVSchemaRequest v_schema - * @property {string|null} [sql] ApplyVSchemaRequest sql + * @interface IDeleteCellInfoResponse */ /** - * Constructs a new ApplyVSchemaRequest. + * Constructs a new DeleteCellInfoResponse. * @memberof vtctldata - * @classdesc Represents an ApplyVSchemaRequest. - * @implements IApplyVSchemaRequest + * @classdesc Represents a DeleteCellInfoResponse. + * @implements IDeleteCellInfoResponse * @constructor - * @param {vtctldata.IApplyVSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IDeleteCellInfoResponse=} [properties] Properties to set */ - function ApplyVSchemaRequest(properties) { - this.cells = []; + function DeleteCellInfoResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -104654,150 +111018,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplyVSchemaRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.keyspace = ""; - - /** - * ApplyVSchemaRequest skip_rebuild. - * @member {boolean} skip_rebuild - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.skip_rebuild = false; - - /** - * ApplyVSchemaRequest dry_run. - * @member {boolean} dry_run - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.dry_run = false; - - /** - * ApplyVSchemaRequest cells. - * @member {Array.} cells - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.cells = $util.emptyArray; - - /** - * ApplyVSchemaRequest v_schema. - * @member {vschema.IKeyspace|null|undefined} v_schema - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.v_schema = null; - - /** - * ApplyVSchemaRequest sql. - * @member {string} sql - * @memberof vtctldata.ApplyVSchemaRequest - * @instance - */ - ApplyVSchemaRequest.prototype.sql = ""; - - /** - * Creates a new ApplyVSchemaRequest instance using the specified properties. + * Creates a new DeleteCellInfoResponse instance using the specified properties. * @function create - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static - * @param {vtctldata.IApplyVSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest instance + * @param {vtctldata.IDeleteCellInfoResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse instance */ - ApplyVSchemaRequest.create = function create(properties) { - return new ApplyVSchemaRequest(properties); + DeleteCellInfoResponse.create = function create(properties) { + return new DeleteCellInfoResponse(properties); }; /** - * Encodes the specified ApplyVSchemaRequest message. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * Encodes the specified DeleteCellInfoResponse message. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static - * @param {vtctldata.IApplyVSchemaRequest} message ApplyVSchemaRequest message or plain object to encode + * @param {vtctldata.IDeleteCellInfoResponse} message DeleteCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyVSchemaRequest.encode = function encode(message, writer) { + DeleteCellInfoResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.skip_rebuild != null && Object.hasOwnProperty.call(message, "skip_rebuild")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.skip_rebuild); - if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.dry_run); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.cells[i]); - if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) - $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.sql != null && Object.hasOwnProperty.call(message, "sql")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.sql); return writer; }; /** - * Encodes the specified ApplyVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaRequest.verify|verify} messages. + * Encodes the specified DeleteCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static - * @param {vtctldata.IApplyVSchemaRequest} message ApplyVSchemaRequest message or plain object to encode + * @param {vtctldata.IDeleteCellInfoResponse} message DeleteCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyVSchemaRequest message from the specified reader or buffer. + * Decodes a DeleteCellInfoResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyVSchemaRequest.decode = function decode(reader, length) { + DeleteCellInfoResponse.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.ApplyVSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteCellInfoResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.skip_rebuild = reader.bool(); - break; - } - case 3: { - message.dry_run = reader.bool(); - break; - } - case 4: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); - break; - } - case 5: { - message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); - break; - } - case 6: { - message.sql = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -104807,181 +111084,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellInfoResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyVSchemaRequest message. + * Verifies a DeleteCellInfoResponse message. * @function verify - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyVSchemaRequest.verify = function verify(message) { + DeleteCellInfoResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) - if (typeof message.skip_rebuild !== "boolean") - return "skip_rebuild: boolean expected"; - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - if (typeof message.dry_run !== "boolean") - return "dry_run: boolean expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } - if (message.v_schema != null && message.hasOwnProperty("v_schema")) { - let error = $root.vschema.Keyspace.verify(message.v_schema); - if (error) - return "v_schema." + error; - } - if (message.sql != null && message.hasOwnProperty("sql")) - if (!$util.isString(message.sql)) - return "sql: string expected"; return null; }; /** - * Creates an ApplyVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyVSchemaRequest} ApplyVSchemaRequest + * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse */ - ApplyVSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyVSchemaRequest) + DeleteCellInfoResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteCellInfoResponse) return object; - let message = new $root.vtctldata.ApplyVSchemaRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.skip_rebuild != null) - message.skip_rebuild = Boolean(object.skip_rebuild); - if (object.dry_run != null) - message.dry_run = Boolean(object.dry_run); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.ApplyVSchemaRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } - if (object.v_schema != null) { - if (typeof object.v_schema !== "object") - throw TypeError(".vtctldata.ApplyVSchemaRequest.v_schema: object expected"); - message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); - } - if (object.sql != null) - message.sql = String(object.sql); - return message; + return new $root.vtctldata.DeleteCellInfoResponse(); }; /** - * Creates a plain object from an ApplyVSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a DeleteCellInfoResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static - * @param {vtctldata.ApplyVSchemaRequest} message ApplyVSchemaRequest + * @param {vtctldata.DeleteCellInfoResponse} message DeleteCellInfoResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyVSchemaRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.cells = []; - if (options.defaults) { - object.keyspace = ""; - object.skip_rebuild = false; - object.dry_run = false; - object.v_schema = null; - object.sql = ""; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) - object.skip_rebuild = message.skip_rebuild; - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - object.dry_run = message.dry_run; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } - if (message.v_schema != null && message.hasOwnProperty("v_schema")) - object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); - if (message.sql != null && message.hasOwnProperty("sql")) - object.sql = message.sql; - return object; + DeleteCellInfoResponse.toObject = function toObject() { + return {}; }; /** - * Converts this ApplyVSchemaRequest to JSON. + * Converts this DeleteCellInfoResponse to JSON. * @function toJSON - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @instance * @returns {Object.} JSON object */ - ApplyVSchemaRequest.prototype.toJSON = function toJSON() { + DeleteCellInfoResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyVSchemaRequest + * Gets the default type url for DeleteCellInfoResponse * @function getTypeUrl - * @memberof vtctldata.ApplyVSchemaRequest + * @memberof vtctldata.DeleteCellInfoResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyVSchemaRequest"; + return typeUrlPrefix + "/vtctldata.DeleteCellInfoResponse"; }; - return ApplyVSchemaRequest; + return DeleteCellInfoResponse; })(); - vtctldata.ApplyVSchemaResponse = (function() { + vtctldata.DeleteCellsAliasRequest = (function() { /** - * Properties of an ApplyVSchemaResponse. + * Properties of a DeleteCellsAliasRequest. * @memberof vtctldata - * @interface IApplyVSchemaResponse - * @property {vschema.IKeyspace|null} [v_schema] ApplyVSchemaResponse v_schema + * @interface IDeleteCellsAliasRequest + * @property {string|null} [name] DeleteCellsAliasRequest name */ /** - * Constructs a new ApplyVSchemaResponse. + * Constructs a new DeleteCellsAliasRequest. * @memberof vtctldata - * @classdesc Represents an ApplyVSchemaResponse. - * @implements IApplyVSchemaResponse + * @classdesc Represents a DeleteCellsAliasRequest. + * @implements IDeleteCellsAliasRequest * @constructor - * @param {vtctldata.IApplyVSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IDeleteCellsAliasRequest=} [properties] Properties to set */ - function ApplyVSchemaResponse(properties) { + function DeleteCellsAliasRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -104989,75 +111194,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ApplyVSchemaResponse v_schema. - * @member {vschema.IKeyspace|null|undefined} v_schema - * @memberof vtctldata.ApplyVSchemaResponse + * DeleteCellsAliasRequest name. + * @member {string} name + * @memberof vtctldata.DeleteCellsAliasRequest * @instance */ - ApplyVSchemaResponse.prototype.v_schema = null; + DeleteCellsAliasRequest.prototype.name = ""; /** - * Creates a new ApplyVSchemaResponse instance using the specified properties. + * Creates a new DeleteCellsAliasRequest instance using the specified properties. * @function create - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static - * @param {vtctldata.IApplyVSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse instance + * @param {vtctldata.IDeleteCellsAliasRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest instance */ - ApplyVSchemaResponse.create = function create(properties) { - return new ApplyVSchemaResponse(properties); + DeleteCellsAliasRequest.create = function create(properties) { + return new DeleteCellsAliasRequest(properties); }; /** - * Encodes the specified ApplyVSchemaResponse message. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * Encodes the specified DeleteCellsAliasRequest message. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static - * @param {vtctldata.IApplyVSchemaResponse} message ApplyVSchemaResponse message or plain object to encode + * @param {vtctldata.IDeleteCellsAliasRequest} message DeleteCellsAliasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyVSchemaResponse.encode = function encode(message, writer) { + DeleteCellsAliasRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) - $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); return writer; }; /** - * Encodes the specified ApplyVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyVSchemaResponse.verify|verify} messages. + * Encodes the specified DeleteCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static - * @param {vtctldata.IApplyVSchemaResponse} message ApplyVSchemaResponse message or plain object to encode + * @param {vtctldata.IDeleteCellsAliasRequest} message DeleteCellsAliasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ApplyVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ApplyVSchemaResponse message from the specified reader or buffer. + * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyVSchemaResponse.decode = function decode(reader, length) { + DeleteCellsAliasRequest.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.ApplyVSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteCellsAliasRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); + message.name = reader.string(); break; } default: @@ -105069,265 +111274,185 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ApplyVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ApplyVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ApplyVSchemaResponse message. + * Verifies a DeleteCellsAliasRequest message. * @function verify - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ApplyVSchemaResponse.verify = function verify(message) { + DeleteCellsAliasRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.v_schema != null && message.hasOwnProperty("v_schema")) { - let error = $root.vschema.Keyspace.verify(message.v_schema); - if (error) - return "v_schema." + error; - } + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; return null; }; /** - * Creates an ApplyVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellsAliasRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ApplyVSchemaResponse} ApplyVSchemaResponse + * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest */ - ApplyVSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ApplyVSchemaResponse) + DeleteCellsAliasRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteCellsAliasRequest) return object; - let message = new $root.vtctldata.ApplyVSchemaResponse(); - if (object.v_schema != null) { - if (typeof object.v_schema !== "object") - throw TypeError(".vtctldata.ApplyVSchemaResponse.v_schema: object expected"); - message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); - } + let message = new $root.vtctldata.DeleteCellsAliasRequest(); + if (object.name != null) + message.name = String(object.name); return message; }; /** - * Creates a plain object from an ApplyVSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteCellsAliasRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static - * @param {vtctldata.ApplyVSchemaResponse} message ApplyVSchemaResponse + * @param {vtctldata.DeleteCellsAliasRequest} message DeleteCellsAliasRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ApplyVSchemaResponse.toObject = function toObject(message, options) { + DeleteCellsAliasRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.v_schema = null; - if (message.v_schema != null && message.hasOwnProperty("v_schema")) - object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); + object.name = ""; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; return object; }; /** - * Converts this ApplyVSchemaResponse to JSON. + * Converts this DeleteCellsAliasRequest to JSON. * @function toJSON - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @instance * @returns {Object.} JSON object */ - ApplyVSchemaResponse.prototype.toJSON = function toJSON() { + DeleteCellsAliasRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ApplyVSchemaResponse + * Gets the default type url for DeleteCellsAliasRequest * @function getTypeUrl - * @memberof vtctldata.ApplyVSchemaResponse + * @memberof vtctldata.DeleteCellsAliasRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ApplyVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ApplyVSchemaResponse"; + return typeUrlPrefix + "/vtctldata.DeleteCellsAliasRequest"; }; - return ApplyVSchemaResponse; + return DeleteCellsAliasRequest; })(); - vtctldata.BackupRequest = (function() { - - /** - * Properties of a BackupRequest. - * @memberof vtctldata - * @interface IBackupRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] BackupRequest tablet_alias - * @property {boolean|null} [allow_primary] BackupRequest allow_primary - * @property {number|Long|null} [concurrency] BackupRequest concurrency - * @property {string|null} [incremental_from_pos] BackupRequest incremental_from_pos - * @property {boolean|null} [upgrade_safe] BackupRequest upgrade_safe - */ + vtctldata.DeleteCellsAliasResponse = (function() { /** - * Constructs a new BackupRequest. + * Properties of a DeleteCellsAliasResponse. * @memberof vtctldata - * @classdesc Represents a BackupRequest. - * @implements IBackupRequest - * @constructor - * @param {vtctldata.IBackupRequest=} [properties] Properties to set - */ - function BackupRequest(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]]; - } - - /** - * BackupRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.BackupRequest - * @instance - */ - BackupRequest.prototype.tablet_alias = null; - - /** - * BackupRequest allow_primary. - * @member {boolean} allow_primary - * @memberof vtctldata.BackupRequest - * @instance - */ - BackupRequest.prototype.allow_primary = false; - - /** - * BackupRequest concurrency. - * @member {number|Long} concurrency - * @memberof vtctldata.BackupRequest - * @instance - */ - BackupRequest.prototype.concurrency = $util.Long ? $util.Long.fromBits(0,0,true) : 0; - - /** - * BackupRequest incremental_from_pos. - * @member {string} incremental_from_pos - * @memberof vtctldata.BackupRequest - * @instance + * @interface IDeleteCellsAliasResponse */ - BackupRequest.prototype.incremental_from_pos = ""; /** - * BackupRequest upgrade_safe. - * @member {boolean} upgrade_safe - * @memberof vtctldata.BackupRequest - * @instance + * Constructs a new DeleteCellsAliasResponse. + * @memberof vtctldata + * @classdesc Represents a DeleteCellsAliasResponse. + * @implements IDeleteCellsAliasResponse + * @constructor + * @param {vtctldata.IDeleteCellsAliasResponse=} [properties] Properties to set */ - BackupRequest.prototype.upgrade_safe = false; + function DeleteCellsAliasResponse(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 BackupRequest instance using the specified properties. + * Creates a new DeleteCellsAliasResponse instance using the specified properties. * @function create - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static - * @param {vtctldata.IBackupRequest=} [properties] Properties to set - * @returns {vtctldata.BackupRequest} BackupRequest instance + * @param {vtctldata.IDeleteCellsAliasResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse instance */ - BackupRequest.create = function create(properties) { - return new BackupRequest(properties); + DeleteCellsAliasResponse.create = function create(properties) { + return new DeleteCellsAliasResponse(properties); }; /** - * Encodes the specified BackupRequest message. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. + * Encodes the specified DeleteCellsAliasResponse message. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static - * @param {vtctldata.IBackupRequest} message BackupRequest message or plain object to encode + * @param {vtctldata.IDeleteCellsAliasResponse} message DeleteCellsAliasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BackupRequest.encode = function encode(message, writer) { + DeleteCellsAliasResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.allow_primary); - if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) - writer.uint32(/* id 3, wireType 0 =*/24).uint64(message.concurrency); - if (message.incremental_from_pos != null && Object.hasOwnProperty.call(message, "incremental_from_pos")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.incremental_from_pos); - if (message.upgrade_safe != null && Object.hasOwnProperty.call(message, "upgrade_safe")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.upgrade_safe); return writer; }; /** - * Encodes the specified BackupRequest message, length delimited. Does not implicitly {@link vtctldata.BackupRequest.verify|verify} messages. + * Encodes the specified DeleteCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static - * @param {vtctldata.IBackupRequest} message BackupRequest message or plain object to encode + * @param {vtctldata.IDeleteCellsAliasResponse} message DeleteCellsAliasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BackupRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BackupRequest message from the specified reader or buffer. + * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.BackupRequest} BackupRequest + * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupRequest.decode = function decode(reader, length) { + DeleteCellsAliasResponse.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.BackupRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteCellsAliasResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - message.allow_primary = reader.bool(); - break; - } - case 3: { - message.concurrency = reader.uint64(); - break; - } - case 4: { - message.incremental_from_pos = reader.string(); - break; - } - case 5: { - message.upgrade_safe = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -105337,177 +111462,111 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a BackupRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.BackupRequest} BackupRequest + * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BackupRequest message. + * Verifies a DeleteCellsAliasResponse message. * @function verify - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BackupRequest.verify = function verify(message) { + DeleteCellsAliasResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - if (typeof message.allow_primary !== "boolean") - return "allow_primary: boolean expected"; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (!$util.isInteger(message.concurrency) && !(message.concurrency && $util.isInteger(message.concurrency.low) && $util.isInteger(message.concurrency.high))) - return "concurrency: integer|Long expected"; - if (message.incremental_from_pos != null && message.hasOwnProperty("incremental_from_pos")) - if (!$util.isString(message.incremental_from_pos)) - return "incremental_from_pos: string expected"; - if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) - if (typeof message.upgrade_safe !== "boolean") - return "upgrade_safe: boolean expected"; return null; }; /** - * Creates a BackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteCellsAliasResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.BackupRequest} BackupRequest + * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse */ - BackupRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.BackupRequest) + DeleteCellsAliasResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteCellsAliasResponse) return object; - let message = new $root.vtctldata.BackupRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.BackupRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.allow_primary != null) - message.allow_primary = Boolean(object.allow_primary); - if (object.concurrency != null) - if ($util.Long) - (message.concurrency = $util.Long.fromValue(object.concurrency)).unsigned = true; - else if (typeof object.concurrency === "string") - message.concurrency = parseInt(object.concurrency, 10); - else if (typeof object.concurrency === "number") - message.concurrency = object.concurrency; - else if (typeof object.concurrency === "object") - message.concurrency = new $util.LongBits(object.concurrency.low >>> 0, object.concurrency.high >>> 0).toNumber(true); - if (object.incremental_from_pos != null) - message.incremental_from_pos = String(object.incremental_from_pos); - if (object.upgrade_safe != null) - message.upgrade_safe = Boolean(object.upgrade_safe); - return message; + return new $root.vtctldata.DeleteCellsAliasResponse(); }; /** - * Creates a plain object from a BackupRequest message. Also converts values to other types if specified. + * Creates a plain object from a DeleteCellsAliasResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static - * @param {vtctldata.BackupRequest} message BackupRequest + * @param {vtctldata.DeleteCellsAliasResponse} message DeleteCellsAliasResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BackupRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.tablet_alias = null; - object.allow_primary = false; - if ($util.Long) { - let long = new $util.Long(0, 0, true); - object.concurrency = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.concurrency = options.longs === String ? "0" : 0; - object.incremental_from_pos = ""; - object.upgrade_safe = false; - } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - object.allow_primary = message.allow_primary; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (typeof message.concurrency === "number") - object.concurrency = options.longs === String ? String(message.concurrency) : message.concurrency; - else - object.concurrency = options.longs === String ? $util.Long.prototype.toString.call(message.concurrency) : options.longs === Number ? new $util.LongBits(message.concurrency.low >>> 0, message.concurrency.high >>> 0).toNumber(true) : message.concurrency; - if (message.incremental_from_pos != null && message.hasOwnProperty("incremental_from_pos")) - object.incremental_from_pos = message.incremental_from_pos; - if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) - object.upgrade_safe = message.upgrade_safe; - return object; + DeleteCellsAliasResponse.toObject = function toObject() { + return {}; }; /** - * Converts this BackupRequest to JSON. + * Converts this DeleteCellsAliasResponse to JSON. * @function toJSON - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @instance * @returns {Object.} JSON object */ - BackupRequest.prototype.toJSON = function toJSON() { + DeleteCellsAliasResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BackupRequest + * Gets the default type url for DeleteCellsAliasResponse * @function getTypeUrl - * @memberof vtctldata.BackupRequest + * @memberof vtctldata.DeleteCellsAliasResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.BackupRequest"; + return typeUrlPrefix + "/vtctldata.DeleteCellsAliasResponse"; }; - return BackupRequest; + return DeleteCellsAliasResponse; })(); - vtctldata.BackupResponse = (function() { + vtctldata.DeleteKeyspaceRequest = (function() { /** - * Properties of a BackupResponse. + * Properties of a DeleteKeyspaceRequest. * @memberof vtctldata - * @interface IBackupResponse - * @property {topodata.ITabletAlias|null} [tablet_alias] BackupResponse tablet_alias - * @property {string|null} [keyspace] BackupResponse keyspace - * @property {string|null} [shard] BackupResponse shard - * @property {logutil.IEvent|null} [event] BackupResponse event + * @interface IDeleteKeyspaceRequest + * @property {string|null} [keyspace] DeleteKeyspaceRequest keyspace + * @property {boolean|null} [recursive] DeleteKeyspaceRequest recursive + * @property {boolean|null} [force] DeleteKeyspaceRequest force */ /** - * Constructs a new BackupResponse. + * Constructs a new DeleteKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a BackupResponse. - * @implements IBackupResponse + * @classdesc Represents a DeleteKeyspaceRequest. + * @implements IDeleteKeyspaceRequest * @constructor - * @param {vtctldata.IBackupResponse=} [properties] Properties to set + * @param {vtctldata.IDeleteKeyspaceRequest=} [properties] Properties to set */ - function BackupResponse(properties) { + function DeleteKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -105515,117 +111574,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * BackupResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.BackupResponse - * @instance - */ - BackupResponse.prototype.tablet_alias = null; - - /** - * BackupResponse keyspace. + * DeleteKeyspaceRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @instance */ - BackupResponse.prototype.keyspace = ""; + DeleteKeyspaceRequest.prototype.keyspace = ""; /** - * BackupResponse shard. - * @member {string} shard - * @memberof vtctldata.BackupResponse + * DeleteKeyspaceRequest recursive. + * @member {boolean} recursive + * @memberof vtctldata.DeleteKeyspaceRequest * @instance */ - BackupResponse.prototype.shard = ""; + DeleteKeyspaceRequest.prototype.recursive = false; /** - * BackupResponse event. - * @member {logutil.IEvent|null|undefined} event - * @memberof vtctldata.BackupResponse + * DeleteKeyspaceRequest force. + * @member {boolean} force + * @memberof vtctldata.DeleteKeyspaceRequest * @instance */ - BackupResponse.prototype.event = null; + DeleteKeyspaceRequest.prototype.force = false; /** - * Creates a new BackupResponse instance using the specified properties. + * Creates a new DeleteKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static - * @param {vtctldata.IBackupResponse=} [properties] Properties to set - * @returns {vtctldata.BackupResponse} BackupResponse instance + * @param {vtctldata.IDeleteKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest instance */ - BackupResponse.create = function create(properties) { - return new BackupResponse(properties); + DeleteKeyspaceRequest.create = function create(properties) { + return new DeleteKeyspaceRequest(properties); }; /** - * Encodes the specified BackupResponse message. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. + * Encodes the specified DeleteKeyspaceRequest message. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static - * @param {vtctldata.IBackupResponse} message BackupResponse message or plain object to encode + * @param {vtctldata.IDeleteKeyspaceRequest} message DeleteKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BackupResponse.encode = function encode(message, writer) { + DeleteKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); - if (message.event != null && Object.hasOwnProperty.call(message, "event")) - $root.logutil.Event.encode(message.event, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.recursive); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); return writer; }; /** - * Encodes the specified BackupResponse message, length delimited. Does not implicitly {@link vtctldata.BackupResponse.verify|verify} messages. + * Encodes the specified DeleteKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static - * @param {vtctldata.IBackupResponse} message BackupResponse message or plain object to encode + * @param {vtctldata.IDeleteKeyspaceRequest} message DeleteKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BackupResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BackupResponse message from the specified reader or buffer. + * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.BackupResponse} BackupResponse + * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupResponse.decode = function decode(reader, length) { + DeleteKeyspaceRequest.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.BackupResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); break; } case 2: { - message.keyspace = reader.string(); + message.recursive = reader.bool(); break; } case 3: { - message.shard = reader.string(); - break; - } - case 4: { - message.event = $root.logutil.Event.decode(reader, reader.uint32()); + message.force = reader.bool(); break; } default: @@ -105637,161 +111682,138 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a BackupResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.BackupResponse} BackupResponse + * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BackupResponse message. + * Verifies a DeleteKeyspaceRequest message. * @function verify - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BackupResponse.verify = function verify(message) { + DeleteKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.event != null && message.hasOwnProperty("event")) { - let error = $root.logutil.Event.verify(message.event); - if (error) - return "event." + error; - } + if (message.recursive != null && message.hasOwnProperty("recursive")) + if (typeof message.recursive !== "boolean") + return "recursive: boolean expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; return null; }; /** - * Creates a BackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.BackupResponse} BackupResponse + * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest */ - BackupResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.BackupResponse) + DeleteKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteKeyspaceRequest) return object; - let message = new $root.vtctldata.BackupResponse(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.BackupResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.DeleteKeyspaceRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.event != null) { - if (typeof object.event !== "object") - throw TypeError(".vtctldata.BackupResponse.event: object expected"); - message.event = $root.logutil.Event.fromObject(object.event); - } + if (object.recursive != null) + message.recursive = Boolean(object.recursive); + if (object.force != null) + message.force = Boolean(object.force); return message; }; /** - * Creates a plain object from a BackupResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static - * @param {vtctldata.BackupResponse} message BackupResponse + * @param {vtctldata.DeleteKeyspaceRequest} message DeleteKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BackupResponse.toObject = function toObject(message, options) { + DeleteKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.tablet_alias = null; object.keyspace = ""; - object.shard = ""; - object.event = null; + object.recursive = false; + object.force = false; } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.event != null && message.hasOwnProperty("event")) - object.event = $root.logutil.Event.toObject(message.event, options); + if (message.recursive != null && message.hasOwnProperty("recursive")) + object.recursive = message.recursive; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; return object; }; /** - * Converts this BackupResponse to JSON. + * Converts this DeleteKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @instance * @returns {Object.} JSON object */ - BackupResponse.prototype.toJSON = function toJSON() { + DeleteKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BackupResponse + * Gets the default type url for DeleteKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.BackupResponse + * @memberof vtctldata.DeleteKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.BackupResponse"; + return typeUrlPrefix + "/vtctldata.DeleteKeyspaceRequest"; }; - return BackupResponse; + return DeleteKeyspaceRequest; })(); - vtctldata.BackupShardRequest = (function() { + vtctldata.DeleteKeyspaceResponse = (function() { /** - * Properties of a BackupShardRequest. + * Properties of a DeleteKeyspaceResponse. * @memberof vtctldata - * @interface IBackupShardRequest - * @property {string|null} [keyspace] BackupShardRequest keyspace - * @property {string|null} [shard] BackupShardRequest shard - * @property {boolean|null} [allow_primary] BackupShardRequest allow_primary - * @property {number|Long|null} [concurrency] BackupShardRequest concurrency - * @property {boolean|null} [upgrade_safe] BackupShardRequest upgrade_safe + * @interface IDeleteKeyspaceResponse */ /** - * Constructs a new BackupShardRequest. + * Constructs a new DeleteKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a BackupShardRequest. - * @implements IBackupShardRequest + * @classdesc Represents a DeleteKeyspaceResponse. + * @implements IDeleteKeyspaceResponse * @constructor - * @param {vtctldata.IBackupShardRequest=} [properties] Properties to set + * @param {vtctldata.IDeleteKeyspaceResponse=} [properties] Properties to set */ - function BackupShardRequest(properties) { + function DeleteKeyspaceResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -105799,133 +111821,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * BackupShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.BackupShardRequest - * @instance - */ - BackupShardRequest.prototype.keyspace = ""; - - /** - * BackupShardRequest shard. - * @member {string} shard - * @memberof vtctldata.BackupShardRequest - * @instance - */ - BackupShardRequest.prototype.shard = ""; - - /** - * BackupShardRequest allow_primary. - * @member {boolean} allow_primary - * @memberof vtctldata.BackupShardRequest - * @instance - */ - BackupShardRequest.prototype.allow_primary = false; - - /** - * BackupShardRequest concurrency. - * @member {number|Long} concurrency - * @memberof vtctldata.BackupShardRequest - * @instance - */ - BackupShardRequest.prototype.concurrency = $util.Long ? $util.Long.fromBits(0,0,true) : 0; - - /** - * BackupShardRequest upgrade_safe. - * @member {boolean} upgrade_safe - * @memberof vtctldata.BackupShardRequest - * @instance - */ - BackupShardRequest.prototype.upgrade_safe = false; - - /** - * Creates a new BackupShardRequest instance using the specified properties. + * Creates a new DeleteKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static - * @param {vtctldata.IBackupShardRequest=} [properties] Properties to set - * @returns {vtctldata.BackupShardRequest} BackupShardRequest instance + * @param {vtctldata.IDeleteKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse instance */ - BackupShardRequest.create = function create(properties) { - return new BackupShardRequest(properties); + DeleteKeyspaceResponse.create = function create(properties) { + return new DeleteKeyspaceResponse(properties); }; /** - * Encodes the specified BackupShardRequest message. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. + * Encodes the specified DeleteKeyspaceResponse message. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static - * @param {vtctldata.IBackupShardRequest} message BackupShardRequest message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - BackupShardRequest.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_primary); - if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) - writer.uint32(/* id 4, wireType 0 =*/32).uint64(message.concurrency); - if (message.upgrade_safe != null && Object.hasOwnProperty.call(message, "upgrade_safe")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.upgrade_safe); + * @param {vtctldata.IDeleteKeyspaceResponse} message DeleteKeyspaceResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + DeleteKeyspaceResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); return writer; }; /** - * Encodes the specified BackupShardRequest message, length delimited. Does not implicitly {@link vtctldata.BackupShardRequest.verify|verify} messages. + * Encodes the specified DeleteKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static - * @param {vtctldata.IBackupShardRequest} message BackupShardRequest message or plain object to encode + * @param {vtctldata.IDeleteKeyspaceResponse} message DeleteKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - BackupShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a BackupShardRequest message from the specified reader or buffer. + * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.BackupShardRequest} BackupShardRequest + * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupShardRequest.decode = function decode(reader, length) { + DeleteKeyspaceResponse.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.BackupShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteKeyspaceResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.allow_primary = reader.bool(); - break; - } - case 4: { - message.concurrency = reader.uint64(); - break; - } - case 5: { - message.upgrade_safe = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -105935,171 +111887,113 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a BackupShardRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.BackupShardRequest} BackupShardRequest + * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - BackupShardRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a BackupShardRequest message. + * Verifies a DeleteKeyspaceResponse message. * @function verify - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - BackupShardRequest.verify = function verify(message) { + DeleteKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - if (typeof message.allow_primary !== "boolean") - return "allow_primary: boolean expected"; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (!$util.isInteger(message.concurrency) && !(message.concurrency && $util.isInteger(message.concurrency.low) && $util.isInteger(message.concurrency.high))) - return "concurrency: integer|Long expected"; - if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) - if (typeof message.upgrade_safe !== "boolean") - return "upgrade_safe: boolean expected"; return null; }; /** - * Creates a BackupShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.BackupShardRequest} BackupShardRequest + * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse */ - BackupShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.BackupShardRequest) + DeleteKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteKeyspaceResponse) return object; - let message = new $root.vtctldata.BackupShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.allow_primary != null) - message.allow_primary = Boolean(object.allow_primary); - if (object.concurrency != null) - if ($util.Long) - (message.concurrency = $util.Long.fromValue(object.concurrency)).unsigned = true; - else if (typeof object.concurrency === "string") - message.concurrency = parseInt(object.concurrency, 10); - else if (typeof object.concurrency === "number") - message.concurrency = object.concurrency; - else if (typeof object.concurrency === "object") - message.concurrency = new $util.LongBits(object.concurrency.low >>> 0, object.concurrency.high >>> 0).toNumber(true); - if (object.upgrade_safe != null) - message.upgrade_safe = Boolean(object.upgrade_safe); - return message; + return new $root.vtctldata.DeleteKeyspaceResponse(); }; /** - * Creates a plain object from a BackupShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a DeleteKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static - * @param {vtctldata.BackupShardRequest} message BackupShardRequest + * @param {vtctldata.DeleteKeyspaceResponse} message DeleteKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - BackupShardRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.allow_primary = false; - if ($util.Long) { - let long = new $util.Long(0, 0, true); - object.concurrency = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.concurrency = options.longs === String ? "0" : 0; - object.upgrade_safe = false; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - object.allow_primary = message.allow_primary; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (typeof message.concurrency === "number") - object.concurrency = options.longs === String ? String(message.concurrency) : message.concurrency; - else - object.concurrency = options.longs === String ? $util.Long.prototype.toString.call(message.concurrency) : options.longs === Number ? new $util.LongBits(message.concurrency.low >>> 0, message.concurrency.high >>> 0).toNumber(true) : message.concurrency; - if (message.upgrade_safe != null && message.hasOwnProperty("upgrade_safe")) - object.upgrade_safe = message.upgrade_safe; - return object; + DeleteKeyspaceResponse.toObject = function toObject() { + return {}; }; /** - * Converts this BackupShardRequest to JSON. + * Converts this DeleteKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @instance * @returns {Object.} JSON object */ - BackupShardRequest.prototype.toJSON = function toJSON() { + DeleteKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for BackupShardRequest + * Gets the default type url for DeleteKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.BackupShardRequest + * @memberof vtctldata.DeleteKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - BackupShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.BackupShardRequest"; + return typeUrlPrefix + "/vtctldata.DeleteKeyspaceResponse"; }; - return BackupShardRequest; + return DeleteKeyspaceResponse; })(); - vtctldata.ChangeTabletTypeRequest = (function() { + vtctldata.DeleteShardsRequest = (function() { /** - * Properties of a ChangeTabletTypeRequest. + * Properties of a DeleteShardsRequest. * @memberof vtctldata - * @interface IChangeTabletTypeRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] ChangeTabletTypeRequest tablet_alias - * @property {topodata.TabletType|null} [db_type] ChangeTabletTypeRequest db_type - * @property {boolean|null} [dry_run] ChangeTabletTypeRequest dry_run + * @interface IDeleteShardsRequest + * @property {Array.|null} [shards] DeleteShardsRequest shards + * @property {boolean|null} [recursive] DeleteShardsRequest recursive + * @property {boolean|null} [even_if_serving] DeleteShardsRequest even_if_serving + * @property {boolean|null} [force] DeleteShardsRequest force */ /** - * Constructs a new ChangeTabletTypeRequest. + * Constructs a new DeleteShardsRequest. * @memberof vtctldata - * @classdesc Represents a ChangeTabletTypeRequest. - * @implements IChangeTabletTypeRequest + * @classdesc Represents a DeleteShardsRequest. + * @implements IDeleteShardsRequest * @constructor - * @param {vtctldata.IChangeTabletTypeRequest=} [properties] Properties to set + * @param {vtctldata.IDeleteShardsRequest=} [properties] Properties to set */ - function ChangeTabletTypeRequest(properties) { + function DeleteShardsRequest(properties) { + this.shards = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -106107,103 +112001,120 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ChangeTabletTypeRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ChangeTabletTypeRequest + * DeleteShardsRequest shards. + * @member {Array.} shards + * @memberof vtctldata.DeleteShardsRequest * @instance */ - ChangeTabletTypeRequest.prototype.tablet_alias = null; + DeleteShardsRequest.prototype.shards = $util.emptyArray; /** - * ChangeTabletTypeRequest db_type. - * @member {topodata.TabletType} db_type - * @memberof vtctldata.ChangeTabletTypeRequest + * DeleteShardsRequest recursive. + * @member {boolean} recursive + * @memberof vtctldata.DeleteShardsRequest * @instance */ - ChangeTabletTypeRequest.prototype.db_type = 0; + DeleteShardsRequest.prototype.recursive = false; /** - * ChangeTabletTypeRequest dry_run. - * @member {boolean} dry_run - * @memberof vtctldata.ChangeTabletTypeRequest + * DeleteShardsRequest even_if_serving. + * @member {boolean} even_if_serving + * @memberof vtctldata.DeleteShardsRequest * @instance */ - ChangeTabletTypeRequest.prototype.dry_run = false; + DeleteShardsRequest.prototype.even_if_serving = false; /** - * Creates a new ChangeTabletTypeRequest instance using the specified properties. + * DeleteShardsRequest force. + * @member {boolean} force + * @memberof vtctldata.DeleteShardsRequest + * @instance + */ + DeleteShardsRequest.prototype.force = false; + + /** + * Creates a new DeleteShardsRequest instance using the specified properties. * @function create - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static - * @param {vtctldata.IChangeTabletTypeRequest=} [properties] Properties to set - * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest instance + * @param {vtctldata.IDeleteShardsRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest instance */ - ChangeTabletTypeRequest.create = function create(properties) { - return new ChangeTabletTypeRequest(properties); + DeleteShardsRequest.create = function create(properties) { + return new DeleteShardsRequest(properties); }; /** - * Encodes the specified ChangeTabletTypeRequest message. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. + * Encodes the specified DeleteShardsRequest message. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static - * @param {vtctldata.IChangeTabletTypeRequest} message ChangeTabletTypeRequest message or plain object to encode + * @param {vtctldata.IDeleteShardsRequest} message DeleteShardsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ChangeTabletTypeRequest.encode = function encode(message, writer) { + DeleteShardsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.db_type != null && Object.hasOwnProperty.call(message, "db_type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.db_type); - if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.dry_run); + if (message.shards != null && message.shards.length) + for (let i = 0; i < message.shards.length; ++i) + $root.vtctldata.Shard.encode(message.shards[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.recursive); + if (message.even_if_serving != null && Object.hasOwnProperty.call(message, "even_if_serving")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.even_if_serving); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.force); return writer; }; /** - * Encodes the specified ChangeTabletTypeRequest message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeRequest.verify|verify} messages. + * Encodes the specified DeleteShardsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static - * @param {vtctldata.IChangeTabletTypeRequest} message ChangeTabletTypeRequest message or plain object to encode + * @param {vtctldata.IDeleteShardsRequest} message DeleteShardsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ChangeTabletTypeRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteShardsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer. + * Decodes a DeleteShardsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest + * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ChangeTabletTypeRequest.decode = function decode(reader, length) { + DeleteShardsRequest.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.ChangeTabletTypeRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteShardsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + if (!(message.shards && message.shards.length)) + message.shards = []; + message.shards.push($root.vtctldata.Shard.decode(reader, reader.uint32())); break; } case 2: { - message.db_type = reader.int32(); + message.recursive = reader.bool(); break; } - case 3: { - message.dry_run = reader.bool(); + case 4: { + message.even_if_serving = reader.bool(); + break; + } + case 5: { + message.force = reader.bool(); break; } default: @@ -106215,316 +112126,228 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ChangeTabletTypeRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteShardsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest + * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ChangeTabletTypeRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteShardsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ChangeTabletTypeRequest message. + * Verifies a DeleteShardsRequest message. * @function verify - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ChangeTabletTypeRequest.verify = function verify(message) { + DeleteShardsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.db_type != null && message.hasOwnProperty("db_type")) - switch (message.db_type) { - default: - return "db_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; + if (message.shards != null && message.hasOwnProperty("shards")) { + if (!Array.isArray(message.shards)) + return "shards: array expected"; + for (let i = 0; i < message.shards.length; ++i) { + let error = $root.vtctldata.Shard.verify(message.shards[i]); + if (error) + return "shards." + error; } - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - if (typeof message.dry_run !== "boolean") - return "dry_run: boolean expected"; + } + if (message.recursive != null && message.hasOwnProperty("recursive")) + if (typeof message.recursive !== "boolean") + return "recursive: boolean expected"; + if (message.even_if_serving != null && message.hasOwnProperty("even_if_serving")) + if (typeof message.even_if_serving !== "boolean") + return "even_if_serving: boolean expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; return null; }; /** - * Creates a ChangeTabletTypeRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteShardsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ChangeTabletTypeRequest} ChangeTabletTypeRequest + * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest */ - ChangeTabletTypeRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ChangeTabletTypeRequest) + DeleteShardsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteShardsRequest) return object; - let message = new $root.vtctldata.ChangeTabletTypeRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ChangeTabletTypeRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - switch (object.db_type) { - default: - if (typeof object.db_type === "number") { - message.db_type = object.db_type; - break; + let message = new $root.vtctldata.DeleteShardsRequest(); + if (object.shards) { + if (!Array.isArray(object.shards)) + throw TypeError(".vtctldata.DeleteShardsRequest.shards: array expected"); + message.shards = []; + for (let i = 0; i < object.shards.length; ++i) { + if (typeof object.shards[i] !== "object") + throw TypeError(".vtctldata.DeleteShardsRequest.shards: object expected"); + message.shards[i] = $root.vtctldata.Shard.fromObject(object.shards[i]); } - break; - case "UNKNOWN": - case 0: - message.db_type = 0; - break; - case "PRIMARY": - case 1: - message.db_type = 1; - break; - case "MASTER": - case 1: - message.db_type = 1; - break; - case "REPLICA": - case 2: - message.db_type = 2; - break; - case "RDONLY": - case 3: - message.db_type = 3; - break; - case "BATCH": - case 3: - message.db_type = 3; - break; - case "SPARE": - case 4: - message.db_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.db_type = 5; - break; - case "BACKUP": - case 6: - message.db_type = 6; - break; - case "RESTORE": - case 7: - message.db_type = 7; - break; - case "DRAINED": - case 8: - message.db_type = 8; - break; } - if (object.dry_run != null) - message.dry_run = Boolean(object.dry_run); + if (object.recursive != null) + message.recursive = Boolean(object.recursive); + if (object.even_if_serving != null) + message.even_if_serving = Boolean(object.even_if_serving); + if (object.force != null) + message.force = Boolean(object.force); return message; }; /** - * Creates a plain object from a ChangeTabletTypeRequest message. Also converts values to other types if specified. + * Creates a plain object from a DeleteShardsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static - * @param {vtctldata.ChangeTabletTypeRequest} message ChangeTabletTypeRequest + * @param {vtctldata.DeleteShardsRequest} message DeleteShardsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ChangeTabletTypeRequest.toObject = function toObject(message, options) { + DeleteShardsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.shards = []; if (options.defaults) { - object.tablet_alias = null; - object.db_type = options.enums === String ? "UNKNOWN" : 0; - object.dry_run = false; + object.recursive = false; + object.even_if_serving = false; + object.force = false; } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.db_type != null && message.hasOwnProperty("db_type")) - object.db_type = options.enums === String ? $root.topodata.TabletType[message.db_type] === undefined ? message.db_type : $root.topodata.TabletType[message.db_type] : message.db_type; - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - object.dry_run = message.dry_run; + if (message.shards && message.shards.length) { + object.shards = []; + for (let j = 0; j < message.shards.length; ++j) + object.shards[j] = $root.vtctldata.Shard.toObject(message.shards[j], options); + } + if (message.recursive != null && message.hasOwnProperty("recursive")) + object.recursive = message.recursive; + if (message.even_if_serving != null && message.hasOwnProperty("even_if_serving")) + object.even_if_serving = message.even_if_serving; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; return object; }; /** - * Converts this ChangeTabletTypeRequest to JSON. + * Converts this DeleteShardsRequest to JSON. * @function toJSON - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @instance * @returns {Object.} JSON object */ - ChangeTabletTypeRequest.prototype.toJSON = function toJSON() { + DeleteShardsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ChangeTabletTypeRequest + * Gets the default type url for DeleteShardsRequest * @function getTypeUrl - * @memberof vtctldata.ChangeTabletTypeRequest + * @memberof vtctldata.DeleteShardsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ChangeTabletTypeRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteShardsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ChangeTabletTypeRequest"; + return typeUrlPrefix + "/vtctldata.DeleteShardsRequest"; }; - return ChangeTabletTypeRequest; + return DeleteShardsRequest; })(); - vtctldata.ChangeTabletTypeResponse = (function() { + vtctldata.DeleteShardsResponse = (function() { /** - * Properties of a ChangeTabletTypeResponse. + * Properties of a DeleteShardsResponse. * @memberof vtctldata - * @interface IChangeTabletTypeResponse - * @property {topodata.ITablet|null} [before_tablet] ChangeTabletTypeResponse before_tablet - * @property {topodata.ITablet|null} [after_tablet] ChangeTabletTypeResponse after_tablet - * @property {boolean|null} [was_dry_run] ChangeTabletTypeResponse was_dry_run + * @interface IDeleteShardsResponse */ /** - * Constructs a new ChangeTabletTypeResponse. + * Constructs a new DeleteShardsResponse. * @memberof vtctldata - * @classdesc Represents a ChangeTabletTypeResponse. - * @implements IChangeTabletTypeResponse + * @classdesc Represents a DeleteShardsResponse. + * @implements IDeleteShardsResponse * @constructor - * @param {vtctldata.IChangeTabletTypeResponse=} [properties] Properties to set - */ - function ChangeTabletTypeResponse(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]]; - } - - /** - * ChangeTabletTypeResponse before_tablet. - * @member {topodata.ITablet|null|undefined} before_tablet - * @memberof vtctldata.ChangeTabletTypeResponse - * @instance - */ - ChangeTabletTypeResponse.prototype.before_tablet = null; - - /** - * ChangeTabletTypeResponse after_tablet. - * @member {topodata.ITablet|null|undefined} after_tablet - * @memberof vtctldata.ChangeTabletTypeResponse - * @instance - */ - ChangeTabletTypeResponse.prototype.after_tablet = null; - - /** - * ChangeTabletTypeResponse was_dry_run. - * @member {boolean} was_dry_run - * @memberof vtctldata.ChangeTabletTypeResponse - * @instance + * @param {vtctldata.IDeleteShardsResponse=} [properties] Properties to set */ - ChangeTabletTypeResponse.prototype.was_dry_run = false; + function DeleteShardsResponse(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 ChangeTabletTypeResponse instance using the specified properties. + * Creates a new DeleteShardsResponse instance using the specified properties. * @function create - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static - * @param {vtctldata.IChangeTabletTypeResponse=} [properties] Properties to set - * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse instance + * @param {vtctldata.IDeleteShardsResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse instance */ - ChangeTabletTypeResponse.create = function create(properties) { - return new ChangeTabletTypeResponse(properties); + DeleteShardsResponse.create = function create(properties) { + return new DeleteShardsResponse(properties); }; /** - * Encodes the specified ChangeTabletTypeResponse message. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. + * Encodes the specified DeleteShardsResponse message. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static - * @param {vtctldata.IChangeTabletTypeResponse} message ChangeTabletTypeResponse message or plain object to encode + * @param {vtctldata.IDeleteShardsResponse} message DeleteShardsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ChangeTabletTypeResponse.encode = function encode(message, writer) { + DeleteShardsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.before_tablet != null && Object.hasOwnProperty.call(message, "before_tablet")) - $root.topodata.Tablet.encode(message.before_tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.after_tablet != null && Object.hasOwnProperty.call(message, "after_tablet")) - $root.topodata.Tablet.encode(message.after_tablet, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.was_dry_run != null && Object.hasOwnProperty.call(message, "was_dry_run")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.was_dry_run); return writer; }; /** - * Encodes the specified ChangeTabletTypeResponse message, length delimited. Does not implicitly {@link vtctldata.ChangeTabletTypeResponse.verify|verify} messages. + * Encodes the specified DeleteShardsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static - * @param {vtctldata.IChangeTabletTypeResponse} message ChangeTabletTypeResponse message or plain object to encode + * @param {vtctldata.IDeleteShardsResponse} message DeleteShardsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ChangeTabletTypeResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteShardsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer. + * Decodes a DeleteShardsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse + * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ChangeTabletTypeResponse.decode = function decode(reader, length) { + DeleteShardsResponse.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.ChangeTabletTypeResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteShardsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.before_tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); - break; - } - case 2: { - message.after_tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); - break; - } - case 3: { - message.was_dry_run = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -106534,158 +112357,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ChangeTabletTypeResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteShardsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse + * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ChangeTabletTypeResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteShardsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ChangeTabletTypeResponse message. + * Verifies a DeleteShardsResponse message. * @function verify - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ChangeTabletTypeResponse.verify = function verify(message) { + DeleteShardsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.before_tablet != null && message.hasOwnProperty("before_tablet")) { - let error = $root.topodata.Tablet.verify(message.before_tablet); - if (error) - return "before_tablet." + error; - } - if (message.after_tablet != null && message.hasOwnProperty("after_tablet")) { - let error = $root.topodata.Tablet.verify(message.after_tablet); - if (error) - return "after_tablet." + error; - } - if (message.was_dry_run != null && message.hasOwnProperty("was_dry_run")) - if (typeof message.was_dry_run !== "boolean") - return "was_dry_run: boolean expected"; return null; }; /** - * Creates a ChangeTabletTypeResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteShardsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ChangeTabletTypeResponse} ChangeTabletTypeResponse + * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse */ - ChangeTabletTypeResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ChangeTabletTypeResponse) + DeleteShardsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteShardsResponse) return object; - let message = new $root.vtctldata.ChangeTabletTypeResponse(); - if (object.before_tablet != null) { - if (typeof object.before_tablet !== "object") - throw TypeError(".vtctldata.ChangeTabletTypeResponse.before_tablet: object expected"); - message.before_tablet = $root.topodata.Tablet.fromObject(object.before_tablet); - } - if (object.after_tablet != null) { - if (typeof object.after_tablet !== "object") - throw TypeError(".vtctldata.ChangeTabletTypeResponse.after_tablet: object expected"); - message.after_tablet = $root.topodata.Tablet.fromObject(object.after_tablet); - } - if (object.was_dry_run != null) - message.was_dry_run = Boolean(object.was_dry_run); - return message; + return new $root.vtctldata.DeleteShardsResponse(); }; /** - * Creates a plain object from a ChangeTabletTypeResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteShardsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static - * @param {vtctldata.ChangeTabletTypeResponse} message ChangeTabletTypeResponse + * @param {vtctldata.DeleteShardsResponse} message DeleteShardsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ChangeTabletTypeResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.before_tablet = null; - object.after_tablet = null; - object.was_dry_run = false; - } - if (message.before_tablet != null && message.hasOwnProperty("before_tablet")) - object.before_tablet = $root.topodata.Tablet.toObject(message.before_tablet, options); - if (message.after_tablet != null && message.hasOwnProperty("after_tablet")) - object.after_tablet = $root.topodata.Tablet.toObject(message.after_tablet, options); - if (message.was_dry_run != null && message.hasOwnProperty("was_dry_run")) - object.was_dry_run = message.was_dry_run; - return object; + DeleteShardsResponse.toObject = function toObject() { + return {}; }; /** - * Converts this ChangeTabletTypeResponse to JSON. + * Converts this DeleteShardsResponse to JSON. * @function toJSON - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @instance * @returns {Object.} JSON object */ - ChangeTabletTypeResponse.prototype.toJSON = function toJSON() { + DeleteShardsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ChangeTabletTypeResponse + * Gets the default type url for DeleteShardsResponse * @function getTypeUrl - * @memberof vtctldata.ChangeTabletTypeResponse + * @memberof vtctldata.DeleteShardsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ChangeTabletTypeResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteShardsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ChangeTabletTypeResponse"; + return typeUrlPrefix + "/vtctldata.DeleteShardsResponse"; }; - return ChangeTabletTypeResponse; + return DeleteShardsResponse; })(); - vtctldata.CreateKeyspaceRequest = (function() { + vtctldata.DeleteSrvVSchemaRequest = (function() { /** - * Properties of a CreateKeyspaceRequest. + * Properties of a DeleteSrvVSchemaRequest. * @memberof vtctldata - * @interface ICreateKeyspaceRequest - * @property {string|null} [name] CreateKeyspaceRequest name - * @property {boolean|null} [force] CreateKeyspaceRequest force - * @property {boolean|null} [allow_empty_v_schema] CreateKeyspaceRequest allow_empty_v_schema - * @property {Array.|null} [served_froms] CreateKeyspaceRequest served_froms - * @property {topodata.KeyspaceType|null} [type] CreateKeyspaceRequest type - * @property {string|null} [base_keyspace] CreateKeyspaceRequest base_keyspace - * @property {vttime.ITime|null} [snapshot_time] CreateKeyspaceRequest snapshot_time - * @property {string|null} [durability_policy] CreateKeyspaceRequest durability_policy - * @property {string|null} [sidecar_db_name] CreateKeyspaceRequest sidecar_db_name + * @interface IDeleteSrvVSchemaRequest + * @property {string|null} [cell] DeleteSrvVSchemaRequest cell */ /** - * Constructs a new CreateKeyspaceRequest. + * Constructs a new DeleteSrvVSchemaRequest. * @memberof vtctldata - * @classdesc Represents a CreateKeyspaceRequest. - * @implements ICreateKeyspaceRequest + * @classdesc Represents a DeleteSrvVSchemaRequest. + * @implements IDeleteSrvVSchemaRequest * @constructor - * @param {vtctldata.ICreateKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IDeleteSrvVSchemaRequest=} [properties] Properties to set */ - function CreateKeyspaceRequest(properties) { - this.served_froms = []; + function DeleteSrvVSchemaRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -106693,190 +112467,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * CreateKeyspaceRequest name. - * @member {string} name - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.name = ""; - - /** - * CreateKeyspaceRequest force. - * @member {boolean} force - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.force = false; - - /** - * CreateKeyspaceRequest allow_empty_v_schema. - * @member {boolean} allow_empty_v_schema - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.allow_empty_v_schema = false; - - /** - * CreateKeyspaceRequest served_froms. - * @member {Array.} served_froms - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.served_froms = $util.emptyArray; - - /** - * CreateKeyspaceRequest type. - * @member {topodata.KeyspaceType} type - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.type = 0; - - /** - * CreateKeyspaceRequest base_keyspace. - * @member {string} base_keyspace - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.base_keyspace = ""; - - /** - * CreateKeyspaceRequest snapshot_time. - * @member {vttime.ITime|null|undefined} snapshot_time - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.snapshot_time = null; - - /** - * CreateKeyspaceRequest durability_policy. - * @member {string} durability_policy - * @memberof vtctldata.CreateKeyspaceRequest - * @instance - */ - CreateKeyspaceRequest.prototype.durability_policy = ""; - - /** - * CreateKeyspaceRequest sidecar_db_name. - * @member {string} sidecar_db_name - * @memberof vtctldata.CreateKeyspaceRequest + * DeleteSrvVSchemaRequest cell. + * @member {string} cell + * @memberof vtctldata.DeleteSrvVSchemaRequest * @instance */ - CreateKeyspaceRequest.prototype.sidecar_db_name = ""; + DeleteSrvVSchemaRequest.prototype.cell = ""; /** - * Creates a new CreateKeyspaceRequest instance using the specified properties. + * Creates a new DeleteSrvVSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static - * @param {vtctldata.ICreateKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest instance + * @param {vtctldata.IDeleteSrvVSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest instance */ - CreateKeyspaceRequest.create = function create(properties) { - return new CreateKeyspaceRequest(properties); + DeleteSrvVSchemaRequest.create = function create(properties) { + return new DeleteSrvVSchemaRequest(properties); }; /** - * Encodes the specified CreateKeyspaceRequest message. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. + * Encodes the specified DeleteSrvVSchemaRequest message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static - * @param {vtctldata.ICreateKeyspaceRequest} message CreateKeyspaceRequest message or plain object to encode + * @param {vtctldata.IDeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateKeyspaceRequest.encode = function encode(message, writer) { + DeleteSrvVSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.force); - if (message.allow_empty_v_schema != null && Object.hasOwnProperty.call(message, "allow_empty_v_schema")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_empty_v_schema); - if (message.served_froms != null && message.served_froms.length) - for (let i = 0; i < message.served_froms.length; ++i) - $root.topodata.Keyspace.ServedFrom.encode(message.served_froms[i], writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.type != null && Object.hasOwnProperty.call(message, "type")) - writer.uint32(/* id 7, wireType 0 =*/56).int32(message.type); - if (message.base_keyspace != null && Object.hasOwnProperty.call(message, "base_keyspace")) - writer.uint32(/* id 8, wireType 2 =*/66).string(message.base_keyspace); - if (message.snapshot_time != null && Object.hasOwnProperty.call(message, "snapshot_time")) - $root.vttime.Time.encode(message.snapshot_time, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); - if (message.durability_policy != null && Object.hasOwnProperty.call(message, "durability_policy")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.durability_policy); - if (message.sidecar_db_name != null && Object.hasOwnProperty.call(message, "sidecar_db_name")) - writer.uint32(/* id 11, wireType 2 =*/90).string(message.sidecar_db_name); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); return writer; }; /** - * Encodes the specified CreateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceRequest.verify|verify} messages. + * Encodes the specified DeleteSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static - * @param {vtctldata.ICreateKeyspaceRequest} message CreateKeyspaceRequest message or plain object to encode + * @param {vtctldata.IDeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteSrvVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateKeyspaceRequest message from the specified reader or buffer. + * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest + * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateKeyspaceRequest.decode = function decode(reader, length) { + DeleteSrvVSchemaRequest.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.CreateKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteSrvVSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); - break; - } - case 2: { - message.force = reader.bool(); - break; - } - case 3: { - message.allow_empty_v_schema = reader.bool(); - break; - } - case 6: { - if (!(message.served_froms && message.served_froms.length)) - message.served_froms = []; - message.served_froms.push($root.topodata.Keyspace.ServedFrom.decode(reader, reader.uint32())); - break; - } - case 7: { - message.type = reader.int32(); - break; - } - case 8: { - message.base_keyspace = reader.string(); - break; - } - case 9: { - message.snapshot_time = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 10: { - message.durability_policy = reader.string(); - break; - } - case 11: { - message.sidecar_db_name = reader.string(); + message.cell = reader.string(); break; } default: @@ -106888,229 +112547,121 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a CreateKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest + * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteSrvVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateKeyspaceRequest message. + * Verifies a DeleteSrvVSchemaRequest message. * @function verify - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateKeyspaceRequest.verify = function verify(message) { + DeleteSrvVSchemaRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; - if (message.allow_empty_v_schema != null && message.hasOwnProperty("allow_empty_v_schema")) - if (typeof message.allow_empty_v_schema !== "boolean") - return "allow_empty_v_schema: boolean expected"; - if (message.served_froms != null && message.hasOwnProperty("served_froms")) { - if (!Array.isArray(message.served_froms)) - return "served_froms: array expected"; - for (let i = 0; i < message.served_froms.length; ++i) { - let error = $root.topodata.Keyspace.ServedFrom.verify(message.served_froms[i]); - if (error) - return "served_froms." + error; - } - } - if (message.type != null && message.hasOwnProperty("type")) - switch (message.type) { - default: - return "type: enum value expected"; - case 0: - case 1: - break; - } - if (message.base_keyspace != null && message.hasOwnProperty("base_keyspace")) - if (!$util.isString(message.base_keyspace)) - return "base_keyspace: string expected"; - if (message.snapshot_time != null && message.hasOwnProperty("snapshot_time")) { - let error = $root.vttime.Time.verify(message.snapshot_time); - if (error) - return "snapshot_time." + error; - } - if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) - if (!$util.isString(message.durability_policy)) - return "durability_policy: string expected"; - if (message.sidecar_db_name != null && message.hasOwnProperty("sidecar_db_name")) - if (!$util.isString(message.sidecar_db_name)) - return "sidecar_db_name: string expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; return null; }; /** - * Creates a CreateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.CreateKeyspaceRequest} CreateKeyspaceRequest + * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest */ - CreateKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.CreateKeyspaceRequest) + DeleteSrvVSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteSrvVSchemaRequest) return object; - let message = new $root.vtctldata.CreateKeyspaceRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.force != null) - message.force = Boolean(object.force); - if (object.allow_empty_v_schema != null) - message.allow_empty_v_schema = Boolean(object.allow_empty_v_schema); - if (object.served_froms) { - if (!Array.isArray(object.served_froms)) - throw TypeError(".vtctldata.CreateKeyspaceRequest.served_froms: array expected"); - message.served_froms = []; - for (let i = 0; i < object.served_froms.length; ++i) { - if (typeof object.served_froms[i] !== "object") - throw TypeError(".vtctldata.CreateKeyspaceRequest.served_froms: object expected"); - message.served_froms[i] = $root.topodata.Keyspace.ServedFrom.fromObject(object.served_froms[i]); - } - } - switch (object.type) { - default: - if (typeof object.type === "number") { - message.type = object.type; - break; - } - break; - case "NORMAL": - case 0: - message.type = 0; - break; - case "SNAPSHOT": - case 1: - message.type = 1; - break; - } - if (object.base_keyspace != null) - message.base_keyspace = String(object.base_keyspace); - if (object.snapshot_time != null) { - if (typeof object.snapshot_time !== "object") - throw TypeError(".vtctldata.CreateKeyspaceRequest.snapshot_time: object expected"); - message.snapshot_time = $root.vttime.Time.fromObject(object.snapshot_time); - } - if (object.durability_policy != null) - message.durability_policy = String(object.durability_policy); - if (object.sidecar_db_name != null) - message.sidecar_db_name = String(object.sidecar_db_name); - return message; - }; - - /** - * Creates a plain object from a CreateKeyspaceRequest message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.CreateKeyspaceRequest - * @static - * @param {vtctldata.CreateKeyspaceRequest} message CreateKeyspaceRequest - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - CreateKeyspaceRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.served_froms = []; - if (options.defaults) { - object.name = ""; - object.force = false; - object.allow_empty_v_schema = false; - object.type = options.enums === String ? "NORMAL" : 0; - object.base_keyspace = ""; - object.snapshot_time = null; - object.durability_policy = ""; - object.sidecar_db_name = ""; - } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; - if (message.allow_empty_v_schema != null && message.hasOwnProperty("allow_empty_v_schema")) - object.allow_empty_v_schema = message.allow_empty_v_schema; - if (message.served_froms && message.served_froms.length) { - object.served_froms = []; - for (let j = 0; j < message.served_froms.length; ++j) - object.served_froms[j] = $root.topodata.Keyspace.ServedFrom.toObject(message.served_froms[j], options); - } - if (message.type != null && message.hasOwnProperty("type")) - object.type = options.enums === String ? $root.topodata.KeyspaceType[message.type] === undefined ? message.type : $root.topodata.KeyspaceType[message.type] : message.type; - if (message.base_keyspace != null && message.hasOwnProperty("base_keyspace")) - object.base_keyspace = message.base_keyspace; - if (message.snapshot_time != null && message.hasOwnProperty("snapshot_time")) - object.snapshot_time = $root.vttime.Time.toObject(message.snapshot_time, options); - if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) - object.durability_policy = message.durability_policy; - if (message.sidecar_db_name != null && message.hasOwnProperty("sidecar_db_name")) - object.sidecar_db_name = message.sidecar_db_name; + let message = new $root.vtctldata.DeleteSrvVSchemaRequest(); + if (object.cell != null) + message.cell = String(object.cell); + return message; + }; + + /** + * Creates a plain object from a DeleteSrvVSchemaRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.DeleteSrvVSchemaRequest + * @static + * @param {vtctldata.DeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + DeleteSrvVSchemaRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.cell = ""; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; return object; }; /** - * Converts this CreateKeyspaceRequest to JSON. + * Converts this DeleteSrvVSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @instance * @returns {Object.} JSON object */ - CreateKeyspaceRequest.prototype.toJSON = function toJSON() { + DeleteSrvVSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateKeyspaceRequest + * Gets the default type url for DeleteSrvVSchemaRequest * @function getTypeUrl - * @memberof vtctldata.CreateKeyspaceRequest + * @memberof vtctldata.DeleteSrvVSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteSrvVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.CreateKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.DeleteSrvVSchemaRequest"; }; - return CreateKeyspaceRequest; + return DeleteSrvVSchemaRequest; })(); - vtctldata.CreateKeyspaceResponse = (function() { + vtctldata.DeleteSrvVSchemaResponse = (function() { /** - * Properties of a CreateKeyspaceResponse. + * Properties of a DeleteSrvVSchemaResponse. * @memberof vtctldata - * @interface ICreateKeyspaceResponse - * @property {vtctldata.IKeyspace|null} [keyspace] CreateKeyspaceResponse keyspace + * @interface IDeleteSrvVSchemaResponse */ /** - * Constructs a new CreateKeyspaceResponse. + * Constructs a new DeleteSrvVSchemaResponse. * @memberof vtctldata - * @classdesc Represents a CreateKeyspaceResponse. - * @implements ICreateKeyspaceResponse + * @classdesc Represents a DeleteSrvVSchemaResponse. + * @implements IDeleteSrvVSchemaResponse * @constructor - * @param {vtctldata.ICreateKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IDeleteSrvVSchemaResponse=} [properties] Properties to set */ - function CreateKeyspaceResponse(properties) { + function DeleteSrvVSchemaResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -107118,77 +112669,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * CreateKeyspaceResponse keyspace. - * @member {vtctldata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.CreateKeyspaceResponse - * @instance - */ - CreateKeyspaceResponse.prototype.keyspace = null; - - /** - * Creates a new CreateKeyspaceResponse instance using the specified properties. + * Creates a new DeleteSrvVSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static - * @param {vtctldata.ICreateKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse instance + * @param {vtctldata.IDeleteSrvVSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse instance */ - CreateKeyspaceResponse.create = function create(properties) { - return new CreateKeyspaceResponse(properties); + DeleteSrvVSchemaResponse.create = function create(properties) { + return new DeleteSrvVSchemaResponse(properties); }; /** - * Encodes the specified CreateKeyspaceResponse message. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. + * Encodes the specified DeleteSrvVSchemaResponse message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static - * @param {vtctldata.ICreateKeyspaceResponse} message CreateKeyspaceResponse message or plain object to encode + * @param {vtctldata.IDeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateKeyspaceResponse.encode = function encode(message, writer) { + DeleteSrvVSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified CreateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.CreateKeyspaceResponse.verify|verify} messages. + * Encodes the specified DeleteSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static - * @param {vtctldata.ICreateKeyspaceResponse} message CreateKeyspaceResponse message or plain object to encode + * @param {vtctldata.IDeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteSrvVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateKeyspaceResponse message from the specified reader or buffer. + * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse + * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateKeyspaceResponse.decode = function decode(reader, length) { + DeleteSrvVSchemaResponse.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.CreateKeyspaceResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteSrvVSchemaResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -107198,130 +112735,111 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a CreateKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse + * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteSrvVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateKeyspaceResponse message. + * Verifies a DeleteSrvVSchemaResponse message. * @function verify - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateKeyspaceResponse.verify = function verify(message) { + DeleteSrvVSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.vtctldata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; - } return null; }; /** - * Creates a CreateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.CreateKeyspaceResponse} CreateKeyspaceResponse + * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse */ - CreateKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.CreateKeyspaceResponse) + DeleteSrvVSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteSrvVSchemaResponse) return object; - let message = new $root.vtctldata.CreateKeyspaceResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.CreateKeyspaceResponse.keyspace: object expected"); - message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); - } - return message; + return new $root.vtctldata.DeleteSrvVSchemaResponse(); }; /** - * Creates a plain object from a CreateKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteSrvVSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static - * @param {vtctldata.CreateKeyspaceResponse} message CreateKeyspaceResponse + * @param {vtctldata.DeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CreateKeyspaceResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.keyspace = null; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); - return object; + DeleteSrvVSchemaResponse.toObject = function toObject() { + return {}; }; /** - * Converts this CreateKeyspaceResponse to JSON. + * Converts this DeleteSrvVSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @instance * @returns {Object.} JSON object */ - CreateKeyspaceResponse.prototype.toJSON = function toJSON() { + DeleteSrvVSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateKeyspaceResponse + * Gets the default type url for DeleteSrvVSchemaResponse * @function getTypeUrl - * @memberof vtctldata.CreateKeyspaceResponse + * @memberof vtctldata.DeleteSrvVSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteSrvVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.CreateKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.DeleteSrvVSchemaResponse"; }; - return CreateKeyspaceResponse; + return DeleteSrvVSchemaResponse; })(); - vtctldata.CreateShardRequest = (function() { + vtctldata.DeleteTabletsRequest = (function() { /** - * Properties of a CreateShardRequest. + * Properties of a DeleteTabletsRequest. * @memberof vtctldata - * @interface ICreateShardRequest - * @property {string|null} [keyspace] CreateShardRequest keyspace - * @property {string|null} [shard_name] CreateShardRequest shard_name - * @property {boolean|null} [force] CreateShardRequest force - * @property {boolean|null} [include_parent] CreateShardRequest include_parent + * @interface IDeleteTabletsRequest + * @property {Array.|null} [tablet_aliases] DeleteTabletsRequest tablet_aliases + * @property {boolean|null} [allow_primary] DeleteTabletsRequest allow_primary */ /** - * Constructs a new CreateShardRequest. + * Constructs a new DeleteTabletsRequest. * @memberof vtctldata - * @classdesc Represents a CreateShardRequest. - * @implements ICreateShardRequest + * @classdesc Represents a DeleteTabletsRequest. + * @implements IDeleteTabletsRequest * @constructor - * @param {vtctldata.ICreateShardRequest=} [properties] Properties to set + * @param {vtctldata.IDeleteTabletsRequest=} [properties] Properties to set */ - function CreateShardRequest(properties) { + function DeleteTabletsRequest(properties) { + this.tablet_aliases = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -107329,117 +112847,92 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * CreateShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.CreateShardRequest - * @instance - */ - CreateShardRequest.prototype.keyspace = ""; - - /** - * CreateShardRequest shard_name. - * @member {string} shard_name - * @memberof vtctldata.CreateShardRequest - * @instance - */ - CreateShardRequest.prototype.shard_name = ""; - - /** - * CreateShardRequest force. - * @member {boolean} force - * @memberof vtctldata.CreateShardRequest + * DeleteTabletsRequest tablet_aliases. + * @member {Array.} tablet_aliases + * @memberof vtctldata.DeleteTabletsRequest * @instance */ - CreateShardRequest.prototype.force = false; + DeleteTabletsRequest.prototype.tablet_aliases = $util.emptyArray; /** - * CreateShardRequest include_parent. - * @member {boolean} include_parent - * @memberof vtctldata.CreateShardRequest + * DeleteTabletsRequest allow_primary. + * @member {boolean} allow_primary + * @memberof vtctldata.DeleteTabletsRequest * @instance */ - CreateShardRequest.prototype.include_parent = false; + DeleteTabletsRequest.prototype.allow_primary = false; /** - * Creates a new CreateShardRequest instance using the specified properties. + * Creates a new DeleteTabletsRequest instance using the specified properties. * @function create - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static - * @param {vtctldata.ICreateShardRequest=} [properties] Properties to set - * @returns {vtctldata.CreateShardRequest} CreateShardRequest instance + * @param {vtctldata.IDeleteTabletsRequest=} [properties] Properties to set + * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest instance */ - CreateShardRequest.create = function create(properties) { - return new CreateShardRequest(properties); + DeleteTabletsRequest.create = function create(properties) { + return new DeleteTabletsRequest(properties); }; /** - * Encodes the specified CreateShardRequest message. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. + * Encodes the specified DeleteTabletsRequest message. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static - * @param {vtctldata.ICreateShardRequest} message CreateShardRequest message or plain object to encode + * @param {vtctldata.IDeleteTabletsRequest} message DeleteTabletsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateShardRequest.encode = function encode(message, writer) { + DeleteTabletsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); - if (message.include_parent != null && Object.hasOwnProperty.call(message, "include_parent")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_parent); + if (message.tablet_aliases != null && message.tablet_aliases.length) + for (let i = 0; i < message.tablet_aliases.length; ++i) + $root.topodata.TabletAlias.encode(message.tablet_aliases[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.allow_primary); return writer; }; /** - * Encodes the specified CreateShardRequest message, length delimited. Does not implicitly {@link vtctldata.CreateShardRequest.verify|verify} messages. + * Encodes the specified DeleteTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static - * @param {vtctldata.ICreateShardRequest} message CreateShardRequest message or plain object to encode + * @param {vtctldata.IDeleteTabletsRequest} message DeleteTabletsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + DeleteTabletsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateShardRequest message from the specified reader or buffer. + * Decodes a DeleteTabletsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.CreateShardRequest} CreateShardRequest + * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateShardRequest.decode = function decode(reader, length) { + DeleteTabletsRequest.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.CreateShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteTabletsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + if (!(message.tablet_aliases && message.tablet_aliases.length)) + message.tablet_aliases = []; + message.tablet_aliases.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); break; } case 2: { - message.shard_name = reader.string(); - break; - } - case 3: { - message.force = reader.bool(); - break; - } - case 4: { - message.include_parent = reader.bool(); + message.allow_primary = reader.bool(); break; } default: @@ -107451,149 +112944,147 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a CreateShardRequest message from the specified reader or buffer, length delimited. + * Decodes a DeleteTabletsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.CreateShardRequest} CreateShardRequest + * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateShardRequest.decodeDelimited = function decodeDelimited(reader) { + DeleteTabletsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateShardRequest message. + * Verifies a DeleteTabletsRequest message. * @function verify - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateShardRequest.verify = function verify(message) { + DeleteTabletsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - if (!$util.isString(message.shard_name)) - return "shard_name: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; - if (message.include_parent != null && message.hasOwnProperty("include_parent")) - if (typeof message.include_parent !== "boolean") - return "include_parent: boolean expected"; + if (message.tablet_aliases != null && message.hasOwnProperty("tablet_aliases")) { + if (!Array.isArray(message.tablet_aliases)) + return "tablet_aliases: array expected"; + for (let i = 0; i < message.tablet_aliases.length; ++i) { + let error = $root.topodata.TabletAlias.verify(message.tablet_aliases[i]); + if (error) + return "tablet_aliases." + error; + } + } + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + if (typeof message.allow_primary !== "boolean") + return "allow_primary: boolean expected"; return null; }; /** - * Creates a CreateShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteTabletsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.CreateShardRequest} CreateShardRequest + * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest */ - CreateShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.CreateShardRequest) + DeleteTabletsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteTabletsRequest) return object; - let message = new $root.vtctldata.CreateShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard_name != null) - message.shard_name = String(object.shard_name); - if (object.force != null) - message.force = Boolean(object.force); - if (object.include_parent != null) - message.include_parent = Boolean(object.include_parent); + let message = new $root.vtctldata.DeleteTabletsRequest(); + if (object.tablet_aliases) { + if (!Array.isArray(object.tablet_aliases)) + throw TypeError(".vtctldata.DeleteTabletsRequest.tablet_aliases: array expected"); + message.tablet_aliases = []; + for (let i = 0; i < object.tablet_aliases.length; ++i) { + if (typeof object.tablet_aliases[i] !== "object") + throw TypeError(".vtctldata.DeleteTabletsRequest.tablet_aliases: object expected"); + message.tablet_aliases[i] = $root.topodata.TabletAlias.fromObject(object.tablet_aliases[i]); + } + } + if (object.allow_primary != null) + message.allow_primary = Boolean(object.allow_primary); return message; }; /** - * Creates a plain object from a CreateShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a DeleteTabletsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static - * @param {vtctldata.CreateShardRequest} message CreateShardRequest + * @param {vtctldata.DeleteTabletsRequest} message DeleteTabletsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CreateShardRequest.toObject = function toObject(message, options) { + DeleteTabletsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard_name = ""; - object.force = false; - object.include_parent = false; + if (options.arrays || options.defaults) + object.tablet_aliases = []; + if (options.defaults) + object.allow_primary = false; + if (message.tablet_aliases && message.tablet_aliases.length) { + object.tablet_aliases = []; + for (let j = 0; j < message.tablet_aliases.length; ++j) + object.tablet_aliases[j] = $root.topodata.TabletAlias.toObject(message.tablet_aliases[j], options); } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - object.shard_name = message.shard_name; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; - if (message.include_parent != null && message.hasOwnProperty("include_parent")) - object.include_parent = message.include_parent; + if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) + object.allow_primary = message.allow_primary; return object; }; /** - * Converts this CreateShardRequest to JSON. + * Converts this DeleteTabletsRequest to JSON. * @function toJSON - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @instance * @returns {Object.} JSON object */ - CreateShardRequest.prototype.toJSON = function toJSON() { + DeleteTabletsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateShardRequest + * Gets the default type url for DeleteTabletsRequest * @function getTypeUrl - * @memberof vtctldata.CreateShardRequest + * @memberof vtctldata.DeleteTabletsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteTabletsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.CreateShardRequest"; + return typeUrlPrefix + "/vtctldata.DeleteTabletsRequest"; }; - return CreateShardRequest; + return DeleteTabletsRequest; })(); - vtctldata.CreateShardResponse = (function() { + vtctldata.DeleteTabletsResponse = (function() { /** - * Properties of a CreateShardResponse. + * Properties of a DeleteTabletsResponse. * @memberof vtctldata - * @interface ICreateShardResponse - * @property {vtctldata.IKeyspace|null} [keyspace] CreateShardResponse keyspace - * @property {vtctldata.IShard|null} [shard] CreateShardResponse shard - * @property {boolean|null} [shard_already_exists] CreateShardResponse shard_already_exists + * @interface IDeleteTabletsResponse */ /** - * Constructs a new CreateShardResponse. + * Constructs a new DeleteTabletsResponse. * @memberof vtctldata - * @classdesc Represents a CreateShardResponse. - * @implements ICreateShardResponse + * @classdesc Represents a DeleteTabletsResponse. + * @implements IDeleteTabletsResponse * @constructor - * @param {vtctldata.ICreateShardResponse=} [properties] Properties to set + * @param {vtctldata.IDeleteTabletsResponse=} [properties] Properties to set */ - function CreateShardResponse(properties) { + function DeleteTabletsResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -107601,105 +113092,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * CreateShardResponse keyspace. - * @member {vtctldata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.CreateShardResponse - * @instance - */ - CreateShardResponse.prototype.keyspace = null; - - /** - * CreateShardResponse shard. - * @member {vtctldata.IShard|null|undefined} shard - * @memberof vtctldata.CreateShardResponse - * @instance - */ - CreateShardResponse.prototype.shard = null; - - /** - * CreateShardResponse shard_already_exists. - * @member {boolean} shard_already_exists - * @memberof vtctldata.CreateShardResponse - * @instance - */ - CreateShardResponse.prototype.shard_already_exists = false; - - /** - * Creates a new CreateShardResponse instance using the specified properties. + * Creates a new DeleteTabletsResponse instance using the specified properties. * @function create - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static - * @param {vtctldata.ICreateShardResponse=} [properties] Properties to set - * @returns {vtctldata.CreateShardResponse} CreateShardResponse instance + * @param {vtctldata.IDeleteTabletsResponse=} [properties] Properties to set + * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse instance */ - CreateShardResponse.create = function create(properties) { - return new CreateShardResponse(properties); + DeleteTabletsResponse.create = function create(properties) { + return new DeleteTabletsResponse(properties); }; /** - * Encodes the specified CreateShardResponse message. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. + * Encodes the specified DeleteTabletsResponse message. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static - * @param {vtctldata.ICreateShardResponse} message CreateShardResponse message or plain object to encode + * @param {vtctldata.IDeleteTabletsResponse} message DeleteTabletsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateShardResponse.encode = function encode(message, writer) { + DeleteTabletsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.vtctldata.Shard.encode(message.shard, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.shard_already_exists != null && Object.hasOwnProperty.call(message, "shard_already_exists")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.shard_already_exists); return writer; }; /** - * Encodes the specified CreateShardResponse message, length delimited. Does not implicitly {@link vtctldata.CreateShardResponse.verify|verify} messages. + * Encodes the specified DeleteTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static - * @param {vtctldata.ICreateShardResponse} message CreateShardResponse message or plain object to encode + * @param {vtctldata.IDeleteTabletsResponse} message DeleteTabletsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - CreateShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + DeleteTabletsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a CreateShardResponse message from the specified reader or buffer. + * Decodes a DeleteTabletsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.CreateShardResponse} CreateShardResponse + * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateShardResponse.decode = function decode(reader, length) { + DeleteTabletsResponse.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.CreateShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.DeleteTabletsResponse(); while (reader.pos < end) { let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); - break; - } - case 2: { - message.shard = $root.vtctldata.Shard.decode(reader, reader.uint32()); - break; - } - case 3: { - message.shard_already_exists = reader.bool(); - break; - } + switch (tag >>> 3) { default: reader.skipType(tag & 7); break; @@ -107709,150 +113158,115 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a CreateShardResponse message from the specified reader or buffer, length delimited. + * Decodes a DeleteTabletsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.CreateShardResponse} CreateShardResponse + * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - CreateShardResponse.decodeDelimited = function decodeDelimited(reader) { + DeleteTabletsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a CreateShardResponse message. + * Verifies a DeleteTabletsResponse message. * @function verify - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - CreateShardResponse.verify = function verify(message) { + DeleteTabletsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.vtctldata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; - } - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.vtctldata.Shard.verify(message.shard); - if (error) - return "shard." + error; - } - if (message.shard_already_exists != null && message.hasOwnProperty("shard_already_exists")) - if (typeof message.shard_already_exists !== "boolean") - return "shard_already_exists: boolean expected"; return null; }; /** - * Creates a CreateShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a DeleteTabletsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.CreateShardResponse} CreateShardResponse + * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse */ - CreateShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.CreateShardResponse) + DeleteTabletsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.DeleteTabletsResponse) return object; - let message = new $root.vtctldata.CreateShardResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.CreateShardResponse.keyspace: object expected"); - message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); - } - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.CreateShardResponse.shard: object expected"); - message.shard = $root.vtctldata.Shard.fromObject(object.shard); - } - if (object.shard_already_exists != null) - message.shard_already_exists = Boolean(object.shard_already_exists); - return message; + return new $root.vtctldata.DeleteTabletsResponse(); }; /** - * Creates a plain object from a CreateShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a DeleteTabletsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static - * @param {vtctldata.CreateShardResponse} message CreateShardResponse + * @param {vtctldata.DeleteTabletsResponse} message DeleteTabletsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - CreateShardResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.keyspace = null; - object.shard = null; - object.shard_already_exists = false; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.vtctldata.Shard.toObject(message.shard, options); - if (message.shard_already_exists != null && message.hasOwnProperty("shard_already_exists")) - object.shard_already_exists = message.shard_already_exists; - return object; + DeleteTabletsResponse.toObject = function toObject() { + return {}; }; /** - * Converts this CreateShardResponse to JSON. + * Converts this DeleteTabletsResponse to JSON. * @function toJSON - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @instance * @returns {Object.} JSON object */ - CreateShardResponse.prototype.toJSON = function toJSON() { + DeleteTabletsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for CreateShardResponse + * Gets the default type url for DeleteTabletsResponse * @function getTypeUrl - * @memberof vtctldata.CreateShardResponse + * @memberof vtctldata.DeleteTabletsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - CreateShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + DeleteTabletsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.CreateShardResponse"; + return typeUrlPrefix + "/vtctldata.DeleteTabletsResponse"; }; - return CreateShardResponse; + return DeleteTabletsResponse; })(); - vtctldata.DeleteCellInfoRequest = (function() { + vtctldata.EmergencyReparentShardRequest = (function() { /** - * Properties of a DeleteCellInfoRequest. + * Properties of an EmergencyReparentShardRequest. * @memberof vtctldata - * @interface IDeleteCellInfoRequest - * @property {string|null} [name] DeleteCellInfoRequest name - * @property {boolean|null} [force] DeleteCellInfoRequest force + * @interface IEmergencyReparentShardRequest + * @property {string|null} [keyspace] EmergencyReparentShardRequest keyspace + * @property {string|null} [shard] EmergencyReparentShardRequest shard + * @property {topodata.ITabletAlias|null} [new_primary] EmergencyReparentShardRequest new_primary + * @property {Array.|null} [ignore_replicas] EmergencyReparentShardRequest ignore_replicas + * @property {vttime.IDuration|null} [wait_replicas_timeout] EmergencyReparentShardRequest wait_replicas_timeout + * @property {boolean|null} [prevent_cross_cell_promotion] EmergencyReparentShardRequest prevent_cross_cell_promotion */ /** - * Constructs a new DeleteCellInfoRequest. + * Constructs a new EmergencyReparentShardRequest. * @memberof vtctldata - * @classdesc Represents a DeleteCellInfoRequest. - * @implements IDeleteCellInfoRequest + * @classdesc Represents an EmergencyReparentShardRequest. + * @implements IEmergencyReparentShardRequest * @constructor - * @param {vtctldata.IDeleteCellInfoRequest=} [properties] Properties to set + * @param {vtctldata.IEmergencyReparentShardRequest=} [properties] Properties to set */ - function DeleteCellInfoRequest(properties) { + function EmergencyReparentShardRequest(properties) { + this.ignore_replicas = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -107860,89 +113274,148 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteCellInfoRequest name. - * @member {string} name - * @memberof vtctldata.DeleteCellInfoRequest + * EmergencyReparentShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.EmergencyReparentShardRequest * @instance */ - DeleteCellInfoRequest.prototype.name = ""; + EmergencyReparentShardRequest.prototype.keyspace = ""; /** - * DeleteCellInfoRequest force. - * @member {boolean} force - * @memberof vtctldata.DeleteCellInfoRequest + * EmergencyReparentShardRequest shard. + * @member {string} shard + * @memberof vtctldata.EmergencyReparentShardRequest * @instance */ - DeleteCellInfoRequest.prototype.force = false; + EmergencyReparentShardRequest.prototype.shard = ""; /** - * Creates a new DeleteCellInfoRequest instance using the specified properties. + * EmergencyReparentShardRequest new_primary. + * @member {topodata.ITabletAlias|null|undefined} new_primary + * @memberof vtctldata.EmergencyReparentShardRequest + * @instance + */ + EmergencyReparentShardRequest.prototype.new_primary = null; + + /** + * EmergencyReparentShardRequest ignore_replicas. + * @member {Array.} ignore_replicas + * @memberof vtctldata.EmergencyReparentShardRequest + * @instance + */ + EmergencyReparentShardRequest.prototype.ignore_replicas = $util.emptyArray; + + /** + * EmergencyReparentShardRequest wait_replicas_timeout. + * @member {vttime.IDuration|null|undefined} wait_replicas_timeout + * @memberof vtctldata.EmergencyReparentShardRequest + * @instance + */ + EmergencyReparentShardRequest.prototype.wait_replicas_timeout = null; + + /** + * EmergencyReparentShardRequest prevent_cross_cell_promotion. + * @member {boolean} prevent_cross_cell_promotion + * @memberof vtctldata.EmergencyReparentShardRequest + * @instance + */ + EmergencyReparentShardRequest.prototype.prevent_cross_cell_promotion = false; + + /** + * Creates a new EmergencyReparentShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static - * @param {vtctldata.IDeleteCellInfoRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest instance + * @param {vtctldata.IEmergencyReparentShardRequest=} [properties] Properties to set + * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest instance */ - DeleteCellInfoRequest.create = function create(properties) { - return new DeleteCellInfoRequest(properties); + EmergencyReparentShardRequest.create = function create(properties) { + return new EmergencyReparentShardRequest(properties); }; /** - * Encodes the specified DeleteCellInfoRequest message. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. + * Encodes the specified EmergencyReparentShardRequest message. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static - * @param {vtctldata.IDeleteCellInfoRequest} message DeleteCellInfoRequest message or plain object to encode + * @param {vtctldata.IEmergencyReparentShardRequest} message EmergencyReparentShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellInfoRequest.encode = function encode(message, writer) { + EmergencyReparentShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.force); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) + $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.ignore_replicas != null && message.ignore_replicas.length) + for (let i = 0; i < message.ignore_replicas.length; ++i) + $root.topodata.TabletAlias.encode(message.ignore_replicas[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) + $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.prevent_cross_cell_promotion != null && Object.hasOwnProperty.call(message, "prevent_cross_cell_promotion")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.prevent_cross_cell_promotion); return writer; }; /** - * Encodes the specified DeleteCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoRequest.verify|verify} messages. + * Encodes the specified EmergencyReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static - * @param {vtctldata.IDeleteCellInfoRequest} message DeleteCellInfoRequest message or plain object to encode + * @param {vtctldata.IEmergencyReparentShardRequest} message EmergencyReparentShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + EmergencyReparentShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteCellInfoRequest message from the specified reader or buffer. + * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest + * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellInfoRequest.decode = function decode(reader, length) { + EmergencyReparentShardRequest.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.DeleteCellInfoRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.EmergencyReparentShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.keyspace = reader.string(); break; } case 2: { - message.force = reader.bool(); + message.shard = reader.string(); + break; + } + case 3: { + message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + if (!(message.ignore_replicas && message.ignore_replicas.length)) + message.ignore_replicas = []; + message.ignore_replicas.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); + break; + } + case 5: { + message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + break; + } + case 6: { + message.prevent_cross_cell_promotion = reader.bool(); break; } default: @@ -107954,130 +113427,195 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest + * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { + EmergencyReparentShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteCellInfoRequest message. + * Verifies an EmergencyReparentShardRequest message. * @function verify - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteCellInfoRequest.verify = function verify(message) { + EmergencyReparentShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) { + let error = $root.topodata.TabletAlias.verify(message.new_primary); + if (error) + return "new_primary." + error; + } + if (message.ignore_replicas != null && message.hasOwnProperty("ignore_replicas")) { + if (!Array.isArray(message.ignore_replicas)) + return "ignore_replicas: array expected"; + for (let i = 0; i < message.ignore_replicas.length; ++i) { + let error = $root.topodata.TabletAlias.verify(message.ignore_replicas[i]); + if (error) + return "ignore_replicas." + error; + } + } + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { + let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); + if (error) + return "wait_replicas_timeout." + error; + } + if (message.prevent_cross_cell_promotion != null && message.hasOwnProperty("prevent_cross_cell_promotion")) + if (typeof message.prevent_cross_cell_promotion !== "boolean") + return "prevent_cross_cell_promotion: boolean expected"; return null; }; /** - * Creates a DeleteCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates an EmergencyReparentShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteCellInfoRequest} DeleteCellInfoRequest + * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest */ - DeleteCellInfoRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteCellInfoRequest) + EmergencyReparentShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.EmergencyReparentShardRequest) return object; - let message = new $root.vtctldata.DeleteCellInfoRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.force != null) - message.force = Boolean(object.force); + let message = new $root.vtctldata.EmergencyReparentShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.new_primary != null) { + if (typeof object.new_primary !== "object") + throw TypeError(".vtctldata.EmergencyReparentShardRequest.new_primary: object expected"); + message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); + } + if (object.ignore_replicas) { + if (!Array.isArray(object.ignore_replicas)) + throw TypeError(".vtctldata.EmergencyReparentShardRequest.ignore_replicas: array expected"); + message.ignore_replicas = []; + for (let i = 0; i < object.ignore_replicas.length; ++i) { + if (typeof object.ignore_replicas[i] !== "object") + throw TypeError(".vtctldata.EmergencyReparentShardRequest.ignore_replicas: object expected"); + message.ignore_replicas[i] = $root.topodata.TabletAlias.fromObject(object.ignore_replicas[i]); + } + } + if (object.wait_replicas_timeout != null) { + if (typeof object.wait_replicas_timeout !== "object") + throw TypeError(".vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout: object expected"); + message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + } + if (object.prevent_cross_cell_promotion != null) + message.prevent_cross_cell_promotion = Boolean(object.prevent_cross_cell_promotion); return message; }; /** - * Creates a plain object from a DeleteCellInfoRequest message. Also converts values to other types if specified. + * Creates a plain object from an EmergencyReparentShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static - * @param {vtctldata.DeleteCellInfoRequest} message DeleteCellInfoRequest + * @param {vtctldata.EmergencyReparentShardRequest} message EmergencyReparentShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteCellInfoRequest.toObject = function toObject(message, options) { + EmergencyReparentShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.ignore_replicas = []; if (options.defaults) { - object.name = ""; - object.force = false; + object.keyspace = ""; + object.shard = ""; + object.new_primary = null; + object.wait_replicas_timeout = null; + object.prevent_cross_cell_promotion = false; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) + object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); + if (message.ignore_replicas && message.ignore_replicas.length) { + object.ignore_replicas = []; + for (let j = 0; j < message.ignore_replicas.length; ++j) + object.ignore_replicas[j] = $root.topodata.TabletAlias.toObject(message.ignore_replicas[j], options); + } + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) + object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); + if (message.prevent_cross_cell_promotion != null && message.hasOwnProperty("prevent_cross_cell_promotion")) + object.prevent_cross_cell_promotion = message.prevent_cross_cell_promotion; return object; }; /** - * Converts this DeleteCellInfoRequest to JSON. + * Converts this EmergencyReparentShardRequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @instance * @returns {Object.} JSON object */ - DeleteCellInfoRequest.prototype.toJSON = function toJSON() { + EmergencyReparentShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteCellInfoRequest + * Gets the default type url for EmergencyReparentShardRequest * @function getTypeUrl - * @memberof vtctldata.DeleteCellInfoRequest + * @memberof vtctldata.EmergencyReparentShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + EmergencyReparentShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteCellInfoRequest"; + return typeUrlPrefix + "/vtctldata.EmergencyReparentShardRequest"; }; - return DeleteCellInfoRequest; + return EmergencyReparentShardRequest; })(); - vtctldata.DeleteCellInfoResponse = (function() { + vtctldata.EmergencyReparentShardResponse = (function() { /** - * Properties of a DeleteCellInfoResponse. + * Properties of an EmergencyReparentShardResponse. * @memberof vtctldata - * @interface IDeleteCellInfoResponse + * @interface IEmergencyReparentShardResponse + * @property {string|null} [keyspace] EmergencyReparentShardResponse keyspace + * @property {string|null} [shard] EmergencyReparentShardResponse shard + * @property {topodata.ITabletAlias|null} [promoted_primary] EmergencyReparentShardResponse promoted_primary + * @property {Array.|null} [events] EmergencyReparentShardResponse events */ /** - * Constructs a new DeleteCellInfoResponse. + * Constructs a new EmergencyReparentShardResponse. * @memberof vtctldata - * @classdesc Represents a DeleteCellInfoResponse. - * @implements IDeleteCellInfoResponse + * @classdesc Represents an EmergencyReparentShardResponse. + * @implements IEmergencyReparentShardResponse * @constructor - * @param {vtctldata.IDeleteCellInfoResponse=} [properties] Properties to set + * @param {vtctldata.IEmergencyReparentShardResponse=} [properties] Properties to set */ - function DeleteCellInfoResponse(properties) { + function EmergencyReparentShardResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -108085,63 +113623,122 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteCellInfoResponse instance using the specified properties. + * EmergencyReparentShardResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.EmergencyReparentShardResponse + * @instance + */ + EmergencyReparentShardResponse.prototype.keyspace = ""; + + /** + * EmergencyReparentShardResponse shard. + * @member {string} shard + * @memberof vtctldata.EmergencyReparentShardResponse + * @instance + */ + EmergencyReparentShardResponse.prototype.shard = ""; + + /** + * EmergencyReparentShardResponse promoted_primary. + * @member {topodata.ITabletAlias|null|undefined} promoted_primary + * @memberof vtctldata.EmergencyReparentShardResponse + * @instance + */ + EmergencyReparentShardResponse.prototype.promoted_primary = null; + + /** + * EmergencyReparentShardResponse events. + * @member {Array.} events + * @memberof vtctldata.EmergencyReparentShardResponse + * @instance + */ + EmergencyReparentShardResponse.prototype.events = $util.emptyArray; + + /** + * Creates a new EmergencyReparentShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static - * @param {vtctldata.IDeleteCellInfoResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse instance + * @param {vtctldata.IEmergencyReparentShardResponse=} [properties] Properties to set + * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse instance */ - DeleteCellInfoResponse.create = function create(properties) { - return new DeleteCellInfoResponse(properties); + EmergencyReparentShardResponse.create = function create(properties) { + return new EmergencyReparentShardResponse(properties); }; /** - * Encodes the specified DeleteCellInfoResponse message. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. + * Encodes the specified EmergencyReparentShardResponse message. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static - * @param {vtctldata.IDeleteCellInfoResponse} message DeleteCellInfoResponse message or plain object to encode + * @param {vtctldata.IEmergencyReparentShardResponse} message EmergencyReparentShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellInfoResponse.encode = function encode(message, writer) { + EmergencyReparentShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.promoted_primary != null && Object.hasOwnProperty.call(message, "promoted_primary")) + $root.topodata.TabletAlias.encode(message.promoted_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellInfoResponse.verify|verify} messages. + * Encodes the specified EmergencyReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static - * @param {vtctldata.IDeleteCellInfoResponse} message DeleteCellInfoResponse message or plain object to encode + * @param {vtctldata.IEmergencyReparentShardResponse} message EmergencyReparentShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + EmergencyReparentShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteCellInfoResponse message from the specified reader or buffer. + * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse + * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellInfoResponse.decode = function decode(reader, length) { + EmergencyReparentShardResponse.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.DeleteCellInfoResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.EmergencyReparentShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.promoted_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + break; + } default: reader.skipType(tag & 7); break; @@ -108151,109 +113748,173 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse + * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { + EmergencyReparentShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteCellInfoResponse message. + * Verifies an EmergencyReparentShardResponse message. * @function verify - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteCellInfoResponse.verify = function verify(message) { + EmergencyReparentShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) { + let error = $root.topodata.TabletAlias.verify(message.promoted_primary); + if (error) + return "promoted_primary." + error; + } + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.logutil.Event.verify(message.events[i]); + if (error) + return "events." + error; + } + } return null; }; /** - * Creates a DeleteCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates an EmergencyReparentShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteCellInfoResponse} DeleteCellInfoResponse + * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse */ - DeleteCellInfoResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteCellInfoResponse) + EmergencyReparentShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.EmergencyReparentShardResponse) return object; - return new $root.vtctldata.DeleteCellInfoResponse(); + let message = new $root.vtctldata.EmergencyReparentShardResponse(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.promoted_primary != null) { + if (typeof object.promoted_primary !== "object") + throw TypeError(".vtctldata.EmergencyReparentShardResponse.promoted_primary: object expected"); + message.promoted_primary = $root.topodata.TabletAlias.fromObject(object.promoted_primary); + } + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".vtctldata.EmergencyReparentShardResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".vtctldata.EmergencyReparentShardResponse.events: object expected"); + message.events[i] = $root.logutil.Event.fromObject(object.events[i]); + } + } + return message; }; /** - * Creates a plain object from a DeleteCellInfoResponse message. Also converts values to other types if specified. + * Creates a plain object from an EmergencyReparentShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static - * @param {vtctldata.DeleteCellInfoResponse} message DeleteCellInfoResponse + * @param {vtctldata.EmergencyReparentShardResponse} message EmergencyReparentShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteCellInfoResponse.toObject = function toObject() { - return {}; + EmergencyReparentShardResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.events = []; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.promoted_primary = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) + object.promoted_primary = $root.topodata.TabletAlias.toObject(message.promoted_primary, options); + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + } + return object; }; /** - * Converts this DeleteCellInfoResponse to JSON. + * Converts this EmergencyReparentShardResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @instance * @returns {Object.} JSON object */ - DeleteCellInfoResponse.prototype.toJSON = function toJSON() { + EmergencyReparentShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteCellInfoResponse + * Gets the default type url for EmergencyReparentShardResponse * @function getTypeUrl - * @memberof vtctldata.DeleteCellInfoResponse + * @memberof vtctldata.EmergencyReparentShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + EmergencyReparentShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteCellInfoResponse"; + return typeUrlPrefix + "/vtctldata.EmergencyReparentShardResponse"; }; - return DeleteCellInfoResponse; + return EmergencyReparentShardResponse; })(); - vtctldata.DeleteCellsAliasRequest = (function() { + vtctldata.ExecuteFetchAsAppRequest = (function() { /** - * Properties of a DeleteCellsAliasRequest. + * Properties of an ExecuteFetchAsAppRequest. * @memberof vtctldata - * @interface IDeleteCellsAliasRequest - * @property {string|null} [name] DeleteCellsAliasRequest name + * @interface IExecuteFetchAsAppRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteFetchAsAppRequest tablet_alias + * @property {string|null} [query] ExecuteFetchAsAppRequest query + * @property {number|Long|null} [max_rows] ExecuteFetchAsAppRequest max_rows + * @property {boolean|null} [use_pool] ExecuteFetchAsAppRequest use_pool */ /** - * Constructs a new DeleteCellsAliasRequest. + * Constructs a new ExecuteFetchAsAppRequest. * @memberof vtctldata - * @classdesc Represents a DeleteCellsAliasRequest. - * @implements IDeleteCellsAliasRequest + * @classdesc Represents an ExecuteFetchAsAppRequest. + * @implements IExecuteFetchAsAppRequest * @constructor - * @param {vtctldata.IDeleteCellsAliasRequest=} [properties] Properties to set + * @param {vtctldata.IExecuteFetchAsAppRequest=} [properties] Properties to set */ - function DeleteCellsAliasRequest(properties) { + function ExecuteFetchAsAppRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -108261,75 +113922,117 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteCellsAliasRequest name. - * @member {string} name - * @memberof vtctldata.DeleteCellsAliasRequest + * ExecuteFetchAsAppRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ExecuteFetchAsAppRequest * @instance */ - DeleteCellsAliasRequest.prototype.name = ""; + ExecuteFetchAsAppRequest.prototype.tablet_alias = null; /** - * Creates a new DeleteCellsAliasRequest instance using the specified properties. + * ExecuteFetchAsAppRequest query. + * @member {string} query + * @memberof vtctldata.ExecuteFetchAsAppRequest + * @instance + */ + ExecuteFetchAsAppRequest.prototype.query = ""; + + /** + * ExecuteFetchAsAppRequest max_rows. + * @member {number|Long} max_rows + * @memberof vtctldata.ExecuteFetchAsAppRequest + * @instance + */ + ExecuteFetchAsAppRequest.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * ExecuteFetchAsAppRequest use_pool. + * @member {boolean} use_pool + * @memberof vtctldata.ExecuteFetchAsAppRequest + * @instance + */ + ExecuteFetchAsAppRequest.prototype.use_pool = false; + + /** + * Creates a new ExecuteFetchAsAppRequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static - * @param {vtctldata.IDeleteCellsAliasRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest instance + * @param {vtctldata.IExecuteFetchAsAppRequest=} [properties] Properties to set + * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest instance */ - DeleteCellsAliasRequest.create = function create(properties) { - return new DeleteCellsAliasRequest(properties); + ExecuteFetchAsAppRequest.create = function create(properties) { + return new ExecuteFetchAsAppRequest(properties); }; /** - * Encodes the specified DeleteCellsAliasRequest message. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. + * Encodes the specified ExecuteFetchAsAppRequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static - * @param {vtctldata.IDeleteCellsAliasRequest} message DeleteCellsAliasRequest message or plain object to encode + * @param {vtctldata.IExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellsAliasRequest.encode = function encode(message, writer) { + ExecuteFetchAsAppRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.query); + if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); + if (message.use_pool != null && Object.hasOwnProperty.call(message, "use_pool")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.use_pool); return writer; }; /** - * Encodes the specified DeleteCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasRequest.verify|verify} messages. + * Encodes the specified ExecuteFetchAsAppRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static - * @param {vtctldata.IDeleteCellsAliasRequest} message DeleteCellsAliasRequest message or plain object to encode + * @param {vtctldata.IExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteFetchAsAppRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer. + * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest + * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellsAliasRequest.decode = function decode(reader, length) { + ExecuteFetchAsAppRequest.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.DeleteCellsAliasRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteFetchAsAppRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.query = reader.string(); + break; + } + case 3: { + message.max_rows = reader.int64(); + break; + } + case 4: { + message.use_pool = reader.bool(); break; } default: @@ -108341,121 +114044,166 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest + * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { + ExecuteFetchAsAppRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteCellsAliasRequest message. + * Verifies an ExecuteFetchAsAppRequest message. * @function verify - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteCellsAliasRequest.verify = function verify(message) { + ExecuteFetchAsAppRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.query != null && message.hasOwnProperty("query")) + if (!$util.isString(message.query)) + return "query: string expected"; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) + return "max_rows: integer|Long expected"; + if (message.use_pool != null && message.hasOwnProperty("use_pool")) + if (typeof message.use_pool !== "boolean") + return "use_pool: boolean expected"; return null; }; /** - * Creates a DeleteCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsAppRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteCellsAliasRequest} DeleteCellsAliasRequest + * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest */ - DeleteCellsAliasRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteCellsAliasRequest) + ExecuteFetchAsAppRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteFetchAsAppRequest) return object; - let message = new $root.vtctldata.DeleteCellsAliasRequest(); - if (object.name != null) - message.name = String(object.name); + let message = new $root.vtctldata.ExecuteFetchAsAppRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ExecuteFetchAsAppRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.query != null) + message.query = String(object.query); + if (object.max_rows != null) + if ($util.Long) + (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; + else if (typeof object.max_rows === "string") + message.max_rows = parseInt(object.max_rows, 10); + else if (typeof object.max_rows === "number") + message.max_rows = object.max_rows; + else if (typeof object.max_rows === "object") + message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); + if (object.use_pool != null) + message.use_pool = Boolean(object.use_pool); return message; }; /** - * Creates a plain object from a DeleteCellsAliasRequest message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteFetchAsAppRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static - * @param {vtctldata.DeleteCellsAliasRequest} message DeleteCellsAliasRequest + * @param {vtctldata.ExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteCellsAliasRequest.toObject = function toObject(message, options) { + ExecuteFetchAsAppRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.name = ""; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; + if (options.defaults) { + object.tablet_alias = null; + object.query = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_rows = options.longs === String ? "0" : 0; + object.use_pool = false; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = message.query; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (typeof message.max_rows === "number") + object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; + else + object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; + if (message.use_pool != null && message.hasOwnProperty("use_pool")) + object.use_pool = message.use_pool; return object; }; /** - * Converts this DeleteCellsAliasRequest to JSON. + * Converts this ExecuteFetchAsAppRequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @instance * @returns {Object.} JSON object */ - DeleteCellsAliasRequest.prototype.toJSON = function toJSON() { + ExecuteFetchAsAppRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteCellsAliasRequest + * Gets the default type url for ExecuteFetchAsAppRequest * @function getTypeUrl - * @memberof vtctldata.DeleteCellsAliasRequest + * @memberof vtctldata.ExecuteFetchAsAppRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteFetchAsAppRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteCellsAliasRequest"; + return typeUrlPrefix + "/vtctldata.ExecuteFetchAsAppRequest"; }; - return DeleteCellsAliasRequest; + return ExecuteFetchAsAppRequest; })(); - vtctldata.DeleteCellsAliasResponse = (function() { + vtctldata.ExecuteFetchAsAppResponse = (function() { /** - * Properties of a DeleteCellsAliasResponse. + * Properties of an ExecuteFetchAsAppResponse. * @memberof vtctldata - * @interface IDeleteCellsAliasResponse + * @interface IExecuteFetchAsAppResponse + * @property {query.IQueryResult|null} [result] ExecuteFetchAsAppResponse result */ /** - * Constructs a new DeleteCellsAliasResponse. + * Constructs a new ExecuteFetchAsAppResponse. * @memberof vtctldata - * @classdesc Represents a DeleteCellsAliasResponse. - * @implements IDeleteCellsAliasResponse + * @classdesc Represents an ExecuteFetchAsAppResponse. + * @implements IExecuteFetchAsAppResponse * @constructor - * @param {vtctldata.IDeleteCellsAliasResponse=} [properties] Properties to set + * @param {vtctldata.IExecuteFetchAsAppResponse=} [properties] Properties to set */ - function DeleteCellsAliasResponse(properties) { + function ExecuteFetchAsAppResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -108463,63 +114211,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteCellsAliasResponse instance using the specified properties. + * ExecuteFetchAsAppResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof vtctldata.ExecuteFetchAsAppResponse + * @instance + */ + ExecuteFetchAsAppResponse.prototype.result = null; + + /** + * Creates a new ExecuteFetchAsAppResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static - * @param {vtctldata.IDeleteCellsAliasResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse instance + * @param {vtctldata.IExecuteFetchAsAppResponse=} [properties] Properties to set + * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse instance */ - DeleteCellsAliasResponse.create = function create(properties) { - return new DeleteCellsAliasResponse(properties); + ExecuteFetchAsAppResponse.create = function create(properties) { + return new ExecuteFetchAsAppResponse(properties); }; /** - * Encodes the specified DeleteCellsAliasResponse message. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. + * Encodes the specified ExecuteFetchAsAppResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static - * @param {vtctldata.IDeleteCellsAliasResponse} message DeleteCellsAliasResponse message or plain object to encode + * @param {vtctldata.IExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellsAliasResponse.encode = function encode(message, writer) { + ExecuteFetchAsAppResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteCellsAliasResponse.verify|verify} messages. + * Encodes the specified ExecuteFetchAsAppResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static - * @param {vtctldata.IDeleteCellsAliasResponse} message DeleteCellsAliasResponse message or plain object to encode + * @param {vtctldata.IExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteFetchAsAppResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer. + * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse + * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellsAliasResponse.decode = function decode(reader, length) { + ExecuteFetchAsAppResponse.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.DeleteCellsAliasResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteFetchAsAppResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -108529,111 +114291,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse + * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { + ExecuteFetchAsAppResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteCellsAliasResponse message. + * Verifies an ExecuteFetchAsAppResponse message. * @function verify - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteCellsAliasResponse.verify = function verify(message) { + ExecuteFetchAsAppResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } return null; }; /** - * Creates a DeleteCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsAppResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteCellsAliasResponse} DeleteCellsAliasResponse + * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse */ - DeleteCellsAliasResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteCellsAliasResponse) + ExecuteFetchAsAppResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteFetchAsAppResponse) return object; - return new $root.vtctldata.DeleteCellsAliasResponse(); + let message = new $root.vtctldata.ExecuteFetchAsAppResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".vtctldata.ExecuteFetchAsAppResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } + return message; }; /** - * Creates a plain object from a DeleteCellsAliasResponse message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteFetchAsAppResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static - * @param {vtctldata.DeleteCellsAliasResponse} message DeleteCellsAliasResponse + * @param {vtctldata.ExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteCellsAliasResponse.toObject = function toObject() { - return {}; + ExecuteFetchAsAppResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + return object; }; /** - * Converts this DeleteCellsAliasResponse to JSON. + * Converts this ExecuteFetchAsAppResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @instance * @returns {Object.} JSON object */ - DeleteCellsAliasResponse.prototype.toJSON = function toJSON() { + ExecuteFetchAsAppResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteCellsAliasResponse + * Gets the default type url for ExecuteFetchAsAppResponse * @function getTypeUrl - * @memberof vtctldata.DeleteCellsAliasResponse + * @memberof vtctldata.ExecuteFetchAsAppResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteFetchAsAppResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteCellsAliasResponse"; + return typeUrlPrefix + "/vtctldata.ExecuteFetchAsAppResponse"; }; - return DeleteCellsAliasResponse; + return ExecuteFetchAsAppResponse; })(); - vtctldata.DeleteKeyspaceRequest = (function() { + vtctldata.ExecuteFetchAsDBARequest = (function() { /** - * Properties of a DeleteKeyspaceRequest. + * Properties of an ExecuteFetchAsDBARequest. * @memberof vtctldata - * @interface IDeleteKeyspaceRequest - * @property {string|null} [keyspace] DeleteKeyspaceRequest keyspace - * @property {boolean|null} [recursive] DeleteKeyspaceRequest recursive - * @property {boolean|null} [force] DeleteKeyspaceRequest force + * @interface IExecuteFetchAsDBARequest + * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteFetchAsDBARequest tablet_alias + * @property {string|null} [query] ExecuteFetchAsDBARequest query + * @property {number|Long|null} [max_rows] ExecuteFetchAsDBARequest max_rows + * @property {boolean|null} [disable_binlogs] ExecuteFetchAsDBARequest disable_binlogs + * @property {boolean|null} [reload_schema] ExecuteFetchAsDBARequest reload_schema */ /** - * Constructs a new DeleteKeyspaceRequest. + * Constructs a new ExecuteFetchAsDBARequest. * @memberof vtctldata - * @classdesc Represents a DeleteKeyspaceRequest. - * @implements IDeleteKeyspaceRequest + * @classdesc Represents an ExecuteFetchAsDBARequest. + * @implements IExecuteFetchAsDBARequest * @constructor - * @param {vtctldata.IDeleteKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IExecuteFetchAsDBARequest=} [properties] Properties to set */ - function DeleteKeyspaceRequest(properties) { + function ExecuteFetchAsDBARequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -108641,103 +114423,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteKeyspaceRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.DeleteKeyspaceRequest + * ExecuteFetchAsDBARequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ExecuteFetchAsDBARequest + * @instance + */ + ExecuteFetchAsDBARequest.prototype.tablet_alias = null; + + /** + * ExecuteFetchAsDBARequest query. + * @member {string} query + * @memberof vtctldata.ExecuteFetchAsDBARequest + * @instance + */ + ExecuteFetchAsDBARequest.prototype.query = ""; + + /** + * ExecuteFetchAsDBARequest max_rows. + * @member {number|Long} max_rows + * @memberof vtctldata.ExecuteFetchAsDBARequest * @instance */ - DeleteKeyspaceRequest.prototype.keyspace = ""; + ExecuteFetchAsDBARequest.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * DeleteKeyspaceRequest recursive. - * @member {boolean} recursive - * @memberof vtctldata.DeleteKeyspaceRequest + * ExecuteFetchAsDBARequest disable_binlogs. + * @member {boolean} disable_binlogs + * @memberof vtctldata.ExecuteFetchAsDBARequest * @instance */ - DeleteKeyspaceRequest.prototype.recursive = false; + ExecuteFetchAsDBARequest.prototype.disable_binlogs = false; /** - * DeleteKeyspaceRequest force. - * @member {boolean} force - * @memberof vtctldata.DeleteKeyspaceRequest + * ExecuteFetchAsDBARequest reload_schema. + * @member {boolean} reload_schema + * @memberof vtctldata.ExecuteFetchAsDBARequest * @instance */ - DeleteKeyspaceRequest.prototype.force = false; + ExecuteFetchAsDBARequest.prototype.reload_schema = false; /** - * Creates a new DeleteKeyspaceRequest instance using the specified properties. + * Creates a new ExecuteFetchAsDBARequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static - * @param {vtctldata.IDeleteKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest instance + * @param {vtctldata.IExecuteFetchAsDBARequest=} [properties] Properties to set + * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest instance */ - DeleteKeyspaceRequest.create = function create(properties) { - return new DeleteKeyspaceRequest(properties); + ExecuteFetchAsDBARequest.create = function create(properties) { + return new ExecuteFetchAsDBARequest(properties); }; /** - * Encodes the specified DeleteKeyspaceRequest message. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. + * Encodes the specified ExecuteFetchAsDBARequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static - * @param {vtctldata.IDeleteKeyspaceRequest} message DeleteKeyspaceRequest message or plain object to encode + * @param {vtctldata.IExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteKeyspaceRequest.encode = function encode(message, writer) { + ExecuteFetchAsDBARequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.recursive); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.query != null && Object.hasOwnProperty.call(message, "query")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.query); + if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); + if (message.disable_binlogs != null && Object.hasOwnProperty.call(message, "disable_binlogs")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.disable_binlogs); + if (message.reload_schema != null && Object.hasOwnProperty.call(message, "reload_schema")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.reload_schema); return writer; }; /** - * Encodes the specified DeleteKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceRequest.verify|verify} messages. + * Encodes the specified ExecuteFetchAsDBARequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static - * @param {vtctldata.IDeleteKeyspaceRequest} message DeleteKeyspaceRequest message or plain object to encode + * @param {vtctldata.IExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteFetchAsDBARequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer. + * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest + * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteKeyspaceRequest.decode = function decode(reader, length) { + ExecuteFetchAsDBARequest.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.DeleteKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteFetchAsDBARequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.recursive = reader.bool(); + message.query = reader.string(); break; } case 3: { - message.force = reader.bool(); + message.max_rows = reader.int64(); + break; + } + case 4: { + message.disable_binlogs = reader.bool(); + break; + } + case 5: { + message.reload_schema = reader.bool(); break; } default: @@ -108749,138 +114559,174 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest + * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + ExecuteFetchAsDBARequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteKeyspaceRequest message. + * Verifies an ExecuteFetchAsDBARequest message. * @function verify - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteKeyspaceRequest.verify = function verify(message) { + ExecuteFetchAsDBARequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.recursive != null && message.hasOwnProperty("recursive")) - if (typeof message.recursive !== "boolean") - return "recursive: boolean expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.query != null && message.hasOwnProperty("query")) + if (!$util.isString(message.query)) + return "query: string expected"; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) + return "max_rows: integer|Long expected"; + if (message.disable_binlogs != null && message.hasOwnProperty("disable_binlogs")) + if (typeof message.disable_binlogs !== "boolean") + return "disable_binlogs: boolean expected"; + if (message.reload_schema != null && message.hasOwnProperty("reload_schema")) + if (typeof message.reload_schema !== "boolean") + return "reload_schema: boolean expected"; return null; }; /** - * Creates a DeleteKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsDBARequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteKeyspaceRequest} DeleteKeyspaceRequest + * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest */ - DeleteKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteKeyspaceRequest) + ExecuteFetchAsDBARequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteFetchAsDBARequest) return object; - let message = new $root.vtctldata.DeleteKeyspaceRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.recursive != null) - message.recursive = Boolean(object.recursive); - if (object.force != null) - message.force = Boolean(object.force); + let message = new $root.vtctldata.ExecuteFetchAsDBARequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ExecuteFetchAsDBARequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.query != null) + message.query = String(object.query); + if (object.max_rows != null) + if ($util.Long) + (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; + else if (typeof object.max_rows === "string") + message.max_rows = parseInt(object.max_rows, 10); + else if (typeof object.max_rows === "number") + message.max_rows = object.max_rows; + else if (typeof object.max_rows === "object") + message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); + if (object.disable_binlogs != null) + message.disable_binlogs = Boolean(object.disable_binlogs); + if (object.reload_schema != null) + message.reload_schema = Boolean(object.reload_schema); return message; }; /** - * Creates a plain object from a DeleteKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteFetchAsDBARequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static - * @param {vtctldata.DeleteKeyspaceRequest} message DeleteKeyspaceRequest + * @param {vtctldata.ExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteKeyspaceRequest.toObject = function toObject(message, options) { + ExecuteFetchAsDBARequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.recursive = false; - object.force = false; + object.tablet_alias = null; + object.query = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_rows = options.longs === String ? "0" : 0; + object.disable_binlogs = false; + object.reload_schema = false; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.recursive != null && message.hasOwnProperty("recursive")) - object.recursive = message.recursive; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.query != null && message.hasOwnProperty("query")) + object.query = message.query; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (typeof message.max_rows === "number") + object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; + else + object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; + if (message.disable_binlogs != null && message.hasOwnProperty("disable_binlogs")) + object.disable_binlogs = message.disable_binlogs; + if (message.reload_schema != null && message.hasOwnProperty("reload_schema")) + object.reload_schema = message.reload_schema; return object; }; /** - * Converts this DeleteKeyspaceRequest to JSON. + * Converts this ExecuteFetchAsDBARequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @instance * @returns {Object.} JSON object */ - DeleteKeyspaceRequest.prototype.toJSON = function toJSON() { + ExecuteFetchAsDBARequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteKeyspaceRequest + * Gets the default type url for ExecuteFetchAsDBARequest * @function getTypeUrl - * @memberof vtctldata.DeleteKeyspaceRequest + * @memberof vtctldata.ExecuteFetchAsDBARequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteFetchAsDBARequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.ExecuteFetchAsDBARequest"; }; - return DeleteKeyspaceRequest; + return ExecuteFetchAsDBARequest; })(); - vtctldata.DeleteKeyspaceResponse = (function() { + vtctldata.ExecuteFetchAsDBAResponse = (function() { /** - * Properties of a DeleteKeyspaceResponse. + * Properties of an ExecuteFetchAsDBAResponse. * @memberof vtctldata - * @interface IDeleteKeyspaceResponse + * @interface IExecuteFetchAsDBAResponse + * @property {query.IQueryResult|null} [result] ExecuteFetchAsDBAResponse result */ /** - * Constructs a new DeleteKeyspaceResponse. + * Constructs a new ExecuteFetchAsDBAResponse. * @memberof vtctldata - * @classdesc Represents a DeleteKeyspaceResponse. - * @implements IDeleteKeyspaceResponse + * @classdesc Represents an ExecuteFetchAsDBAResponse. + * @implements IExecuteFetchAsDBAResponse * @constructor - * @param {vtctldata.IDeleteKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IExecuteFetchAsDBAResponse=} [properties] Properties to set */ - function DeleteKeyspaceResponse(properties) { + function ExecuteFetchAsDBAResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -108888,63 +114734,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteKeyspaceResponse instance using the specified properties. + * ExecuteFetchAsDBAResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @instance + */ + ExecuteFetchAsDBAResponse.prototype.result = null; + + /** + * Creates a new ExecuteFetchAsDBAResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static - * @param {vtctldata.IDeleteKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse instance + * @param {vtctldata.IExecuteFetchAsDBAResponse=} [properties] Properties to set + * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse instance */ - DeleteKeyspaceResponse.create = function create(properties) { - return new DeleteKeyspaceResponse(properties); + ExecuteFetchAsDBAResponse.create = function create(properties) { + return new ExecuteFetchAsDBAResponse(properties); }; /** - * Encodes the specified DeleteKeyspaceResponse message. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. + * Encodes the specified ExecuteFetchAsDBAResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static - * @param {vtctldata.IDeleteKeyspaceResponse} message DeleteKeyspaceResponse message or plain object to encode + * @param {vtctldata.IExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteKeyspaceResponse.encode = function encode(message, writer) { + ExecuteFetchAsDBAResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteKeyspaceResponse.verify|verify} messages. + * Encodes the specified ExecuteFetchAsDBAResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static - * @param {vtctldata.IDeleteKeyspaceResponse} message DeleteKeyspaceResponse message or plain object to encode + * @param {vtctldata.IExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteFetchAsDBAResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer. + * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse + * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteKeyspaceResponse.decode = function decode(reader, length) { + ExecuteFetchAsDBAResponse.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.DeleteKeyspaceResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteFetchAsDBAResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -108954,113 +114814,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse + * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + ExecuteFetchAsDBAResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteKeyspaceResponse message. + * Verifies an ExecuteFetchAsDBAResponse message. * @function verify - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteKeyspaceResponse.verify = function verify(message) { + ExecuteFetchAsDBAResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); + if (error) + return "result." + error; + } return null; }; /** - * Creates a DeleteKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteFetchAsDBAResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteKeyspaceResponse} DeleteKeyspaceResponse + * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse */ - DeleteKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteKeyspaceResponse) + ExecuteFetchAsDBAResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteFetchAsDBAResponse) return object; - return new $root.vtctldata.DeleteKeyspaceResponse(); + let message = new $root.vtctldata.ExecuteFetchAsDBAResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".vtctldata.ExecuteFetchAsDBAResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); + } + return message; }; /** - * Creates a plain object from a DeleteKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteFetchAsDBAResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static - * @param {vtctldata.DeleteKeyspaceResponse} message DeleteKeyspaceResponse + * @param {vtctldata.ExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteKeyspaceResponse.toObject = function toObject() { - return {}; + ExecuteFetchAsDBAResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); + return object; }; /** - * Converts this DeleteKeyspaceResponse to JSON. + * Converts this ExecuteFetchAsDBAResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @instance * @returns {Object.} JSON object */ - DeleteKeyspaceResponse.prototype.toJSON = function toJSON() { + ExecuteFetchAsDBAResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteKeyspaceResponse + * Gets the default type url for ExecuteFetchAsDBAResponse * @function getTypeUrl - * @memberof vtctldata.DeleteKeyspaceResponse + * @memberof vtctldata.ExecuteFetchAsDBAResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteFetchAsDBAResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.ExecuteFetchAsDBAResponse"; }; - return DeleteKeyspaceResponse; + return ExecuteFetchAsDBAResponse; })(); - vtctldata.DeleteShardsRequest = (function() { + vtctldata.ExecuteHookRequest = (function() { /** - * Properties of a DeleteShardsRequest. + * Properties of an ExecuteHookRequest. * @memberof vtctldata - * @interface IDeleteShardsRequest - * @property {Array.|null} [shards] DeleteShardsRequest shards - * @property {boolean|null} [recursive] DeleteShardsRequest recursive - * @property {boolean|null} [even_if_serving] DeleteShardsRequest even_if_serving - * @property {boolean|null} [force] DeleteShardsRequest force + * @interface IExecuteHookRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteHookRequest tablet_alias + * @property {tabletmanagerdata.IExecuteHookRequest|null} [tablet_hook_request] ExecuteHookRequest tablet_hook_request */ /** - * Constructs a new DeleteShardsRequest. + * Constructs a new ExecuteHookRequest. * @memberof vtctldata - * @classdesc Represents a DeleteShardsRequest. - * @implements IDeleteShardsRequest + * @classdesc Represents an ExecuteHookRequest. + * @implements IExecuteHookRequest * @constructor - * @param {vtctldata.IDeleteShardsRequest=} [properties] Properties to set + * @param {vtctldata.IExecuteHookRequest=} [properties] Properties to set */ - function DeleteShardsRequest(properties) { - this.shards = []; + function ExecuteHookRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -109068,120 +114943,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteShardsRequest shards. - * @member {Array.} shards - * @memberof vtctldata.DeleteShardsRequest - * @instance - */ - DeleteShardsRequest.prototype.shards = $util.emptyArray; - - /** - * DeleteShardsRequest recursive. - * @member {boolean} recursive - * @memberof vtctldata.DeleteShardsRequest - * @instance - */ - DeleteShardsRequest.prototype.recursive = false; - - /** - * DeleteShardsRequest even_if_serving. - * @member {boolean} even_if_serving - * @memberof vtctldata.DeleteShardsRequest + * ExecuteHookRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ExecuteHookRequest * @instance */ - DeleteShardsRequest.prototype.even_if_serving = false; + ExecuteHookRequest.prototype.tablet_alias = null; /** - * DeleteShardsRequest force. - * @member {boolean} force - * @memberof vtctldata.DeleteShardsRequest + * ExecuteHookRequest tablet_hook_request. + * @member {tabletmanagerdata.IExecuteHookRequest|null|undefined} tablet_hook_request + * @memberof vtctldata.ExecuteHookRequest * @instance */ - DeleteShardsRequest.prototype.force = false; + ExecuteHookRequest.prototype.tablet_hook_request = null; /** - * Creates a new DeleteShardsRequest instance using the specified properties. + * Creates a new ExecuteHookRequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static - * @param {vtctldata.IDeleteShardsRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest instance + * @param {vtctldata.IExecuteHookRequest=} [properties] Properties to set + * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest instance */ - DeleteShardsRequest.create = function create(properties) { - return new DeleteShardsRequest(properties); + ExecuteHookRequest.create = function create(properties) { + return new ExecuteHookRequest(properties); }; /** - * Encodes the specified DeleteShardsRequest message. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. + * Encodes the specified ExecuteHookRequest message. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static - * @param {vtctldata.IDeleteShardsRequest} message DeleteShardsRequest message or plain object to encode + * @param {vtctldata.IExecuteHookRequest} message ExecuteHookRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteShardsRequest.encode = function encode(message, writer) { + ExecuteHookRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shards != null && message.shards.length) - for (let i = 0; i < message.shards.length; ++i) - $root.vtctldata.Shard.encode(message.shards[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.recursive); - if (message.even_if_serving != null && Object.hasOwnProperty.call(message, "even_if_serving")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.even_if_serving); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.force); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.tablet_hook_request != null && Object.hasOwnProperty.call(message, "tablet_hook_request")) + $root.tabletmanagerdata.ExecuteHookRequest.encode(message.tablet_hook_request, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteShardsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsRequest.verify|verify} messages. + * Encodes the specified ExecuteHookRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static - * @param {vtctldata.IDeleteShardsRequest} message DeleteShardsRequest message or plain object to encode + * @param {vtctldata.IExecuteHookRequest} message ExecuteHookRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteShardsRequest.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteHookRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteShardsRequest message from the specified reader or buffer. + * Decodes an ExecuteHookRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest + * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteShardsRequest.decode = function decode(reader, length) { + ExecuteHookRequest.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.DeleteShardsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteHookRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.shards && message.shards.length)) - message.shards = []; - message.shards.push($root.vtctldata.Shard.decode(reader, reader.uint32())); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.recursive = reader.bool(); - break; - } - case 4: { - message.even_if_serving = reader.bool(); - break; - } - case 5: { - message.force = reader.bool(); + message.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.decode(reader, reader.uint32()); break; } default: @@ -109193,164 +115037,141 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteShardsRequest message from the specified reader or buffer, length delimited. + * Decodes an ExecuteHookRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest + * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteShardsRequest.decodeDelimited = function decodeDelimited(reader) { + ExecuteHookRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteShardsRequest message. + * Verifies an ExecuteHookRequest message. * @function verify - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteShardsRequest.verify = function verify(message) { + ExecuteHookRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!Array.isArray(message.shards)) - return "shards: array expected"; - for (let i = 0; i < message.shards.length; ++i) { - let error = $root.vtctldata.Shard.verify(message.shards[i]); - if (error) - return "shards." + error; - } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.tablet_hook_request != null && message.hasOwnProperty("tablet_hook_request")) { + let error = $root.tabletmanagerdata.ExecuteHookRequest.verify(message.tablet_hook_request); + if (error) + return "tablet_hook_request." + error; } - if (message.recursive != null && message.hasOwnProperty("recursive")) - if (typeof message.recursive !== "boolean") - return "recursive: boolean expected"; - if (message.even_if_serving != null && message.hasOwnProperty("even_if_serving")) - if (typeof message.even_if_serving !== "boolean") - return "even_if_serving: boolean expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; return null; }; /** - * Creates a DeleteShardsRequest message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteHookRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteShardsRequest} DeleteShardsRequest + * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest */ - DeleteShardsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteShardsRequest) + ExecuteHookRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteHookRequest) return object; - let message = new $root.vtctldata.DeleteShardsRequest(); - if (object.shards) { - if (!Array.isArray(object.shards)) - throw TypeError(".vtctldata.DeleteShardsRequest.shards: array expected"); - message.shards = []; - for (let i = 0; i < object.shards.length; ++i) { - if (typeof object.shards[i] !== "object") - throw TypeError(".vtctldata.DeleteShardsRequest.shards: object expected"); - message.shards[i] = $root.vtctldata.Shard.fromObject(object.shards[i]); - } + let message = new $root.vtctldata.ExecuteHookRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ExecuteHookRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.tablet_hook_request != null) { + if (typeof object.tablet_hook_request !== "object") + throw TypeError(".vtctldata.ExecuteHookRequest.tablet_hook_request: object expected"); + message.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.fromObject(object.tablet_hook_request); } - if (object.recursive != null) - message.recursive = Boolean(object.recursive); - if (object.even_if_serving != null) - message.even_if_serving = Boolean(object.even_if_serving); - if (object.force != null) - message.force = Boolean(object.force); return message; }; /** - * Creates a plain object from a DeleteShardsRequest message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteHookRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static - * @param {vtctldata.DeleteShardsRequest} message DeleteShardsRequest + * @param {vtctldata.ExecuteHookRequest} message ExecuteHookRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteShardsRequest.toObject = function toObject(message, options) { + ExecuteHookRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.shards = []; if (options.defaults) { - object.recursive = false; - object.even_if_serving = false; - object.force = false; - } - if (message.shards && message.shards.length) { - object.shards = []; - for (let j = 0; j < message.shards.length; ++j) - object.shards[j] = $root.vtctldata.Shard.toObject(message.shards[j], options); + object.tablet_alias = null; + object.tablet_hook_request = null; } - if (message.recursive != null && message.hasOwnProperty("recursive")) - object.recursive = message.recursive; - if (message.even_if_serving != null && message.hasOwnProperty("even_if_serving")) - object.even_if_serving = message.even_if_serving; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.tablet_hook_request != null && message.hasOwnProperty("tablet_hook_request")) + object.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.toObject(message.tablet_hook_request, options); return object; }; /** - * Converts this DeleteShardsRequest to JSON. + * Converts this ExecuteHookRequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @instance * @returns {Object.} JSON object */ - DeleteShardsRequest.prototype.toJSON = function toJSON() { + ExecuteHookRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteShardsRequest + * Gets the default type url for ExecuteHookRequest * @function getTypeUrl - * @memberof vtctldata.DeleteShardsRequest + * @memberof vtctldata.ExecuteHookRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteShardsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteHookRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteShardsRequest"; + return typeUrlPrefix + "/vtctldata.ExecuteHookRequest"; }; - return DeleteShardsRequest; + return ExecuteHookRequest; })(); - vtctldata.DeleteShardsResponse = (function() { + vtctldata.ExecuteHookResponse = (function() { /** - * Properties of a DeleteShardsResponse. + * Properties of an ExecuteHookResponse. * @memberof vtctldata - * @interface IDeleteShardsResponse + * @interface IExecuteHookResponse + * @property {tabletmanagerdata.IExecuteHookResponse|null} [hook_result] ExecuteHookResponse hook_result */ /** - * Constructs a new DeleteShardsResponse. + * Constructs a new ExecuteHookResponse. * @memberof vtctldata - * @classdesc Represents a DeleteShardsResponse. - * @implements IDeleteShardsResponse + * @classdesc Represents an ExecuteHookResponse. + * @implements IExecuteHookResponse * @constructor - * @param {vtctldata.IDeleteShardsResponse=} [properties] Properties to set + * @param {vtctldata.IExecuteHookResponse=} [properties] Properties to set */ - function DeleteShardsResponse(properties) { + function ExecuteHookResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -109358,63 +115179,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteShardsResponse instance using the specified properties. + * ExecuteHookResponse hook_result. + * @member {tabletmanagerdata.IExecuteHookResponse|null|undefined} hook_result + * @memberof vtctldata.ExecuteHookResponse + * @instance + */ + ExecuteHookResponse.prototype.hook_result = null; + + /** + * Creates a new ExecuteHookResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static - * @param {vtctldata.IDeleteShardsResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse instance + * @param {vtctldata.IExecuteHookResponse=} [properties] Properties to set + * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse instance */ - DeleteShardsResponse.create = function create(properties) { - return new DeleteShardsResponse(properties); + ExecuteHookResponse.create = function create(properties) { + return new ExecuteHookResponse(properties); }; /** - * Encodes the specified DeleteShardsResponse message. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. + * Encodes the specified ExecuteHookResponse message. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static - * @param {vtctldata.IDeleteShardsResponse} message DeleteShardsResponse message or plain object to encode + * @param {vtctldata.IExecuteHookResponse} message ExecuteHookResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteShardsResponse.encode = function encode(message, writer) { + ExecuteHookResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.hook_result != null && Object.hasOwnProperty.call(message, "hook_result")) + $root.tabletmanagerdata.ExecuteHookResponse.encode(message.hook_result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteShardsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteShardsResponse.verify|verify} messages. + * Encodes the specified ExecuteHookResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static - * @param {vtctldata.IDeleteShardsResponse} message DeleteShardsResponse message or plain object to encode + * @param {vtctldata.IExecuteHookResponse} message ExecuteHookResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteShardsResponse.encodeDelimited = function encodeDelimited(message, writer) { + ExecuteHookResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteShardsResponse message from the specified reader or buffer. + * Decodes an ExecuteHookResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse + * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteShardsResponse.decode = function decode(reader, length) { + ExecuteHookResponse.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.DeleteShardsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ExecuteHookResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -109424,109 +115259,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteShardsResponse message from the specified reader or buffer, length delimited. + * Decodes an ExecuteHookResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse + * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteShardsResponse.decodeDelimited = function decodeDelimited(reader) { + ExecuteHookResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteShardsResponse message. + * Verifies an ExecuteHookResponse message. * @function verify - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteShardsResponse.verify = function verify(message) { + ExecuteHookResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.hook_result != null && message.hasOwnProperty("hook_result")) { + let error = $root.tabletmanagerdata.ExecuteHookResponse.verify(message.hook_result); + if (error) + return "hook_result." + error; + } return null; }; /** - * Creates a DeleteShardsResponse message from a plain object. Also converts values to their respective internal types. + * Creates an ExecuteHookResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteShardsResponse} DeleteShardsResponse + * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse */ - DeleteShardsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteShardsResponse) + ExecuteHookResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ExecuteHookResponse) return object; - return new $root.vtctldata.DeleteShardsResponse(); + let message = new $root.vtctldata.ExecuteHookResponse(); + if (object.hook_result != null) { + if (typeof object.hook_result !== "object") + throw TypeError(".vtctldata.ExecuteHookResponse.hook_result: object expected"); + message.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.fromObject(object.hook_result); + } + return message; }; /** - * Creates a plain object from a DeleteShardsResponse message. Also converts values to other types if specified. + * Creates a plain object from an ExecuteHookResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static - * @param {vtctldata.DeleteShardsResponse} message DeleteShardsResponse + * @param {vtctldata.ExecuteHookResponse} message ExecuteHookResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteShardsResponse.toObject = function toObject() { - return {}; + ExecuteHookResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.hook_result = null; + if (message.hook_result != null && message.hasOwnProperty("hook_result")) + object.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.toObject(message.hook_result, options); + return object; }; /** - * Converts this DeleteShardsResponse to JSON. + * Converts this ExecuteHookResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @instance * @returns {Object.} JSON object */ - DeleteShardsResponse.prototype.toJSON = function toJSON() { + ExecuteHookResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteShardsResponse + * Gets the default type url for ExecuteHookResponse * @function getTypeUrl - * @memberof vtctldata.DeleteShardsResponse + * @memberof vtctldata.ExecuteHookResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteShardsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ExecuteHookResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteShardsResponse"; + return typeUrlPrefix + "/vtctldata.ExecuteHookResponse"; }; - return DeleteShardsResponse; + return ExecuteHookResponse; })(); - vtctldata.DeleteSrvVSchemaRequest = (function() { + vtctldata.FindAllShardsInKeyspaceRequest = (function() { /** - * Properties of a DeleteSrvVSchemaRequest. + * Properties of a FindAllShardsInKeyspaceRequest. * @memberof vtctldata - * @interface IDeleteSrvVSchemaRequest - * @property {string|null} [cell] DeleteSrvVSchemaRequest cell + * @interface IFindAllShardsInKeyspaceRequest + * @property {string|null} [keyspace] FindAllShardsInKeyspaceRequest keyspace */ /** - * Constructs a new DeleteSrvVSchemaRequest. + * Constructs a new FindAllShardsInKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a DeleteSrvVSchemaRequest. - * @implements IDeleteSrvVSchemaRequest + * @classdesc Represents a FindAllShardsInKeyspaceRequest. + * @implements IFindAllShardsInKeyspaceRequest * @constructor - * @param {vtctldata.IDeleteSrvVSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IFindAllShardsInKeyspaceRequest=} [properties] Properties to set */ - function DeleteSrvVSchemaRequest(properties) { + function FindAllShardsInKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -109534,75 +115387,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteSrvVSchemaRequest cell. - * @member {string} cell - * @memberof vtctldata.DeleteSrvVSchemaRequest + * FindAllShardsInKeyspaceRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @instance */ - DeleteSrvVSchemaRequest.prototype.cell = ""; + FindAllShardsInKeyspaceRequest.prototype.keyspace = ""; /** - * Creates a new DeleteSrvVSchemaRequest instance using the specified properties. + * Creates a new FindAllShardsInKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static - * @param {vtctldata.IDeleteSrvVSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest instance + * @param {vtctldata.IFindAllShardsInKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest instance */ - DeleteSrvVSchemaRequest.create = function create(properties) { - return new DeleteSrvVSchemaRequest(properties); + FindAllShardsInKeyspaceRequest.create = function create(properties) { + return new FindAllShardsInKeyspaceRequest(properties); }; /** - * Encodes the specified DeleteSrvVSchemaRequest message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. + * Encodes the specified FindAllShardsInKeyspaceRequest message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static - * @param {vtctldata.IDeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest message or plain object to encode + * @param {vtctldata.IFindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteSrvVSchemaRequest.encode = function encode(message, writer) { + FindAllShardsInKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); return writer; }; /** - * Encodes the specified DeleteSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaRequest.verify|verify} messages. + * Encodes the specified FindAllShardsInKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static - * @param {vtctldata.IDeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest message or plain object to encode + * @param {vtctldata.IFindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteSrvVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + FindAllShardsInKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer. + * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest + * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteSrvVSchemaRequest.decode = function decode(reader, length) { + FindAllShardsInKeyspaceRequest.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.DeleteSrvVSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.FindAllShardsInKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.cell = reader.string(); + message.keyspace = reader.string(); break; } default: @@ -109614,121 +115467,123 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteSrvVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest + * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteSrvVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + FindAllShardsInKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteSrvVSchemaRequest message. + * Verifies a FindAllShardsInKeyspaceRequest message. * @function verify - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteSrvVSchemaRequest.verify = function verify(message) { + FindAllShardsInKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; return null; }; /** - * Creates a DeleteSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a FindAllShardsInKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteSrvVSchemaRequest} DeleteSrvVSchemaRequest + * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest */ - DeleteSrvVSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteSrvVSchemaRequest) + FindAllShardsInKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.FindAllShardsInKeyspaceRequest) return object; - let message = new $root.vtctldata.DeleteSrvVSchemaRequest(); - if (object.cell != null) - message.cell = String(object.cell); + let message = new $root.vtctldata.FindAllShardsInKeyspaceRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); return message; }; /** - * Creates a plain object from a DeleteSrvVSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a FindAllShardsInKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static - * @param {vtctldata.DeleteSrvVSchemaRequest} message DeleteSrvVSchemaRequest + * @param {vtctldata.FindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteSrvVSchemaRequest.toObject = function toObject(message, options) { + FindAllShardsInKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.cell = ""; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; + object.keyspace = ""; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; return object; }; /** - * Converts this DeleteSrvVSchemaRequest to JSON. + * Converts this FindAllShardsInKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @instance * @returns {Object.} JSON object */ - DeleteSrvVSchemaRequest.prototype.toJSON = function toJSON() { + FindAllShardsInKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteSrvVSchemaRequest + * Gets the default type url for FindAllShardsInKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.DeleteSrvVSchemaRequest + * @memberof vtctldata.FindAllShardsInKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteSrvVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + FindAllShardsInKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteSrvVSchemaRequest"; + return typeUrlPrefix + "/vtctldata.FindAllShardsInKeyspaceRequest"; }; - return DeleteSrvVSchemaRequest; + return FindAllShardsInKeyspaceRequest; })(); - vtctldata.DeleteSrvVSchemaResponse = (function() { + vtctldata.FindAllShardsInKeyspaceResponse = (function() { /** - * Properties of a DeleteSrvVSchemaResponse. + * Properties of a FindAllShardsInKeyspaceResponse. * @memberof vtctldata - * @interface IDeleteSrvVSchemaResponse + * @interface IFindAllShardsInKeyspaceResponse + * @property {Object.|null} [shards] FindAllShardsInKeyspaceResponse shards */ /** - * Constructs a new DeleteSrvVSchemaResponse. + * Constructs a new FindAllShardsInKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a DeleteSrvVSchemaResponse. - * @implements IDeleteSrvVSchemaResponse + * @classdesc Represents a FindAllShardsInKeyspaceResponse. + * @implements IFindAllShardsInKeyspaceResponse * @constructor - * @param {vtctldata.IDeleteSrvVSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IFindAllShardsInKeyspaceResponse=} [properties] Properties to set */ - function DeleteSrvVSchemaResponse(properties) { + function FindAllShardsInKeyspaceResponse(properties) { + this.shards = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -109736,63 +115591,99 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteSrvVSchemaResponse instance using the specified properties. + * FindAllShardsInKeyspaceResponse shards. + * @member {Object.} shards + * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @instance + */ + FindAllShardsInKeyspaceResponse.prototype.shards = $util.emptyObject; + + /** + * Creates a new FindAllShardsInKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static - * @param {vtctldata.IDeleteSrvVSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse instance + * @param {vtctldata.IFindAllShardsInKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse instance */ - DeleteSrvVSchemaResponse.create = function create(properties) { - return new DeleteSrvVSchemaResponse(properties); + FindAllShardsInKeyspaceResponse.create = function create(properties) { + return new FindAllShardsInKeyspaceResponse(properties); }; /** - * Encodes the specified DeleteSrvVSchemaResponse message. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. + * Encodes the specified FindAllShardsInKeyspaceResponse message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static - * @param {vtctldata.IDeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse message or plain object to encode + * @param {vtctldata.IFindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteSrvVSchemaResponse.encode = function encode(message, writer) { + FindAllShardsInKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.shards != null && Object.hasOwnProperty.call(message, "shards")) + for (let keys = Object.keys(message.shards), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.Shard.encode(message.shards[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified DeleteSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteSrvVSchemaResponse.verify|verify} messages. + * Encodes the specified FindAllShardsInKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static - * @param {vtctldata.IDeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse message or plain object to encode + * @param {vtctldata.IFindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteSrvVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + FindAllShardsInKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer. + * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse + * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteSrvVSchemaResponse.decode = function decode(reader, length) { + FindAllShardsInKeyspaceResponse.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.DeleteSrvVSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.FindAllShardsInKeyspaceResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + if (message.shards === $util.emptyObject) + message.shards = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.Shard.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.shards[key] = value; + break; + } default: reader.skipType(tag & 7); break; @@ -109802,111 +115693,145 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteSrvVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse + * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteSrvVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + FindAllShardsInKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteSrvVSchemaResponse message. + * Verifies a FindAllShardsInKeyspaceResponse message. * @function verify - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteSrvVSchemaResponse.verify = function verify(message) { + FindAllShardsInKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.shards != null && message.hasOwnProperty("shards")) { + if (!$util.isObject(message.shards)) + return "shards: object expected"; + let key = Object.keys(message.shards); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.Shard.verify(message.shards[key[i]]); + if (error) + return "shards." + error; + } + } return null; }; /** - * Creates a DeleteSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a FindAllShardsInKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteSrvVSchemaResponse} DeleteSrvVSchemaResponse + * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse */ - DeleteSrvVSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteSrvVSchemaResponse) + FindAllShardsInKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.FindAllShardsInKeyspaceResponse) return object; - return new $root.vtctldata.DeleteSrvVSchemaResponse(); + let message = new $root.vtctldata.FindAllShardsInKeyspaceResponse(); + if (object.shards) { + if (typeof object.shards !== "object") + throw TypeError(".vtctldata.FindAllShardsInKeyspaceResponse.shards: object expected"); + message.shards = {}; + for (let keys = Object.keys(object.shards), i = 0; i < keys.length; ++i) { + if (typeof object.shards[keys[i]] !== "object") + throw TypeError(".vtctldata.FindAllShardsInKeyspaceResponse.shards: object expected"); + message.shards[keys[i]] = $root.vtctldata.Shard.fromObject(object.shards[keys[i]]); + } + } + return message; }; /** - * Creates a plain object from a DeleteSrvVSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a FindAllShardsInKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static - * @param {vtctldata.DeleteSrvVSchemaResponse} message DeleteSrvVSchemaResponse + * @param {vtctldata.FindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteSrvVSchemaResponse.toObject = function toObject() { - return {}; + FindAllShardsInKeyspaceResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.objects || options.defaults) + object.shards = {}; + let keys2; + if (message.shards && (keys2 = Object.keys(message.shards)).length) { + object.shards = {}; + for (let j = 0; j < keys2.length; ++j) + object.shards[keys2[j]] = $root.vtctldata.Shard.toObject(message.shards[keys2[j]], options); + } + return object; }; /** - * Converts this DeleteSrvVSchemaResponse to JSON. + * Converts this FindAllShardsInKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @instance * @returns {Object.} JSON object */ - DeleteSrvVSchemaResponse.prototype.toJSON = function toJSON() { + FindAllShardsInKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteSrvVSchemaResponse + * Gets the default type url for FindAllShardsInKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.DeleteSrvVSchemaResponse + * @memberof vtctldata.FindAllShardsInKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteSrvVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + FindAllShardsInKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteSrvVSchemaResponse"; + return typeUrlPrefix + "/vtctldata.FindAllShardsInKeyspaceResponse"; }; - return DeleteSrvVSchemaResponse; + return FindAllShardsInKeyspaceResponse; })(); - vtctldata.DeleteTabletsRequest = (function() { + vtctldata.GetBackupsRequest = (function() { /** - * Properties of a DeleteTabletsRequest. + * Properties of a GetBackupsRequest. * @memberof vtctldata - * @interface IDeleteTabletsRequest - * @property {Array.|null} [tablet_aliases] DeleteTabletsRequest tablet_aliases - * @property {boolean|null} [allow_primary] DeleteTabletsRequest allow_primary + * @interface IGetBackupsRequest + * @property {string|null} [keyspace] GetBackupsRequest keyspace + * @property {string|null} [shard] GetBackupsRequest shard + * @property {number|null} [limit] GetBackupsRequest limit + * @property {boolean|null} [detailed] GetBackupsRequest detailed + * @property {number|null} [detailed_limit] GetBackupsRequest detailed_limit */ /** - * Constructs a new DeleteTabletsRequest. + * Constructs a new GetBackupsRequest. * @memberof vtctldata - * @classdesc Represents a DeleteTabletsRequest. - * @implements IDeleteTabletsRequest + * @classdesc Represents a GetBackupsRequest. + * @implements IGetBackupsRequest * @constructor - * @param {vtctldata.IDeleteTabletsRequest=} [properties] Properties to set + * @param {vtctldata.IGetBackupsRequest=} [properties] Properties to set */ - function DeleteTabletsRequest(properties) { - this.tablet_aliases = []; + function GetBackupsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -109914,92 +115839,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * DeleteTabletsRequest tablet_aliases. - * @member {Array.} tablet_aliases - * @memberof vtctldata.DeleteTabletsRequest + * GetBackupsRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetBackupsRequest * @instance */ - DeleteTabletsRequest.prototype.tablet_aliases = $util.emptyArray; + GetBackupsRequest.prototype.keyspace = ""; /** - * DeleteTabletsRequest allow_primary. - * @member {boolean} allow_primary - * @memberof vtctldata.DeleteTabletsRequest + * GetBackupsRequest shard. + * @member {string} shard + * @memberof vtctldata.GetBackupsRequest * @instance */ - DeleteTabletsRequest.prototype.allow_primary = false; + GetBackupsRequest.prototype.shard = ""; /** - * Creates a new DeleteTabletsRequest instance using the specified properties. + * GetBackupsRequest limit. + * @member {number} limit + * @memberof vtctldata.GetBackupsRequest + * @instance + */ + GetBackupsRequest.prototype.limit = 0; + + /** + * GetBackupsRequest detailed. + * @member {boolean} detailed + * @memberof vtctldata.GetBackupsRequest + * @instance + */ + GetBackupsRequest.prototype.detailed = false; + + /** + * GetBackupsRequest detailed_limit. + * @member {number} detailed_limit + * @memberof vtctldata.GetBackupsRequest + * @instance + */ + GetBackupsRequest.prototype.detailed_limit = 0; + + /** + * Creates a new GetBackupsRequest instance using the specified properties. * @function create - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static - * @param {vtctldata.IDeleteTabletsRequest=} [properties] Properties to set - * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest instance + * @param {vtctldata.IGetBackupsRequest=} [properties] Properties to set + * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest instance */ - DeleteTabletsRequest.create = function create(properties) { - return new DeleteTabletsRequest(properties); + GetBackupsRequest.create = function create(properties) { + return new GetBackupsRequest(properties); }; /** - * Encodes the specified DeleteTabletsRequest message. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. + * Encodes the specified GetBackupsRequest message. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static - * @param {vtctldata.IDeleteTabletsRequest} message DeleteTabletsRequest message or plain object to encode + * @param {vtctldata.IGetBackupsRequest} message GetBackupsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteTabletsRequest.encode = function encode(message, writer) { + GetBackupsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_aliases != null && message.tablet_aliases.length) - for (let i = 0; i < message.tablet_aliases.length; ++i) - $root.topodata.TabletAlias.encode(message.tablet_aliases[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.allow_primary != null && Object.hasOwnProperty.call(message, "allow_primary")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.allow_primary); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.limit != null && Object.hasOwnProperty.call(message, "limit")) + writer.uint32(/* id 3, wireType 0 =*/24).uint32(message.limit); + if (message.detailed != null && Object.hasOwnProperty.call(message, "detailed")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.detailed); + if (message.detailed_limit != null && Object.hasOwnProperty.call(message, "detailed_limit")) + writer.uint32(/* id 5, wireType 0 =*/40).uint32(message.detailed_limit); return writer; }; /** - * Encodes the specified DeleteTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsRequest.verify|verify} messages. + * Encodes the specified GetBackupsRequest message, length delimited. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static - * @param {vtctldata.IDeleteTabletsRequest} message DeleteTabletsRequest message or plain object to encode + * @param {vtctldata.IGetBackupsRequest} message GetBackupsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteTabletsRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetBackupsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteTabletsRequest message from the specified reader or buffer. + * Decodes a GetBackupsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest + * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteTabletsRequest.decode = function decode(reader, length) { + GetBackupsRequest.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.DeleteTabletsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetBackupsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.tablet_aliases && message.tablet_aliases.length)) - message.tablet_aliases = []; - message.tablet_aliases.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); + message.keyspace = reader.string(); break; } case 2: { - message.allow_primary = reader.bool(); + message.shard = reader.string(); + break; + } + case 3: { + message.limit = reader.uint32(); + break; + } + case 4: { + message.detailed = reader.bool(); + break; + } + case 5: { + message.detailed_limit = reader.uint32(); break; } default: @@ -110011,147 +115975,156 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteTabletsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetBackupsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest + * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteTabletsRequest.decodeDelimited = function decodeDelimited(reader) { + GetBackupsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteTabletsRequest message. + * Verifies a GetBackupsRequest message. * @function verify - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteTabletsRequest.verify = function verify(message) { + GetBackupsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_aliases != null && message.hasOwnProperty("tablet_aliases")) { - if (!Array.isArray(message.tablet_aliases)) - return "tablet_aliases: array expected"; - for (let i = 0; i < message.tablet_aliases.length; ++i) { - let error = $root.topodata.TabletAlias.verify(message.tablet_aliases[i]); - if (error) - return "tablet_aliases." + error; - } - } - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - if (typeof message.allow_primary !== "boolean") - return "allow_primary: boolean expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.limit != null && message.hasOwnProperty("limit")) + if (!$util.isInteger(message.limit)) + return "limit: integer expected"; + if (message.detailed != null && message.hasOwnProperty("detailed")) + if (typeof message.detailed !== "boolean") + return "detailed: boolean expected"; + if (message.detailed_limit != null && message.hasOwnProperty("detailed_limit")) + if (!$util.isInteger(message.detailed_limit)) + return "detailed_limit: integer expected"; return null; }; /** - * Creates a DeleteTabletsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetBackupsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteTabletsRequest} DeleteTabletsRequest + * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest */ - DeleteTabletsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteTabletsRequest) + GetBackupsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetBackupsRequest) return object; - let message = new $root.vtctldata.DeleteTabletsRequest(); - if (object.tablet_aliases) { - if (!Array.isArray(object.tablet_aliases)) - throw TypeError(".vtctldata.DeleteTabletsRequest.tablet_aliases: array expected"); - message.tablet_aliases = []; - for (let i = 0; i < object.tablet_aliases.length; ++i) { - if (typeof object.tablet_aliases[i] !== "object") - throw TypeError(".vtctldata.DeleteTabletsRequest.tablet_aliases: object expected"); - message.tablet_aliases[i] = $root.topodata.TabletAlias.fromObject(object.tablet_aliases[i]); - } - } - if (object.allow_primary != null) - message.allow_primary = Boolean(object.allow_primary); + let message = new $root.vtctldata.GetBackupsRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.limit != null) + message.limit = object.limit >>> 0; + if (object.detailed != null) + message.detailed = Boolean(object.detailed); + if (object.detailed_limit != null) + message.detailed_limit = object.detailed_limit >>> 0; return message; }; /** - * Creates a plain object from a DeleteTabletsRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetBackupsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static - * @param {vtctldata.DeleteTabletsRequest} message DeleteTabletsRequest + * @param {vtctldata.GetBackupsRequest} message GetBackupsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteTabletsRequest.toObject = function toObject(message, options) { + GetBackupsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tablet_aliases = []; - if (options.defaults) - object.allow_primary = false; - if (message.tablet_aliases && message.tablet_aliases.length) { - object.tablet_aliases = []; - for (let j = 0; j < message.tablet_aliases.length; ++j) - object.tablet_aliases[j] = $root.topodata.TabletAlias.toObject(message.tablet_aliases[j], options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.limit = 0; + object.detailed = false; + object.detailed_limit = 0; } - if (message.allow_primary != null && message.hasOwnProperty("allow_primary")) - object.allow_primary = message.allow_primary; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.limit != null && message.hasOwnProperty("limit")) + object.limit = message.limit; + if (message.detailed != null && message.hasOwnProperty("detailed")) + object.detailed = message.detailed; + if (message.detailed_limit != null && message.hasOwnProperty("detailed_limit")) + object.detailed_limit = message.detailed_limit; return object; }; /** - * Converts this DeleteTabletsRequest to JSON. + * Converts this GetBackupsRequest to JSON. * @function toJSON - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @instance * @returns {Object.} JSON object */ - DeleteTabletsRequest.prototype.toJSON = function toJSON() { + GetBackupsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteTabletsRequest + * Gets the default type url for GetBackupsRequest * @function getTypeUrl - * @memberof vtctldata.DeleteTabletsRequest + * @memberof vtctldata.GetBackupsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteTabletsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetBackupsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteTabletsRequest"; + return typeUrlPrefix + "/vtctldata.GetBackupsRequest"; }; - return DeleteTabletsRequest; + return GetBackupsRequest; })(); - vtctldata.DeleteTabletsResponse = (function() { + vtctldata.GetBackupsResponse = (function() { /** - * Properties of a DeleteTabletsResponse. + * Properties of a GetBackupsResponse. * @memberof vtctldata - * @interface IDeleteTabletsResponse + * @interface IGetBackupsResponse + * @property {Array.|null} [backups] GetBackupsResponse backups */ /** - * Constructs a new DeleteTabletsResponse. + * Constructs a new GetBackupsResponse. * @memberof vtctldata - * @classdesc Represents a DeleteTabletsResponse. - * @implements IDeleteTabletsResponse + * @classdesc Represents a GetBackupsResponse. + * @implements IGetBackupsResponse * @constructor - * @param {vtctldata.IDeleteTabletsResponse=} [properties] Properties to set + * @param {vtctldata.IGetBackupsResponse=} [properties] Properties to set */ - function DeleteTabletsResponse(properties) { + function GetBackupsResponse(properties) { + this.backups = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -110159,63 +116132,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new DeleteTabletsResponse instance using the specified properties. + * GetBackupsResponse backups. + * @member {Array.} backups + * @memberof vtctldata.GetBackupsResponse + * @instance + */ + GetBackupsResponse.prototype.backups = $util.emptyArray; + + /** + * Creates a new GetBackupsResponse instance using the specified properties. * @function create - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static - * @param {vtctldata.IDeleteTabletsResponse=} [properties] Properties to set - * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse instance + * @param {vtctldata.IGetBackupsResponse=} [properties] Properties to set + * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse instance */ - DeleteTabletsResponse.create = function create(properties) { - return new DeleteTabletsResponse(properties); + GetBackupsResponse.create = function create(properties) { + return new GetBackupsResponse(properties); }; /** - * Encodes the specified DeleteTabletsResponse message. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. + * Encodes the specified GetBackupsResponse message. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static - * @param {vtctldata.IDeleteTabletsResponse} message DeleteTabletsResponse message or plain object to encode + * @param {vtctldata.IGetBackupsResponse} message GetBackupsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteTabletsResponse.encode = function encode(message, writer) { + GetBackupsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.backups != null && message.backups.length) + for (let i = 0; i < message.backups.length; ++i) + $root.mysqlctl.BackupInfo.encode(message.backups[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified DeleteTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.DeleteTabletsResponse.verify|verify} messages. + * Encodes the specified GetBackupsResponse message, length delimited. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static - * @param {vtctldata.IDeleteTabletsResponse} message DeleteTabletsResponse message or plain object to encode + * @param {vtctldata.IGetBackupsResponse} message GetBackupsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - DeleteTabletsResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetBackupsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a DeleteTabletsResponse message from the specified reader or buffer. + * Decodes a GetBackupsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse + * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteTabletsResponse.decode = function decode(reader, length) { + GetBackupsResponse.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.DeleteTabletsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetBackupsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + if (!(message.backups && message.backups.length)) + message.backups = []; + message.backups.push($root.mysqlctl.BackupInfo.decode(reader, reader.uint32())); + break; + } default: reader.skipType(tag & 7); break; @@ -110225,264 +116215,215 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a DeleteTabletsResponse message from the specified reader or buffer, length delimited. + * Decodes a GetBackupsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse + * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - DeleteTabletsResponse.decodeDelimited = function decodeDelimited(reader) { + GetBackupsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a DeleteTabletsResponse message. + * Verifies a GetBackupsResponse message. * @function verify - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - DeleteTabletsResponse.verify = function verify(message) { + GetBackupsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.backups != null && message.hasOwnProperty("backups")) { + if (!Array.isArray(message.backups)) + return "backups: array expected"; + for (let i = 0; i < message.backups.length; ++i) { + let error = $root.mysqlctl.BackupInfo.verify(message.backups[i]); + if (error) + return "backups." + error; + } + } return null; }; /** - * Creates a DeleteTabletsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetBackupsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.DeleteTabletsResponse} DeleteTabletsResponse + * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse */ - DeleteTabletsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.DeleteTabletsResponse) + GetBackupsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetBackupsResponse) return object; - return new $root.vtctldata.DeleteTabletsResponse(); + let message = new $root.vtctldata.GetBackupsResponse(); + if (object.backups) { + if (!Array.isArray(object.backups)) + throw TypeError(".vtctldata.GetBackupsResponse.backups: array expected"); + message.backups = []; + for (let i = 0; i < object.backups.length; ++i) { + if (typeof object.backups[i] !== "object") + throw TypeError(".vtctldata.GetBackupsResponse.backups: object expected"); + message.backups[i] = $root.mysqlctl.BackupInfo.fromObject(object.backups[i]); + } + } + return message; }; /** - * Creates a plain object from a DeleteTabletsResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetBackupsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static - * @param {vtctldata.DeleteTabletsResponse} message DeleteTabletsResponse + * @param {vtctldata.GetBackupsResponse} message GetBackupsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - DeleteTabletsResponse.toObject = function toObject() { - return {}; + GetBackupsResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.backups = []; + if (message.backups && message.backups.length) { + object.backups = []; + for (let j = 0; j < message.backups.length; ++j) + object.backups[j] = $root.mysqlctl.BackupInfo.toObject(message.backups[j], options); + } + return object; }; /** - * Converts this DeleteTabletsResponse to JSON. + * Converts this GetBackupsResponse to JSON. * @function toJSON - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @instance * @returns {Object.} JSON object */ - DeleteTabletsResponse.prototype.toJSON = function toJSON() { + GetBackupsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for DeleteTabletsResponse + * Gets the default type url for GetBackupsResponse * @function getTypeUrl - * @memberof vtctldata.DeleteTabletsResponse + * @memberof vtctldata.GetBackupsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - DeleteTabletsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetBackupsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.DeleteTabletsResponse"; + return typeUrlPrefix + "/vtctldata.GetBackupsResponse"; }; - return DeleteTabletsResponse; + return GetBackupsResponse; })(); - vtctldata.EmergencyReparentShardRequest = (function() { + vtctldata.GetCellInfoRequest = (function() { /** - * Properties of an EmergencyReparentShardRequest. + * Properties of a GetCellInfoRequest. * @memberof vtctldata - * @interface IEmergencyReparentShardRequest - * @property {string|null} [keyspace] EmergencyReparentShardRequest keyspace - * @property {string|null} [shard] EmergencyReparentShardRequest shard - * @property {topodata.ITabletAlias|null} [new_primary] EmergencyReparentShardRequest new_primary - * @property {Array.|null} [ignore_replicas] EmergencyReparentShardRequest ignore_replicas - * @property {vttime.IDuration|null} [wait_replicas_timeout] EmergencyReparentShardRequest wait_replicas_timeout - * @property {boolean|null} [prevent_cross_cell_promotion] EmergencyReparentShardRequest prevent_cross_cell_promotion + * @interface IGetCellInfoRequest + * @property {string|null} [cell] GetCellInfoRequest cell */ /** - * Constructs a new EmergencyReparentShardRequest. + * Constructs a new GetCellInfoRequest. * @memberof vtctldata - * @classdesc Represents an EmergencyReparentShardRequest. - * @implements IEmergencyReparentShardRequest + * @classdesc Represents a GetCellInfoRequest. + * @implements IGetCellInfoRequest * @constructor - * @param {vtctldata.IEmergencyReparentShardRequest=} [properties] Properties to set - */ - function EmergencyReparentShardRequest(properties) { - this.ignore_replicas = []; - 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]]; - } - - /** - * EmergencyReparentShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.EmergencyReparentShardRequest - * @instance - */ - EmergencyReparentShardRequest.prototype.keyspace = ""; - - /** - * EmergencyReparentShardRequest shard. - * @member {string} shard - * @memberof vtctldata.EmergencyReparentShardRequest - * @instance - */ - EmergencyReparentShardRequest.prototype.shard = ""; - - /** - * EmergencyReparentShardRequest new_primary. - * @member {topodata.ITabletAlias|null|undefined} new_primary - * @memberof vtctldata.EmergencyReparentShardRequest - * @instance - */ - EmergencyReparentShardRequest.prototype.new_primary = null; - - /** - * EmergencyReparentShardRequest ignore_replicas. - * @member {Array.} ignore_replicas - * @memberof vtctldata.EmergencyReparentShardRequest - * @instance - */ - EmergencyReparentShardRequest.prototype.ignore_replicas = $util.emptyArray; - - /** - * EmergencyReparentShardRequest wait_replicas_timeout. - * @member {vttime.IDuration|null|undefined} wait_replicas_timeout - * @memberof vtctldata.EmergencyReparentShardRequest - * @instance + * @param {vtctldata.IGetCellInfoRequest=} [properties] Properties to set */ - EmergencyReparentShardRequest.prototype.wait_replicas_timeout = null; + function GetCellInfoRequest(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]]; + } /** - * EmergencyReparentShardRequest prevent_cross_cell_promotion. - * @member {boolean} prevent_cross_cell_promotion - * @memberof vtctldata.EmergencyReparentShardRequest + * GetCellInfoRequest cell. + * @member {string} cell + * @memberof vtctldata.GetCellInfoRequest * @instance */ - EmergencyReparentShardRequest.prototype.prevent_cross_cell_promotion = false; + GetCellInfoRequest.prototype.cell = ""; /** - * Creates a new EmergencyReparentShardRequest instance using the specified properties. + * Creates a new GetCellInfoRequest instance using the specified properties. * @function create - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static - * @param {vtctldata.IEmergencyReparentShardRequest=} [properties] Properties to set - * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest instance + * @param {vtctldata.IGetCellInfoRequest=} [properties] Properties to set + * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest instance */ - EmergencyReparentShardRequest.create = function create(properties) { - return new EmergencyReparentShardRequest(properties); + GetCellInfoRequest.create = function create(properties) { + return new GetCellInfoRequest(properties); }; /** - * Encodes the specified EmergencyReparentShardRequest message. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. + * Encodes the specified GetCellInfoRequest message. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static - * @param {vtctldata.IEmergencyReparentShardRequest} message EmergencyReparentShardRequest message or plain object to encode + * @param {vtctldata.IGetCellInfoRequest} message GetCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EmergencyReparentShardRequest.encode = function encode(message, writer) { + GetCellInfoRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) - $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.ignore_replicas != null && message.ignore_replicas.length) - for (let i = 0; i < message.ignore_replicas.length; ++i) - $root.topodata.TabletAlias.encode(message.ignore_replicas[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) - $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.prevent_cross_cell_promotion != null && Object.hasOwnProperty.call(message, "prevent_cross_cell_promotion")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.prevent_cross_cell_promotion); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); return writer; }; /** - * Encodes the specified EmergencyReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardRequest.verify|verify} messages. + * Encodes the specified GetCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static - * @param {vtctldata.IEmergencyReparentShardRequest} message EmergencyReparentShardRequest message or plain object to encode + * @param {vtctldata.IGetCellInfoRequest} message GetCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EmergencyReparentShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer. + * Decodes a GetCellInfoRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest + * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EmergencyReparentShardRequest.decode = function decode(reader, length) { + GetCellInfoRequest.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.EmergencyReparentShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellInfoRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - if (!(message.ignore_replicas && message.ignore_replicas.length)) - message.ignore_replicas = []; - message.ignore_replicas.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); - break; - } - case 5: { - message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); - break; - } - case 6: { - message.prevent_cross_cell_promotion = reader.bool(); + message.cell = reader.string(); break; } default: @@ -110494,195 +116435,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an EmergencyReparentShardRequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest + * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EmergencyReparentShardRequest.decodeDelimited = function decodeDelimited(reader) { + GetCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an EmergencyReparentShardRequest message. + * Verifies a GetCellInfoRequest message. * @function verify - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - EmergencyReparentShardRequest.verify = function verify(message) { + GetCellInfoRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) { - let error = $root.topodata.TabletAlias.verify(message.new_primary); - if (error) - return "new_primary." + error; - } - if (message.ignore_replicas != null && message.hasOwnProperty("ignore_replicas")) { - if (!Array.isArray(message.ignore_replicas)) - return "ignore_replicas: array expected"; - for (let i = 0; i < message.ignore_replicas.length; ++i) { - let error = $root.topodata.TabletAlias.verify(message.ignore_replicas[i]); - if (error) - return "ignore_replicas." + error; - } - } - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { - let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); - if (error) - return "wait_replicas_timeout." + error; - } - if (message.prevent_cross_cell_promotion != null && message.hasOwnProperty("prevent_cross_cell_promotion")) - if (typeof message.prevent_cross_cell_promotion !== "boolean") - return "prevent_cross_cell_promotion: boolean expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; return null; }; /** - * Creates an EmergencyReparentShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.EmergencyReparentShardRequest} EmergencyReparentShardRequest + * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest */ - EmergencyReparentShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.EmergencyReparentShardRequest) + GetCellInfoRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellInfoRequest) return object; - let message = new $root.vtctldata.EmergencyReparentShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.new_primary != null) { - if (typeof object.new_primary !== "object") - throw TypeError(".vtctldata.EmergencyReparentShardRequest.new_primary: object expected"); - message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); - } - if (object.ignore_replicas) { - if (!Array.isArray(object.ignore_replicas)) - throw TypeError(".vtctldata.EmergencyReparentShardRequest.ignore_replicas: array expected"); - message.ignore_replicas = []; - for (let i = 0; i < object.ignore_replicas.length; ++i) { - if (typeof object.ignore_replicas[i] !== "object") - throw TypeError(".vtctldata.EmergencyReparentShardRequest.ignore_replicas: object expected"); - message.ignore_replicas[i] = $root.topodata.TabletAlias.fromObject(object.ignore_replicas[i]); - } - } - if (object.wait_replicas_timeout != null) { - if (typeof object.wait_replicas_timeout !== "object") - throw TypeError(".vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout: object expected"); - message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); - } - if (object.prevent_cross_cell_promotion != null) - message.prevent_cross_cell_promotion = Boolean(object.prevent_cross_cell_promotion); + let message = new $root.vtctldata.GetCellInfoRequest(); + if (object.cell != null) + message.cell = String(object.cell); return message; }; /** - * Creates a plain object from an EmergencyReparentShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetCellInfoRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static - * @param {vtctldata.EmergencyReparentShardRequest} message EmergencyReparentShardRequest + * @param {vtctldata.GetCellInfoRequest} message GetCellInfoRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - EmergencyReparentShardRequest.toObject = function toObject(message, options) { + GetCellInfoRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.ignore_replicas = []; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.new_primary = null; - object.wait_replicas_timeout = null; - object.prevent_cross_cell_promotion = false; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) - object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); - if (message.ignore_replicas && message.ignore_replicas.length) { - object.ignore_replicas = []; - for (let j = 0; j < message.ignore_replicas.length; ++j) - object.ignore_replicas[j] = $root.topodata.TabletAlias.toObject(message.ignore_replicas[j], options); - } - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) - object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); - if (message.prevent_cross_cell_promotion != null && message.hasOwnProperty("prevent_cross_cell_promotion")) - object.prevent_cross_cell_promotion = message.prevent_cross_cell_promotion; + if (options.defaults) + object.cell = ""; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; return object; }; /** - * Converts this EmergencyReparentShardRequest to JSON. + * Converts this GetCellInfoRequest to JSON. * @function toJSON - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @instance * @returns {Object.} JSON object */ - EmergencyReparentShardRequest.prototype.toJSON = function toJSON() { + GetCellInfoRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for EmergencyReparentShardRequest + * Gets the default type url for GetCellInfoRequest * @function getTypeUrl - * @memberof vtctldata.EmergencyReparentShardRequest + * @memberof vtctldata.GetCellInfoRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - EmergencyReparentShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.EmergencyReparentShardRequest"; + return typeUrlPrefix + "/vtctldata.GetCellInfoRequest"; }; - return EmergencyReparentShardRequest; + return GetCellInfoRequest; })(); - vtctldata.EmergencyReparentShardResponse = (function() { + vtctldata.GetCellInfoResponse = (function() { /** - * Properties of an EmergencyReparentShardResponse. + * Properties of a GetCellInfoResponse. * @memberof vtctldata - * @interface IEmergencyReparentShardResponse - * @property {string|null} [keyspace] EmergencyReparentShardResponse keyspace - * @property {string|null} [shard] EmergencyReparentShardResponse shard - * @property {topodata.ITabletAlias|null} [promoted_primary] EmergencyReparentShardResponse promoted_primary - * @property {Array.|null} [events] EmergencyReparentShardResponse events + * @interface IGetCellInfoResponse + * @property {topodata.ICellInfo|null} [cell_info] GetCellInfoResponse cell_info */ /** - * Constructs a new EmergencyReparentShardResponse. + * Constructs a new GetCellInfoResponse. * @memberof vtctldata - * @classdesc Represents an EmergencyReparentShardResponse. - * @implements IEmergencyReparentShardResponse + * @classdesc Represents a GetCellInfoResponse. + * @implements IGetCellInfoResponse * @constructor - * @param {vtctldata.IEmergencyReparentShardResponse=} [properties] Properties to set + * @param {vtctldata.IGetCellInfoResponse=} [properties] Properties to set */ - function EmergencyReparentShardResponse(properties) { - this.events = []; + function GetCellInfoResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -110690,120 +116558,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * EmergencyReparentShardResponse keyspace. - * @member {string} keyspace - * @memberof vtctldata.EmergencyReparentShardResponse - * @instance - */ - EmergencyReparentShardResponse.prototype.keyspace = ""; - - /** - * EmergencyReparentShardResponse shard. - * @member {string} shard - * @memberof vtctldata.EmergencyReparentShardResponse - * @instance - */ - EmergencyReparentShardResponse.prototype.shard = ""; - - /** - * EmergencyReparentShardResponse promoted_primary. - * @member {topodata.ITabletAlias|null|undefined} promoted_primary - * @memberof vtctldata.EmergencyReparentShardResponse - * @instance - */ - EmergencyReparentShardResponse.prototype.promoted_primary = null; - - /** - * EmergencyReparentShardResponse events. - * @member {Array.} events - * @memberof vtctldata.EmergencyReparentShardResponse + * GetCellInfoResponse cell_info. + * @member {topodata.ICellInfo|null|undefined} cell_info + * @memberof vtctldata.GetCellInfoResponse * @instance */ - EmergencyReparentShardResponse.prototype.events = $util.emptyArray; + GetCellInfoResponse.prototype.cell_info = null; /** - * Creates a new EmergencyReparentShardResponse instance using the specified properties. + * Creates a new GetCellInfoResponse instance using the specified properties. * @function create - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static - * @param {vtctldata.IEmergencyReparentShardResponse=} [properties] Properties to set - * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse instance + * @param {vtctldata.IGetCellInfoResponse=} [properties] Properties to set + * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse instance */ - EmergencyReparentShardResponse.create = function create(properties) { - return new EmergencyReparentShardResponse(properties); + GetCellInfoResponse.create = function create(properties) { + return new GetCellInfoResponse(properties); }; /** - * Encodes the specified EmergencyReparentShardResponse message. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. + * Encodes the specified GetCellInfoResponse message. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static - * @param {vtctldata.IEmergencyReparentShardResponse} message EmergencyReparentShardResponse message or plain object to encode + * @param {vtctldata.IGetCellInfoResponse} message GetCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EmergencyReparentShardResponse.encode = function encode(message, writer) { + GetCellInfoResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.promoted_primary != null && Object.hasOwnProperty.call(message, "promoted_primary")) - $root.topodata.TabletAlias.encode(message.promoted_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) + $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified EmergencyReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.EmergencyReparentShardResponse.verify|verify} messages. + * Encodes the specified GetCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static - * @param {vtctldata.IEmergencyReparentShardResponse} message EmergencyReparentShardResponse message or plain object to encode + * @param {vtctldata.IGetCellInfoResponse} message GetCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - EmergencyReparentShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer. + * Decodes a GetCellInfoResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse + * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EmergencyReparentShardResponse.decode = function decode(reader, length) { + GetCellInfoResponse.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.EmergencyReparentShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellInfoResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.promoted_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); break; } default: @@ -110815,293 +116638,190 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an EmergencyReparentShardResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse + * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - EmergencyReparentShardResponse.decodeDelimited = function decodeDelimited(reader) { + GetCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an EmergencyReparentShardResponse message. + * Verifies a GetCellInfoResponse message. * @function verify - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - EmergencyReparentShardResponse.verify = function verify(message) { + GetCellInfoResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) { - let error = $root.topodata.TabletAlias.verify(message.promoted_primary); + if (message.cell_info != null && message.hasOwnProperty("cell_info")) { + let error = $root.topodata.CellInfo.verify(message.cell_info); if (error) - return "promoted_primary." + error; - } - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.logutil.Event.verify(message.events[i]); - if (error) - return "events." + error; - } + return "cell_info." + error; } return null; }; /** - * Creates an EmergencyReparentShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.EmergencyReparentShardResponse} EmergencyReparentShardResponse + * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse */ - EmergencyReparentShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.EmergencyReparentShardResponse) + GetCellInfoResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellInfoResponse) return object; - let message = new $root.vtctldata.EmergencyReparentShardResponse(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.promoted_primary != null) { - if (typeof object.promoted_primary !== "object") - throw TypeError(".vtctldata.EmergencyReparentShardResponse.promoted_primary: object expected"); - message.promoted_primary = $root.topodata.TabletAlias.fromObject(object.promoted_primary); - } - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".vtctldata.EmergencyReparentShardResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".vtctldata.EmergencyReparentShardResponse.events: object expected"); - message.events[i] = $root.logutil.Event.fromObject(object.events[i]); - } + let message = new $root.vtctldata.GetCellInfoResponse(); + if (object.cell_info != null) { + if (typeof object.cell_info !== "object") + throw TypeError(".vtctldata.GetCellInfoResponse.cell_info: object expected"); + message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); } return message; }; /** - * Creates a plain object from an EmergencyReparentShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetCellInfoResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static - * @param {vtctldata.EmergencyReparentShardResponse} message EmergencyReparentShardResponse + * @param {vtctldata.GetCellInfoResponse} message GetCellInfoResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - EmergencyReparentShardResponse.toObject = function toObject(message, options) { + GetCellInfoResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.events = []; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.promoted_primary = null; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) - object.promoted_primary = $root.topodata.TabletAlias.toObject(message.promoted_primary, options); - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.logutil.Event.toObject(message.events[j], options); - } + if (options.defaults) + object.cell_info = null; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) + object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); return object; }; /** - * Converts this EmergencyReparentShardResponse to JSON. + * Converts this GetCellInfoResponse to JSON. * @function toJSON - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @instance * @returns {Object.} JSON object */ - EmergencyReparentShardResponse.prototype.toJSON = function toJSON() { + GetCellInfoResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for EmergencyReparentShardResponse + * Gets the default type url for GetCellInfoResponse * @function getTypeUrl - * @memberof vtctldata.EmergencyReparentShardResponse + * @memberof vtctldata.GetCellInfoResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - EmergencyReparentShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.EmergencyReparentShardResponse"; + return typeUrlPrefix + "/vtctldata.GetCellInfoResponse"; }; - return EmergencyReparentShardResponse; + return GetCellInfoResponse; })(); - vtctldata.ExecuteFetchAsAppRequest = (function() { - - /** - * Properties of an ExecuteFetchAsAppRequest. - * @memberof vtctldata - * @interface IExecuteFetchAsAppRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteFetchAsAppRequest tablet_alias - * @property {string|null} [query] ExecuteFetchAsAppRequest query - * @property {number|Long|null} [max_rows] ExecuteFetchAsAppRequest max_rows - * @property {boolean|null} [use_pool] ExecuteFetchAsAppRequest use_pool - */ - - /** - * Constructs a new ExecuteFetchAsAppRequest. - * @memberof vtctldata - * @classdesc Represents an ExecuteFetchAsAppRequest. - * @implements IExecuteFetchAsAppRequest - * @constructor - * @param {vtctldata.IExecuteFetchAsAppRequest=} [properties] Properties to set - */ - function ExecuteFetchAsAppRequest(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]]; - } - - /** - * ExecuteFetchAsAppRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ExecuteFetchAsAppRequest - * @instance - */ - ExecuteFetchAsAppRequest.prototype.tablet_alias = null; - - /** - * ExecuteFetchAsAppRequest query. - * @member {string} query - * @memberof vtctldata.ExecuteFetchAsAppRequest - * @instance - */ - ExecuteFetchAsAppRequest.prototype.query = ""; + vtctldata.GetCellInfoNamesRequest = (function() { /** - * ExecuteFetchAsAppRequest max_rows. - * @member {number|Long} max_rows - * @memberof vtctldata.ExecuteFetchAsAppRequest - * @instance + * Properties of a GetCellInfoNamesRequest. + * @memberof vtctldata + * @interface IGetCellInfoNamesRequest */ - ExecuteFetchAsAppRequest.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * ExecuteFetchAsAppRequest use_pool. - * @member {boolean} use_pool - * @memberof vtctldata.ExecuteFetchAsAppRequest - * @instance + * Constructs a new GetCellInfoNamesRequest. + * @memberof vtctldata + * @classdesc Represents a GetCellInfoNamesRequest. + * @implements IGetCellInfoNamesRequest + * @constructor + * @param {vtctldata.IGetCellInfoNamesRequest=} [properties] Properties to set */ - ExecuteFetchAsAppRequest.prototype.use_pool = false; + function GetCellInfoNamesRequest(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 ExecuteFetchAsAppRequest instance using the specified properties. + * Creates a new GetCellInfoNamesRequest instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static - * @param {vtctldata.IExecuteFetchAsAppRequest=} [properties] Properties to set - * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest instance + * @param {vtctldata.IGetCellInfoNamesRequest=} [properties] Properties to set + * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest instance */ - ExecuteFetchAsAppRequest.create = function create(properties) { - return new ExecuteFetchAsAppRequest(properties); + GetCellInfoNamesRequest.create = function create(properties) { + return new GetCellInfoNamesRequest(properties); }; /** - * Encodes the specified ExecuteFetchAsAppRequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. + * Encodes the specified GetCellInfoNamesRequest message. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static - * @param {vtctldata.IExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest message or plain object to encode + * @param {vtctldata.IGetCellInfoNamesRequest} message GetCellInfoNamesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsAppRequest.encode = function encode(message, writer) { + GetCellInfoNamesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.query); - if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); - if (message.use_pool != null && Object.hasOwnProperty.call(message, "use_pool")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.use_pool); return writer; }; /** - * Encodes the specified ExecuteFetchAsAppRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppRequest.verify|verify} messages. + * Encodes the specified GetCellInfoNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static - * @param {vtctldata.IExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest message or plain object to encode + * @param {vtctldata.IGetCellInfoNamesRequest} message GetCellInfoNamesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsAppRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetCellInfoNamesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer. + * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest + * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsAppRequest.decode = function decode(reader, length) { + GetCellInfoNamesRequest.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.ExecuteFetchAsAppRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellInfoNamesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - message.query = reader.string(); - break; - } - case 3: { - message.max_rows = reader.int64(); - break; - } - case 4: { - message.use_pool = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -111111,166 +116831,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteFetchAsAppRequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest + * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsAppRequest.decodeDelimited = function decodeDelimited(reader) { + GetCellInfoNamesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteFetchAsAppRequest message. + * Verifies a GetCellInfoNamesRequest message. * @function verify - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteFetchAsAppRequest.verify = function verify(message) { + GetCellInfoNamesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.query != null && message.hasOwnProperty("query")) - if (!$util.isString(message.query)) - return "query: string expected"; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) - return "max_rows: integer|Long expected"; - if (message.use_pool != null && message.hasOwnProperty("use_pool")) - if (typeof message.use_pool !== "boolean") - return "use_pool: boolean expected"; return null; }; /** - * Creates an ExecuteFetchAsAppRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoNamesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteFetchAsAppRequest} ExecuteFetchAsAppRequest + * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest */ - ExecuteFetchAsAppRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteFetchAsAppRequest) + GetCellInfoNamesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellInfoNamesRequest) return object; - let message = new $root.vtctldata.ExecuteFetchAsAppRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ExecuteFetchAsAppRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.query != null) - message.query = String(object.query); - if (object.max_rows != null) - if ($util.Long) - (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; - else if (typeof object.max_rows === "string") - message.max_rows = parseInt(object.max_rows, 10); - else if (typeof object.max_rows === "number") - message.max_rows = object.max_rows; - else if (typeof object.max_rows === "object") - message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); - if (object.use_pool != null) - message.use_pool = Boolean(object.use_pool); - return message; + return new $root.vtctldata.GetCellInfoNamesRequest(); }; /** - * Creates a plain object from an ExecuteFetchAsAppRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetCellInfoNamesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static - * @param {vtctldata.ExecuteFetchAsAppRequest} message ExecuteFetchAsAppRequest + * @param {vtctldata.GetCellInfoNamesRequest} message GetCellInfoNamesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteFetchAsAppRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.tablet_alias = null; - object.query = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_rows = options.longs === String ? "0" : 0; - object.use_pool = false; - } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = message.query; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (typeof message.max_rows === "number") - object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; - else - object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; - if (message.use_pool != null && message.hasOwnProperty("use_pool")) - object.use_pool = message.use_pool; - return object; + GetCellInfoNamesRequest.toObject = function toObject() { + return {}; }; /** - * Converts this ExecuteFetchAsAppRequest to JSON. + * Converts this GetCellInfoNamesRequest to JSON. * @function toJSON - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @instance * @returns {Object.} JSON object */ - ExecuteFetchAsAppRequest.prototype.toJSON = function toJSON() { + GetCellInfoNamesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteFetchAsAppRequest + * Gets the default type url for GetCellInfoNamesRequest * @function getTypeUrl - * @memberof vtctldata.ExecuteFetchAsAppRequest + * @memberof vtctldata.GetCellInfoNamesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteFetchAsAppRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellInfoNamesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteFetchAsAppRequest"; + return typeUrlPrefix + "/vtctldata.GetCellInfoNamesRequest"; }; - return ExecuteFetchAsAppRequest; + return GetCellInfoNamesRequest; })(); - vtctldata.ExecuteFetchAsAppResponse = (function() { + vtctldata.GetCellInfoNamesResponse = (function() { /** - * Properties of an ExecuteFetchAsAppResponse. + * Properties of a GetCellInfoNamesResponse. * @memberof vtctldata - * @interface IExecuteFetchAsAppResponse - * @property {query.IQueryResult|null} [result] ExecuteFetchAsAppResponse result + * @interface IGetCellInfoNamesResponse + * @property {Array.|null} [names] GetCellInfoNamesResponse names */ /** - * Constructs a new ExecuteFetchAsAppResponse. + * Constructs a new GetCellInfoNamesResponse. * @memberof vtctldata - * @classdesc Represents an ExecuteFetchAsAppResponse. - * @implements IExecuteFetchAsAppResponse + * @classdesc Represents a GetCellInfoNamesResponse. + * @implements IGetCellInfoNamesResponse * @constructor - * @param {vtctldata.IExecuteFetchAsAppResponse=} [properties] Properties to set + * @param {vtctldata.IGetCellInfoNamesResponse=} [properties] Properties to set */ - function ExecuteFetchAsAppResponse(properties) { + function GetCellInfoNamesResponse(properties) { + this.names = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -111278,75 +116942,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteFetchAsAppResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof vtctldata.ExecuteFetchAsAppResponse + * GetCellInfoNamesResponse names. + * @member {Array.} names + * @memberof vtctldata.GetCellInfoNamesResponse * @instance */ - ExecuteFetchAsAppResponse.prototype.result = null; + GetCellInfoNamesResponse.prototype.names = $util.emptyArray; /** - * Creates a new ExecuteFetchAsAppResponse instance using the specified properties. + * Creates a new GetCellInfoNamesResponse instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static - * @param {vtctldata.IExecuteFetchAsAppResponse=} [properties] Properties to set - * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse instance + * @param {vtctldata.IGetCellInfoNamesResponse=} [properties] Properties to set + * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse instance */ - ExecuteFetchAsAppResponse.create = function create(properties) { - return new ExecuteFetchAsAppResponse(properties); + GetCellInfoNamesResponse.create = function create(properties) { + return new GetCellInfoNamesResponse(properties); }; /** - * Encodes the specified ExecuteFetchAsAppResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. + * Encodes the specified GetCellInfoNamesResponse message. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static - * @param {vtctldata.IExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse message or plain object to encode + * @param {vtctldata.IGetCellInfoNamesResponse} message GetCellInfoNamesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsAppResponse.encode = function encode(message, writer) { + GetCellInfoNamesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.names != null && message.names.length) + for (let i = 0; i < message.names.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.names[i]); return writer; }; /** - * Encodes the specified ExecuteFetchAsAppResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsAppResponse.verify|verify} messages. + * Encodes the specified GetCellInfoNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static - * @param {vtctldata.IExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse message or plain object to encode + * @param {vtctldata.IGetCellInfoNamesResponse} message GetCellInfoNamesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsAppResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetCellInfoNamesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer. + * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse + * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsAppResponse.decode = function decode(reader, length) { + GetCellInfoNamesResponse.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.ExecuteFetchAsAppResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellInfoNamesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + if (!(message.names && message.names.length)) + message.names = []; + message.names.push(reader.string()); break; } default: @@ -111358,131 +117025,133 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteFetchAsAppResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse + * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsAppResponse.decodeDelimited = function decodeDelimited(reader) { + GetCellInfoNamesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteFetchAsAppResponse message. + * Verifies a GetCellInfoNamesResponse message. * @function verify - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteFetchAsAppResponse.verify = function verify(message) { + GetCellInfoNamesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; + if (message.names != null && message.hasOwnProperty("names")) { + if (!Array.isArray(message.names)) + return "names: array expected"; + for (let i = 0; i < message.names.length; ++i) + if (!$util.isString(message.names[i])) + return "names: string[] expected"; } return null; }; /** - * Creates an ExecuteFetchAsAppResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellInfoNamesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteFetchAsAppResponse} ExecuteFetchAsAppResponse + * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse */ - ExecuteFetchAsAppResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteFetchAsAppResponse) + GetCellInfoNamesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellInfoNamesResponse) return object; - let message = new $root.vtctldata.ExecuteFetchAsAppResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".vtctldata.ExecuteFetchAsAppResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.vtctldata.GetCellInfoNamesResponse(); + if (object.names) { + if (!Array.isArray(object.names)) + throw TypeError(".vtctldata.GetCellInfoNamesResponse.names: array expected"); + message.names = []; + for (let i = 0; i < object.names.length; ++i) + message.names[i] = String(object.names[i]); } return message; }; /** - * Creates a plain object from an ExecuteFetchAsAppResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetCellInfoNamesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static - * @param {vtctldata.ExecuteFetchAsAppResponse} message ExecuteFetchAsAppResponse + * @param {vtctldata.GetCellInfoNamesResponse} message GetCellInfoNamesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteFetchAsAppResponse.toObject = function toObject(message, options) { + GetCellInfoNamesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); + if (options.arrays || options.defaults) + object.names = []; + if (message.names && message.names.length) { + object.names = []; + for (let j = 0; j < message.names.length; ++j) + object.names[j] = message.names[j]; + } return object; }; /** - * Converts this ExecuteFetchAsAppResponse to JSON. + * Converts this GetCellInfoNamesResponse to JSON. * @function toJSON - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @instance * @returns {Object.} JSON object */ - ExecuteFetchAsAppResponse.prototype.toJSON = function toJSON() { + GetCellInfoNamesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteFetchAsAppResponse + * Gets the default type url for GetCellInfoNamesResponse * @function getTypeUrl - * @memberof vtctldata.ExecuteFetchAsAppResponse + * @memberof vtctldata.GetCellInfoNamesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteFetchAsAppResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellInfoNamesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteFetchAsAppResponse"; + return typeUrlPrefix + "/vtctldata.GetCellInfoNamesResponse"; }; - return ExecuteFetchAsAppResponse; + return GetCellInfoNamesResponse; })(); - vtctldata.ExecuteFetchAsDBARequest = (function() { + vtctldata.GetCellsAliasesRequest = (function() { /** - * Properties of an ExecuteFetchAsDBARequest. + * Properties of a GetCellsAliasesRequest. * @memberof vtctldata - * @interface IExecuteFetchAsDBARequest - * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteFetchAsDBARequest tablet_alias - * @property {string|null} [query] ExecuteFetchAsDBARequest query - * @property {number|Long|null} [max_rows] ExecuteFetchAsDBARequest max_rows - * @property {boolean|null} [disable_binlogs] ExecuteFetchAsDBARequest disable_binlogs - * @property {boolean|null} [reload_schema] ExecuteFetchAsDBARequest reload_schema + * @interface IGetCellsAliasesRequest */ /** - * Constructs a new ExecuteFetchAsDBARequest. + * Constructs a new GetCellsAliasesRequest. * @memberof vtctldata - * @classdesc Represents an ExecuteFetchAsDBARequest. - * @implements IExecuteFetchAsDBARequest + * @classdesc Represents a GetCellsAliasesRequest. + * @implements IGetCellsAliasesRequest * @constructor - * @param {vtctldata.IExecuteFetchAsDBARequest=} [properties] Properties to set + * @param {vtctldata.IGetCellsAliasesRequest=} [properties] Properties to set */ - function ExecuteFetchAsDBARequest(properties) { + function GetCellsAliasesRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -111490,133 +117159,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteFetchAsDBARequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ExecuteFetchAsDBARequest - * @instance - */ - ExecuteFetchAsDBARequest.prototype.tablet_alias = null; - - /** - * ExecuteFetchAsDBARequest query. - * @member {string} query - * @memberof vtctldata.ExecuteFetchAsDBARequest - * @instance - */ - ExecuteFetchAsDBARequest.prototype.query = ""; - - /** - * ExecuteFetchAsDBARequest max_rows. - * @member {number|Long} max_rows - * @memberof vtctldata.ExecuteFetchAsDBARequest - * @instance - */ - ExecuteFetchAsDBARequest.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - - /** - * ExecuteFetchAsDBARequest disable_binlogs. - * @member {boolean} disable_binlogs - * @memberof vtctldata.ExecuteFetchAsDBARequest - * @instance - */ - ExecuteFetchAsDBARequest.prototype.disable_binlogs = false; - - /** - * ExecuteFetchAsDBARequest reload_schema. - * @member {boolean} reload_schema - * @memberof vtctldata.ExecuteFetchAsDBARequest - * @instance - */ - ExecuteFetchAsDBARequest.prototype.reload_schema = false; - - /** - * Creates a new ExecuteFetchAsDBARequest instance using the specified properties. + * Creates a new GetCellsAliasesRequest instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static - * @param {vtctldata.IExecuteFetchAsDBARequest=} [properties] Properties to set - * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest instance + * @param {vtctldata.IGetCellsAliasesRequest=} [properties] Properties to set + * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest instance */ - ExecuteFetchAsDBARequest.create = function create(properties) { - return new ExecuteFetchAsDBARequest(properties); + GetCellsAliasesRequest.create = function create(properties) { + return new GetCellsAliasesRequest(properties); }; /** - * Encodes the specified ExecuteFetchAsDBARequest message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. + * Encodes the specified GetCellsAliasesRequest message. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static - * @param {vtctldata.IExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest message or plain object to encode + * @param {vtctldata.IGetCellsAliasesRequest} message GetCellsAliasesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsDBARequest.encode = function encode(message, writer) { + GetCellsAliasesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.query != null && Object.hasOwnProperty.call(message, "query")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.query); - if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); - if (message.disable_binlogs != null && Object.hasOwnProperty.call(message, "disable_binlogs")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.disable_binlogs); - if (message.reload_schema != null && Object.hasOwnProperty.call(message, "reload_schema")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.reload_schema); return writer; }; /** - * Encodes the specified ExecuteFetchAsDBARequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBARequest.verify|verify} messages. + * Encodes the specified GetCellsAliasesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static - * @param {vtctldata.IExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest message or plain object to encode + * @param {vtctldata.IGetCellsAliasesRequest} message GetCellsAliasesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsDBARequest.encodeDelimited = function encodeDelimited(message, writer) { + GetCellsAliasesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer. + * Decodes a GetCellsAliasesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest + * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsDBARequest.decode = function decode(reader, length) { + GetCellsAliasesRequest.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.ExecuteFetchAsDBARequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellsAliasesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - message.query = reader.string(); - break; - } - case 3: { - message.max_rows = reader.int64(); - break; - } - case 4: { - message.disable_binlogs = reader.bool(); - break; - } - case 5: { - message.reload_schema = reader.bool(); - break; - } default: reader.skipType(tag & 7); break; @@ -111626,174 +117225,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteFetchAsDBARequest message from the specified reader or buffer, length delimited. + * Decodes a GetCellsAliasesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest + * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsDBARequest.decodeDelimited = function decodeDelimited(reader) { + GetCellsAliasesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteFetchAsDBARequest message. + * Verifies a GetCellsAliasesRequest message. * @function verify - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteFetchAsDBARequest.verify = function verify(message) { + GetCellsAliasesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.query != null && message.hasOwnProperty("query")) - if (!$util.isString(message.query)) - return "query: string expected"; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) - return "max_rows: integer|Long expected"; - if (message.disable_binlogs != null && message.hasOwnProperty("disable_binlogs")) - if (typeof message.disable_binlogs !== "boolean") - return "disable_binlogs: boolean expected"; - if (message.reload_schema != null && message.hasOwnProperty("reload_schema")) - if (typeof message.reload_schema !== "boolean") - return "reload_schema: boolean expected"; return null; }; /** - * Creates an ExecuteFetchAsDBARequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellsAliasesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteFetchAsDBARequest} ExecuteFetchAsDBARequest + * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest */ - ExecuteFetchAsDBARequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteFetchAsDBARequest) + GetCellsAliasesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellsAliasesRequest) return object; - let message = new $root.vtctldata.ExecuteFetchAsDBARequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ExecuteFetchAsDBARequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.query != null) - message.query = String(object.query); - if (object.max_rows != null) - if ($util.Long) - (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; - else if (typeof object.max_rows === "string") - message.max_rows = parseInt(object.max_rows, 10); - else if (typeof object.max_rows === "number") - message.max_rows = object.max_rows; - else if (typeof object.max_rows === "object") - message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); - if (object.disable_binlogs != null) - message.disable_binlogs = Boolean(object.disable_binlogs); - if (object.reload_schema != null) - message.reload_schema = Boolean(object.reload_schema); - return message; + return new $root.vtctldata.GetCellsAliasesRequest(); }; /** - * Creates a plain object from an ExecuteFetchAsDBARequest message. Also converts values to other types if specified. + * Creates a plain object from a GetCellsAliasesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static - * @param {vtctldata.ExecuteFetchAsDBARequest} message ExecuteFetchAsDBARequest + * @param {vtctldata.GetCellsAliasesRequest} message GetCellsAliasesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteFetchAsDBARequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.tablet_alias = null; - object.query = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_rows = options.longs === String ? "0" : 0; - object.disable_binlogs = false; - object.reload_schema = false; - } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.query != null && message.hasOwnProperty("query")) - object.query = message.query; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (typeof message.max_rows === "number") - object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; - else - object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; - if (message.disable_binlogs != null && message.hasOwnProperty("disable_binlogs")) - object.disable_binlogs = message.disable_binlogs; - if (message.reload_schema != null && message.hasOwnProperty("reload_schema")) - object.reload_schema = message.reload_schema; - return object; + GetCellsAliasesRequest.toObject = function toObject() { + return {}; }; /** - * Converts this ExecuteFetchAsDBARequest to JSON. + * Converts this GetCellsAliasesRequest to JSON. * @function toJSON - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @instance * @returns {Object.} JSON object */ - ExecuteFetchAsDBARequest.prototype.toJSON = function toJSON() { + GetCellsAliasesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteFetchAsDBARequest + * Gets the default type url for GetCellsAliasesRequest * @function getTypeUrl - * @memberof vtctldata.ExecuteFetchAsDBARequest + * @memberof vtctldata.GetCellsAliasesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteFetchAsDBARequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellsAliasesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteFetchAsDBARequest"; + return typeUrlPrefix + "/vtctldata.GetCellsAliasesRequest"; }; - return ExecuteFetchAsDBARequest; + return GetCellsAliasesRequest; })(); - vtctldata.ExecuteFetchAsDBAResponse = (function() { + vtctldata.GetCellsAliasesResponse = (function() { /** - * Properties of an ExecuteFetchAsDBAResponse. + * Properties of a GetCellsAliasesResponse. * @memberof vtctldata - * @interface IExecuteFetchAsDBAResponse - * @property {query.IQueryResult|null} [result] ExecuteFetchAsDBAResponse result + * @interface IGetCellsAliasesResponse + * @property {Object.|null} [aliases] GetCellsAliasesResponse aliases */ /** - * Constructs a new ExecuteFetchAsDBAResponse. + * Constructs a new GetCellsAliasesResponse. * @memberof vtctldata - * @classdesc Represents an ExecuteFetchAsDBAResponse. - * @implements IExecuteFetchAsDBAResponse + * @classdesc Represents a GetCellsAliasesResponse. + * @implements IGetCellsAliasesResponse * @constructor - * @param {vtctldata.IExecuteFetchAsDBAResponse=} [properties] Properties to set + * @param {vtctldata.IGetCellsAliasesResponse=} [properties] Properties to set */ - function ExecuteFetchAsDBAResponse(properties) { + function GetCellsAliasesResponse(properties) { + this.aliases = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -111801,75 +117336,97 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteFetchAsDBAResponse result. - * @member {query.IQueryResult|null|undefined} result - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * GetCellsAliasesResponse aliases. + * @member {Object.} aliases + * @memberof vtctldata.GetCellsAliasesResponse * @instance */ - ExecuteFetchAsDBAResponse.prototype.result = null; + GetCellsAliasesResponse.prototype.aliases = $util.emptyObject; /** - * Creates a new ExecuteFetchAsDBAResponse instance using the specified properties. + * Creates a new GetCellsAliasesResponse instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static - * @param {vtctldata.IExecuteFetchAsDBAResponse=} [properties] Properties to set - * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse instance + * @param {vtctldata.IGetCellsAliasesResponse=} [properties] Properties to set + * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse instance */ - ExecuteFetchAsDBAResponse.create = function create(properties) { - return new ExecuteFetchAsDBAResponse(properties); + GetCellsAliasesResponse.create = function create(properties) { + return new GetCellsAliasesResponse(properties); }; /** - * Encodes the specified ExecuteFetchAsDBAResponse message. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. + * Encodes the specified GetCellsAliasesResponse message. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static - * @param {vtctldata.IExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse message or plain object to encode + * @param {vtctldata.IGetCellsAliasesResponse} message GetCellsAliasesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsDBAResponse.encode = function encode(message, writer) { + GetCellsAliasesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.result != null && Object.hasOwnProperty.call(message, "result")) - $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.aliases != null && Object.hasOwnProperty.call(message, "aliases")) + for (let keys = Object.keys(message.aliases), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.topodata.CellsAlias.encode(message.aliases[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified ExecuteFetchAsDBAResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteFetchAsDBAResponse.verify|verify} messages. + * Encodes the specified GetCellsAliasesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static - * @param {vtctldata.IExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse message or plain object to encode + * @param {vtctldata.IGetCellsAliasesResponse} message GetCellsAliasesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteFetchAsDBAResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetCellsAliasesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer. + * Decodes a GetCellsAliasesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse + * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsDBAResponse.decode = function decode(reader, length) { + GetCellsAliasesResponse.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.ExecuteFetchAsDBAResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetCellsAliasesResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.result = $root.query.QueryResult.decode(reader, reader.uint32()); + if (message.aliases === $util.emptyObject) + message.aliases = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.topodata.CellsAlias.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.aliases[key] = value; break; } default: @@ -111881,128 +117438,141 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteFetchAsDBAResponse message from the specified reader or buffer, length delimited. + * Decodes a GetCellsAliasesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse + * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteFetchAsDBAResponse.decodeDelimited = function decodeDelimited(reader) { + GetCellsAliasesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteFetchAsDBAResponse message. + * Verifies a GetCellsAliasesResponse message. * @function verify - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteFetchAsDBAResponse.verify = function verify(message) { + GetCellsAliasesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.result != null && message.hasOwnProperty("result")) { - let error = $root.query.QueryResult.verify(message.result); - if (error) - return "result." + error; + if (message.aliases != null && message.hasOwnProperty("aliases")) { + if (!$util.isObject(message.aliases)) + return "aliases: object expected"; + let key = Object.keys(message.aliases); + for (let i = 0; i < key.length; ++i) { + let error = $root.topodata.CellsAlias.verify(message.aliases[key[i]]); + if (error) + return "aliases." + error; + } } return null; }; /** - * Creates an ExecuteFetchAsDBAResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetCellsAliasesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteFetchAsDBAResponse} ExecuteFetchAsDBAResponse + * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse */ - ExecuteFetchAsDBAResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteFetchAsDBAResponse) + GetCellsAliasesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetCellsAliasesResponse) return object; - let message = new $root.vtctldata.ExecuteFetchAsDBAResponse(); - if (object.result != null) { - if (typeof object.result !== "object") - throw TypeError(".vtctldata.ExecuteFetchAsDBAResponse.result: object expected"); - message.result = $root.query.QueryResult.fromObject(object.result); + let message = new $root.vtctldata.GetCellsAliasesResponse(); + if (object.aliases) { + if (typeof object.aliases !== "object") + throw TypeError(".vtctldata.GetCellsAliasesResponse.aliases: object expected"); + message.aliases = {}; + for (let keys = Object.keys(object.aliases), i = 0; i < keys.length; ++i) { + if (typeof object.aliases[keys[i]] !== "object") + throw TypeError(".vtctldata.GetCellsAliasesResponse.aliases: object expected"); + message.aliases[keys[i]] = $root.topodata.CellsAlias.fromObject(object.aliases[keys[i]]); + } } return message; }; /** - * Creates a plain object from an ExecuteFetchAsDBAResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetCellsAliasesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static - * @param {vtctldata.ExecuteFetchAsDBAResponse} message ExecuteFetchAsDBAResponse + * @param {vtctldata.GetCellsAliasesResponse} message GetCellsAliasesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteFetchAsDBAResponse.toObject = function toObject(message, options) { + GetCellsAliasesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.result = null; - if (message.result != null && message.hasOwnProperty("result")) - object.result = $root.query.QueryResult.toObject(message.result, options); + if (options.objects || options.defaults) + object.aliases = {}; + let keys2; + if (message.aliases && (keys2 = Object.keys(message.aliases)).length) { + object.aliases = {}; + for (let j = 0; j < keys2.length; ++j) + object.aliases[keys2[j]] = $root.topodata.CellsAlias.toObject(message.aliases[keys2[j]], options); + } return object; }; /** - * Converts this ExecuteFetchAsDBAResponse to JSON. + * Converts this GetCellsAliasesResponse to JSON. * @function toJSON - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @instance * @returns {Object.} JSON object */ - ExecuteFetchAsDBAResponse.prototype.toJSON = function toJSON() { + GetCellsAliasesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteFetchAsDBAResponse + * Gets the default type url for GetCellsAliasesResponse * @function getTypeUrl - * @memberof vtctldata.ExecuteFetchAsDBAResponse + * @memberof vtctldata.GetCellsAliasesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteFetchAsDBAResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetCellsAliasesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteFetchAsDBAResponse"; + return typeUrlPrefix + "/vtctldata.GetCellsAliasesResponse"; }; - return ExecuteFetchAsDBAResponse; + return GetCellsAliasesResponse; })(); - vtctldata.ExecuteHookRequest = (function() { + vtctldata.GetFullStatusRequest = (function() { /** - * Properties of an ExecuteHookRequest. + * Properties of a GetFullStatusRequest. * @memberof vtctldata - * @interface IExecuteHookRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] ExecuteHookRequest tablet_alias - * @property {tabletmanagerdata.IExecuteHookRequest|null} [tablet_hook_request] ExecuteHookRequest tablet_hook_request + * @interface IGetFullStatusRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] GetFullStatusRequest tablet_alias */ /** - * Constructs a new ExecuteHookRequest. + * Constructs a new GetFullStatusRequest. * @memberof vtctldata - * @classdesc Represents an ExecuteHookRequest. - * @implements IExecuteHookRequest + * @classdesc Represents a GetFullStatusRequest. + * @implements IGetFullStatusRequest * @constructor - * @param {vtctldata.IExecuteHookRequest=} [properties] Properties to set + * @param {vtctldata.IGetFullStatusRequest=} [properties] Properties to set */ - function ExecuteHookRequest(properties) { + function GetFullStatusRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -112010,80 +117580,70 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteHookRequest tablet_alias. + * GetFullStatusRequest tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ExecuteHookRequest - * @instance - */ - ExecuteHookRequest.prototype.tablet_alias = null; - - /** - * ExecuteHookRequest tablet_hook_request. - * @member {tabletmanagerdata.IExecuteHookRequest|null|undefined} tablet_hook_request - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @instance */ - ExecuteHookRequest.prototype.tablet_hook_request = null; + GetFullStatusRequest.prototype.tablet_alias = null; /** - * Creates a new ExecuteHookRequest instance using the specified properties. + * Creates a new GetFullStatusRequest instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static - * @param {vtctldata.IExecuteHookRequest=} [properties] Properties to set - * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest instance + * @param {vtctldata.IGetFullStatusRequest=} [properties] Properties to set + * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest instance */ - ExecuteHookRequest.create = function create(properties) { - return new ExecuteHookRequest(properties); + GetFullStatusRequest.create = function create(properties) { + return new GetFullStatusRequest(properties); }; /** - * Encodes the specified ExecuteHookRequest message. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. + * Encodes the specified GetFullStatusRequest message. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static - * @param {vtctldata.IExecuteHookRequest} message ExecuteHookRequest message or plain object to encode + * @param {vtctldata.IGetFullStatusRequest} message GetFullStatusRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteHookRequest.encode = function encode(message, writer) { + GetFullStatusRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.tablet_hook_request != null && Object.hasOwnProperty.call(message, "tablet_hook_request")) - $root.tabletmanagerdata.ExecuteHookRequest.encode(message.tablet_hook_request, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified ExecuteHookRequest message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookRequest.verify|verify} messages. + * Encodes the specified GetFullStatusRequest message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static - * @param {vtctldata.IExecuteHookRequest} message ExecuteHookRequest message or plain object to encode + * @param {vtctldata.IGetFullStatusRequest} message GetFullStatusRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteHookRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetFullStatusRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteHookRequest message from the specified reader or buffer. + * Decodes a GetFullStatusRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest + * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteHookRequest.decode = function decode(reader, length) { + GetFullStatusRequest.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.ExecuteHookRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetFullStatusRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -112091,10 +117651,6 @@ export const vtctldata = $root.vtctldata = (() => { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } - case 2: { - message.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -112104,30 +117660,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteHookRequest message from the specified reader or buffer, length delimited. + * Decodes a GetFullStatusRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest + * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteHookRequest.decodeDelimited = function decodeDelimited(reader) { + GetFullStatusRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteHookRequest message. + * Verifies a GetFullStatusRequest message. * @function verify - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteHookRequest.verify = function verify(message) { + GetFullStatusRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { @@ -112135,110 +117691,96 @@ export const vtctldata = $root.vtctldata = (() => { if (error) return "tablet_alias." + error; } - if (message.tablet_hook_request != null && message.hasOwnProperty("tablet_hook_request")) { - let error = $root.tabletmanagerdata.ExecuteHookRequest.verify(message.tablet_hook_request); - if (error) - return "tablet_hook_request." + error; - } return null; }; /** - * Creates an ExecuteHookRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetFullStatusRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteHookRequest} ExecuteHookRequest + * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest */ - ExecuteHookRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteHookRequest) + GetFullStatusRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetFullStatusRequest) return object; - let message = new $root.vtctldata.ExecuteHookRequest(); + let message = new $root.vtctldata.GetFullStatusRequest(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ExecuteHookRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.GetFullStatusRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.tablet_hook_request != null) { - if (typeof object.tablet_hook_request !== "object") - throw TypeError(".vtctldata.ExecuteHookRequest.tablet_hook_request: object expected"); - message.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.fromObject(object.tablet_hook_request); - } return message; }; /** - * Creates a plain object from an ExecuteHookRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetFullStatusRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static - * @param {vtctldata.ExecuteHookRequest} message ExecuteHookRequest + * @param {vtctldata.GetFullStatusRequest} message GetFullStatusRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteHookRequest.toObject = function toObject(message, options) { + GetFullStatusRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { + if (options.defaults) object.tablet_alias = null; - object.tablet_hook_request = null; - } if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.tablet_hook_request != null && message.hasOwnProperty("tablet_hook_request")) - object.tablet_hook_request = $root.tabletmanagerdata.ExecuteHookRequest.toObject(message.tablet_hook_request, options); return object; }; /** - * Converts this ExecuteHookRequest to JSON. + * Converts this GetFullStatusRequest to JSON. * @function toJSON - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @instance * @returns {Object.} JSON object */ - ExecuteHookRequest.prototype.toJSON = function toJSON() { + GetFullStatusRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteHookRequest + * Gets the default type url for GetFullStatusRequest * @function getTypeUrl - * @memberof vtctldata.ExecuteHookRequest + * @memberof vtctldata.GetFullStatusRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteHookRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetFullStatusRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteHookRequest"; + return typeUrlPrefix + "/vtctldata.GetFullStatusRequest"; }; - return ExecuteHookRequest; + return GetFullStatusRequest; })(); - vtctldata.ExecuteHookResponse = (function() { + vtctldata.GetFullStatusResponse = (function() { /** - * Properties of an ExecuteHookResponse. + * Properties of a GetFullStatusResponse. * @memberof vtctldata - * @interface IExecuteHookResponse - * @property {tabletmanagerdata.IExecuteHookResponse|null} [hook_result] ExecuteHookResponse hook_result + * @interface IGetFullStatusResponse + * @property {replicationdata.IFullStatus|null} [status] GetFullStatusResponse status */ /** - * Constructs a new ExecuteHookResponse. + * Constructs a new GetFullStatusResponse. * @memberof vtctldata - * @classdesc Represents an ExecuteHookResponse. - * @implements IExecuteHookResponse + * @classdesc Represents a GetFullStatusResponse. + * @implements IGetFullStatusResponse * @constructor - * @param {vtctldata.IExecuteHookResponse=} [properties] Properties to set + * @param {vtctldata.IGetFullStatusResponse=} [properties] Properties to set */ - function ExecuteHookResponse(properties) { + function GetFullStatusResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -112246,75 +117788,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ExecuteHookResponse hook_result. - * @member {tabletmanagerdata.IExecuteHookResponse|null|undefined} hook_result - * @memberof vtctldata.ExecuteHookResponse + * GetFullStatusResponse status. + * @member {replicationdata.IFullStatus|null|undefined} status + * @memberof vtctldata.GetFullStatusResponse * @instance */ - ExecuteHookResponse.prototype.hook_result = null; + GetFullStatusResponse.prototype.status = null; /** - * Creates a new ExecuteHookResponse instance using the specified properties. + * Creates a new GetFullStatusResponse instance using the specified properties. * @function create - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static - * @param {vtctldata.IExecuteHookResponse=} [properties] Properties to set - * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse instance + * @param {vtctldata.IGetFullStatusResponse=} [properties] Properties to set + * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse instance */ - ExecuteHookResponse.create = function create(properties) { - return new ExecuteHookResponse(properties); + GetFullStatusResponse.create = function create(properties) { + return new GetFullStatusResponse(properties); }; /** - * Encodes the specified ExecuteHookResponse message. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. + * Encodes the specified GetFullStatusResponse message. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static - * @param {vtctldata.IExecuteHookResponse} message ExecuteHookResponse message or plain object to encode + * @param {vtctldata.IGetFullStatusResponse} message GetFullStatusResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteHookResponse.encode = function encode(message, writer) { + GetFullStatusResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.hook_result != null && Object.hasOwnProperty.call(message, "hook_result")) - $root.tabletmanagerdata.ExecuteHookResponse.encode(message.hook_result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.status != null && Object.hasOwnProperty.call(message, "status")) + $root.replicationdata.FullStatus.encode(message.status, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ExecuteHookResponse message, length delimited. Does not implicitly {@link vtctldata.ExecuteHookResponse.verify|verify} messages. + * Encodes the specified GetFullStatusResponse message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static - * @param {vtctldata.IExecuteHookResponse} message ExecuteHookResponse message or plain object to encode + * @param {vtctldata.IGetFullStatusResponse} message GetFullStatusResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ExecuteHookResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetFullStatusResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an ExecuteHookResponse message from the specified reader or buffer. + * Decodes a GetFullStatusResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse + * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteHookResponse.decode = function decode(reader, length) { + GetFullStatusResponse.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.ExecuteHookResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetFullStatusResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.decode(reader, reader.uint32()); + message.status = $root.replicationdata.FullStatus.decode(reader, reader.uint32()); break; } default: @@ -112326,127 +117868,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an ExecuteHookResponse message from the specified reader or buffer, length delimited. + * Decodes a GetFullStatusResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse + * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ExecuteHookResponse.decodeDelimited = function decodeDelimited(reader) { + GetFullStatusResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an ExecuteHookResponse message. + * Verifies a GetFullStatusResponse message. * @function verify - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ExecuteHookResponse.verify = function verify(message) { + GetFullStatusResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.hook_result != null && message.hasOwnProperty("hook_result")) { - let error = $root.tabletmanagerdata.ExecuteHookResponse.verify(message.hook_result); + if (message.status != null && message.hasOwnProperty("status")) { + let error = $root.replicationdata.FullStatus.verify(message.status); if (error) - return "hook_result." + error; + return "status." + error; } return null; }; /** - * Creates an ExecuteHookResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetFullStatusResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ExecuteHookResponse} ExecuteHookResponse + * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse */ - ExecuteHookResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ExecuteHookResponse) - return object; - let message = new $root.vtctldata.ExecuteHookResponse(); - if (object.hook_result != null) { - if (typeof object.hook_result !== "object") - throw TypeError(".vtctldata.ExecuteHookResponse.hook_result: object expected"); - message.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.fromObject(object.hook_result); + GetFullStatusResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetFullStatusResponse) + return object; + let message = new $root.vtctldata.GetFullStatusResponse(); + if (object.status != null) { + if (typeof object.status !== "object") + throw TypeError(".vtctldata.GetFullStatusResponse.status: object expected"); + message.status = $root.replicationdata.FullStatus.fromObject(object.status); } return message; }; /** - * Creates a plain object from an ExecuteHookResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetFullStatusResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static - * @param {vtctldata.ExecuteHookResponse} message ExecuteHookResponse + * @param {vtctldata.GetFullStatusResponse} message GetFullStatusResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ExecuteHookResponse.toObject = function toObject(message, options) { + GetFullStatusResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.hook_result = null; - if (message.hook_result != null && message.hasOwnProperty("hook_result")) - object.hook_result = $root.tabletmanagerdata.ExecuteHookResponse.toObject(message.hook_result, options); + object.status = null; + if (message.status != null && message.hasOwnProperty("status")) + object.status = $root.replicationdata.FullStatus.toObject(message.status, options); return object; }; /** - * Converts this ExecuteHookResponse to JSON. + * Converts this GetFullStatusResponse to JSON. * @function toJSON - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @instance * @returns {Object.} JSON object */ - ExecuteHookResponse.prototype.toJSON = function toJSON() { + GetFullStatusResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ExecuteHookResponse + * Gets the default type url for GetFullStatusResponse * @function getTypeUrl - * @memberof vtctldata.ExecuteHookResponse + * @memberof vtctldata.GetFullStatusResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ExecuteHookResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetFullStatusResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ExecuteHookResponse"; + return typeUrlPrefix + "/vtctldata.GetFullStatusResponse"; }; - return ExecuteHookResponse; + return GetFullStatusResponse; })(); - vtctldata.FindAllShardsInKeyspaceRequest = (function() { + vtctldata.GetKeyspacesRequest = (function() { /** - * Properties of a FindAllShardsInKeyspaceRequest. + * Properties of a GetKeyspacesRequest. * @memberof vtctldata - * @interface IFindAllShardsInKeyspaceRequest - * @property {string|null} [keyspace] FindAllShardsInKeyspaceRequest keyspace + * @interface IGetKeyspacesRequest */ /** - * Constructs a new FindAllShardsInKeyspaceRequest. + * Constructs a new GetKeyspacesRequest. * @memberof vtctldata - * @classdesc Represents a FindAllShardsInKeyspaceRequest. - * @implements IFindAllShardsInKeyspaceRequest + * @classdesc Represents a GetKeyspacesRequest. + * @implements IGetKeyspacesRequest * @constructor - * @param {vtctldata.IFindAllShardsInKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IGetKeyspacesRequest=} [properties] Properties to set */ - function FindAllShardsInKeyspaceRequest(properties) { + function GetKeyspacesRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -112454,77 +117995,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * FindAllShardsInKeyspaceRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.FindAllShardsInKeyspaceRequest - * @instance - */ - FindAllShardsInKeyspaceRequest.prototype.keyspace = ""; - - /** - * Creates a new FindAllShardsInKeyspaceRequest instance using the specified properties. + * Creates a new GetKeyspacesRequest instance using the specified properties. * @function create - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static - * @param {vtctldata.IFindAllShardsInKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest instance + * @param {vtctldata.IGetKeyspacesRequest=} [properties] Properties to set + * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest instance */ - FindAllShardsInKeyspaceRequest.create = function create(properties) { - return new FindAllShardsInKeyspaceRequest(properties); + GetKeyspacesRequest.create = function create(properties) { + return new GetKeyspacesRequest(properties); }; /** - * Encodes the specified FindAllShardsInKeyspaceRequest message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. + * Encodes the specified GetKeyspacesRequest message. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static - * @param {vtctldata.IFindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest message or plain object to encode + * @param {vtctldata.IGetKeyspacesRequest} message GetKeyspacesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FindAllShardsInKeyspaceRequest.encode = function encode(message, writer) { + GetKeyspacesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); return writer; }; /** - * Encodes the specified FindAllShardsInKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceRequest.verify|verify} messages. + * Encodes the specified GetKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static - * @param {vtctldata.IFindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest message or plain object to encode + * @param {vtctldata.IGetKeyspacesRequest} message GetKeyspacesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FindAllShardsInKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetKeyspacesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer. + * Decodes a GetKeyspacesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest + * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FindAllShardsInKeyspaceRequest.decode = function decode(reader, length) { + GetKeyspacesRequest.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.FindAllShardsInKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetKeyspacesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -112534,123 +118061,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a FindAllShardsInKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspacesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest + * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FindAllShardsInKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + GetKeyspacesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a FindAllShardsInKeyspaceRequest message. + * Verifies a GetKeyspacesRequest message. * @function verify - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - FindAllShardsInKeyspaceRequest.verify = function verify(message) { + GetKeyspacesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; return null; }; /** - * Creates a FindAllShardsInKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspacesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.FindAllShardsInKeyspaceRequest} FindAllShardsInKeyspaceRequest + * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest */ - FindAllShardsInKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.FindAllShardsInKeyspaceRequest) + GetKeyspacesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspacesRequest) return object; - let message = new $root.vtctldata.FindAllShardsInKeyspaceRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - return message; + return new $root.vtctldata.GetKeyspacesRequest(); }; /** - * Creates a plain object from a FindAllShardsInKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetKeyspacesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static - * @param {vtctldata.FindAllShardsInKeyspaceRequest} message FindAllShardsInKeyspaceRequest + * @param {vtctldata.GetKeyspacesRequest} message GetKeyspacesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - FindAllShardsInKeyspaceRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.keyspace = ""; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - return object; + GetKeyspacesRequest.toObject = function toObject() { + return {}; }; /** - * Converts this FindAllShardsInKeyspaceRequest to JSON. + * Converts this GetKeyspacesRequest to JSON. * @function toJSON - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @instance * @returns {Object.} JSON object */ - FindAllShardsInKeyspaceRequest.prototype.toJSON = function toJSON() { + GetKeyspacesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for FindAllShardsInKeyspaceRequest + * Gets the default type url for GetKeyspacesRequest * @function getTypeUrl - * @memberof vtctldata.FindAllShardsInKeyspaceRequest + * @memberof vtctldata.GetKeyspacesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - FindAllShardsInKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetKeyspacesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.FindAllShardsInKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.GetKeyspacesRequest"; }; - return FindAllShardsInKeyspaceRequest; + return GetKeyspacesRequest; })(); - vtctldata.FindAllShardsInKeyspaceResponse = (function() { + vtctldata.GetKeyspacesResponse = (function() { /** - * Properties of a FindAllShardsInKeyspaceResponse. + * Properties of a GetKeyspacesResponse. * @memberof vtctldata - * @interface IFindAllShardsInKeyspaceResponse - * @property {Object.|null} [shards] FindAllShardsInKeyspaceResponse shards + * @interface IGetKeyspacesResponse + * @property {Array.|null} [keyspaces] GetKeyspacesResponse keyspaces */ /** - * Constructs a new FindAllShardsInKeyspaceResponse. + * Constructs a new GetKeyspacesResponse. * @memberof vtctldata - * @classdesc Represents a FindAllShardsInKeyspaceResponse. - * @implements IFindAllShardsInKeyspaceResponse + * @classdesc Represents a GetKeyspacesResponse. + * @implements IGetKeyspacesResponse * @constructor - * @param {vtctldata.IFindAllShardsInKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IGetKeyspacesResponse=} [properties] Properties to set */ - function FindAllShardsInKeyspaceResponse(properties) { - this.shards = {}; + function GetKeyspacesResponse(properties) { + this.keyspaces = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -112658,97 +118172,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * FindAllShardsInKeyspaceResponse shards. - * @member {Object.} shards - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * GetKeyspacesResponse keyspaces. + * @member {Array.} keyspaces + * @memberof vtctldata.GetKeyspacesResponse * @instance */ - FindAllShardsInKeyspaceResponse.prototype.shards = $util.emptyObject; + GetKeyspacesResponse.prototype.keyspaces = $util.emptyArray; /** - * Creates a new FindAllShardsInKeyspaceResponse instance using the specified properties. + * Creates a new GetKeyspacesResponse instance using the specified properties. * @function create - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static - * @param {vtctldata.IFindAllShardsInKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse instance + * @param {vtctldata.IGetKeyspacesResponse=} [properties] Properties to set + * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse instance */ - FindAllShardsInKeyspaceResponse.create = function create(properties) { - return new FindAllShardsInKeyspaceResponse(properties); + GetKeyspacesResponse.create = function create(properties) { + return new GetKeyspacesResponse(properties); }; /** - * Encodes the specified FindAllShardsInKeyspaceResponse message. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. + * Encodes the specified GetKeyspacesResponse message. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static - * @param {vtctldata.IFindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse message or plain object to encode + * @param {vtctldata.IGetKeyspacesResponse} message GetKeyspacesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FindAllShardsInKeyspaceResponse.encode = function encode(message, writer) { + GetKeyspacesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shards != null && Object.hasOwnProperty.call(message, "shards")) - for (let keys = Object.keys(message.shards), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.Shard.encode(message.shards[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.keyspaces != null && message.keyspaces.length) + for (let i = 0; i < message.keyspaces.length; ++i) + $root.vtctldata.Keyspace.encode(message.keyspaces[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified FindAllShardsInKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.FindAllShardsInKeyspaceResponse.verify|verify} messages. + * Encodes the specified GetKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static - * @param {vtctldata.IFindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse message or plain object to encode + * @param {vtctldata.IGetKeyspacesResponse} message GetKeyspacesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - FindAllShardsInKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetKeyspacesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer. + * Decodes a GetKeyspacesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse + * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FindAllShardsInKeyspaceResponse.decode = function decode(reader, length) { + GetKeyspacesResponse.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.FindAllShardsInKeyspaceResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetKeyspacesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.shards === $util.emptyObject) - message.shards = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vtctldata.Shard.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.shards[key] = value; + if (!(message.keyspaces && message.keyspaces.length)) + message.keyspaces = []; + message.keyspaces.push($root.vtctldata.Keyspace.decode(reader, reader.uint32())); break; } default: @@ -112760,145 +118255,139 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a FindAllShardsInKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspacesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse + * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - FindAllShardsInKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + GetKeyspacesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a FindAllShardsInKeyspaceResponse message. + * Verifies a GetKeyspacesResponse message. * @function verify - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - FindAllShardsInKeyspaceResponse.verify = function verify(message) { + GetKeyspacesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!$util.isObject(message.shards)) - return "shards: object expected"; - let key = Object.keys(message.shards); - for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.Shard.verify(message.shards[key[i]]); + if (message.keyspaces != null && message.hasOwnProperty("keyspaces")) { + if (!Array.isArray(message.keyspaces)) + return "keyspaces: array expected"; + for (let i = 0; i < message.keyspaces.length; ++i) { + let error = $root.vtctldata.Keyspace.verify(message.keyspaces[i]); if (error) - return "shards." + error; + return "keyspaces." + error; } } return null; }; /** - * Creates a FindAllShardsInKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspacesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.FindAllShardsInKeyspaceResponse} FindAllShardsInKeyspaceResponse + * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse */ - FindAllShardsInKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.FindAllShardsInKeyspaceResponse) + GetKeyspacesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspacesResponse) return object; - let message = new $root.vtctldata.FindAllShardsInKeyspaceResponse(); - if (object.shards) { - if (typeof object.shards !== "object") - throw TypeError(".vtctldata.FindAllShardsInKeyspaceResponse.shards: object expected"); - message.shards = {}; - for (let keys = Object.keys(object.shards), i = 0; i < keys.length; ++i) { - if (typeof object.shards[keys[i]] !== "object") - throw TypeError(".vtctldata.FindAllShardsInKeyspaceResponse.shards: object expected"); - message.shards[keys[i]] = $root.vtctldata.Shard.fromObject(object.shards[keys[i]]); + let message = new $root.vtctldata.GetKeyspacesResponse(); + if (object.keyspaces) { + if (!Array.isArray(object.keyspaces)) + throw TypeError(".vtctldata.GetKeyspacesResponse.keyspaces: array expected"); + message.keyspaces = []; + for (let i = 0; i < object.keyspaces.length; ++i) { + if (typeof object.keyspaces[i] !== "object") + throw TypeError(".vtctldata.GetKeyspacesResponse.keyspaces: object expected"); + message.keyspaces[i] = $root.vtctldata.Keyspace.fromObject(object.keyspaces[i]); } } return message; }; /** - * Creates a plain object from a FindAllShardsInKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetKeyspacesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static - * @param {vtctldata.FindAllShardsInKeyspaceResponse} message FindAllShardsInKeyspaceResponse + * @param {vtctldata.GetKeyspacesResponse} message GetKeyspacesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - FindAllShardsInKeyspaceResponse.toObject = function toObject(message, options) { + GetKeyspacesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.shards = {}; - let keys2; - if (message.shards && (keys2 = Object.keys(message.shards)).length) { - object.shards = {}; - for (let j = 0; j < keys2.length; ++j) - object.shards[keys2[j]] = $root.vtctldata.Shard.toObject(message.shards[keys2[j]], options); + if (options.arrays || options.defaults) + object.keyspaces = []; + if (message.keyspaces && message.keyspaces.length) { + object.keyspaces = []; + for (let j = 0; j < message.keyspaces.length; ++j) + object.keyspaces[j] = $root.vtctldata.Keyspace.toObject(message.keyspaces[j], options); } return object; }; /** - * Converts this FindAllShardsInKeyspaceResponse to JSON. + * Converts this GetKeyspacesResponse to JSON. * @function toJSON - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @instance * @returns {Object.} JSON object */ - FindAllShardsInKeyspaceResponse.prototype.toJSON = function toJSON() { + GetKeyspacesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for FindAllShardsInKeyspaceResponse + * Gets the default type url for GetKeyspacesResponse * @function getTypeUrl - * @memberof vtctldata.FindAllShardsInKeyspaceResponse + * @memberof vtctldata.GetKeyspacesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - FindAllShardsInKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetKeyspacesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.FindAllShardsInKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.GetKeyspacesResponse"; }; - return FindAllShardsInKeyspaceResponse; + return GetKeyspacesResponse; })(); - vtctldata.GetBackupsRequest = (function() { + vtctldata.GetKeyspaceRequest = (function() { /** - * Properties of a GetBackupsRequest. + * Properties of a GetKeyspaceRequest. * @memberof vtctldata - * @interface IGetBackupsRequest - * @property {string|null} [keyspace] GetBackupsRequest keyspace - * @property {string|null} [shard] GetBackupsRequest shard - * @property {number|null} [limit] GetBackupsRequest limit - * @property {boolean|null} [detailed] GetBackupsRequest detailed - * @property {number|null} [detailed_limit] GetBackupsRequest detailed_limit + * @interface IGetKeyspaceRequest + * @property {string|null} [keyspace] GetKeyspaceRequest keyspace */ /** - * Constructs a new GetBackupsRequest. + * Constructs a new GetKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a GetBackupsRequest. - * @implements IGetBackupsRequest + * @classdesc Represents a GetKeyspaceRequest. + * @implements IGetKeyspaceRequest * @constructor - * @param {vtctldata.IGetBackupsRequest=} [properties] Properties to set + * @param {vtctldata.IGetKeyspaceRequest=} [properties] Properties to set */ - function GetBackupsRequest(properties) { + function GetKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -112906,110 +118395,70 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetBackupsRequest keyspace. + * GetKeyspaceRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.GetBackupsRequest - * @instance - */ - GetBackupsRequest.prototype.keyspace = ""; - - /** - * GetBackupsRequest shard. - * @member {string} shard - * @memberof vtctldata.GetBackupsRequest - * @instance - */ - GetBackupsRequest.prototype.shard = ""; - - /** - * GetBackupsRequest limit. - * @member {number} limit - * @memberof vtctldata.GetBackupsRequest - * @instance - */ - GetBackupsRequest.prototype.limit = 0; - - /** - * GetBackupsRequest detailed. - * @member {boolean} detailed - * @memberof vtctldata.GetBackupsRequest - * @instance - */ - GetBackupsRequest.prototype.detailed = false; - - /** - * GetBackupsRequest detailed_limit. - * @member {number} detailed_limit - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @instance */ - GetBackupsRequest.prototype.detailed_limit = 0; + GetKeyspaceRequest.prototype.keyspace = ""; /** - * Creates a new GetBackupsRequest instance using the specified properties. + * Creates a new GetKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static - * @param {vtctldata.IGetBackupsRequest=} [properties] Properties to set - * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest instance + * @param {vtctldata.IGetKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest instance */ - GetBackupsRequest.create = function create(properties) { - return new GetBackupsRequest(properties); + GetKeyspaceRequest.create = function create(properties) { + return new GetKeyspaceRequest(properties); }; /** - * Encodes the specified GetBackupsRequest message. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. + * Encodes the specified GetKeyspaceRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static - * @param {vtctldata.IGetBackupsRequest} message GetBackupsRequest message or plain object to encode + * @param {vtctldata.IGetKeyspaceRequest} message GetKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetBackupsRequest.encode = function encode(message, writer) { + GetKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.limit != null && Object.hasOwnProperty.call(message, "limit")) - writer.uint32(/* id 3, wireType 0 =*/24).uint32(message.limit); - if (message.detailed != null && Object.hasOwnProperty.call(message, "detailed")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.detailed); - if (message.detailed_limit != null && Object.hasOwnProperty.call(message, "detailed_limit")) - writer.uint32(/* id 5, wireType 0 =*/40).uint32(message.detailed_limit); return writer; }; /** - * Encodes the specified GetBackupsRequest message, length delimited. Does not implicitly {@link vtctldata.GetBackupsRequest.verify|verify} messages. + * Encodes the specified GetKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static - * @param {vtctldata.IGetBackupsRequest} message GetBackupsRequest message or plain object to encode + * @param {vtctldata.IGetKeyspaceRequest} message GetKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetBackupsRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetBackupsRequest message from the specified reader or buffer. + * Decodes a GetKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest + * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetBackupsRequest.decode = function decode(reader, length) { + GetKeyspaceRequest.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.GetBackupsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -113017,22 +118466,6 @@ export const vtctldata = $root.vtctldata = (() => { message.keyspace = reader.string(); break; } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.limit = reader.uint32(); - break; - } - case 4: { - message.detailed = reader.bool(); - break; - } - case 5: { - message.detailed_limit = reader.uint32(); - break; - } default: reader.skipType(tag & 7); break; @@ -113042,156 +118475,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetBackupsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest + * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetBackupsRequest.decodeDelimited = function decodeDelimited(reader) { + GetKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetBackupsRequest message. + * Verifies a GetKeyspaceRequest message. * @function verify - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetBackupsRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.limit != null && message.hasOwnProperty("limit")) - if (!$util.isInteger(message.limit)) - return "limit: integer expected"; - if (message.detailed != null && message.hasOwnProperty("detailed")) - if (typeof message.detailed !== "boolean") - return "detailed: boolean expected"; - if (message.detailed_limit != null && message.hasOwnProperty("detailed_limit")) - if (!$util.isInteger(message.detailed_limit)) - return "detailed_limit: integer expected"; + GetKeyspaceRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; return null; }; /** - * Creates a GetBackupsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetBackupsRequest} GetBackupsRequest + * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest */ - GetBackupsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetBackupsRequest) + GetKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspaceRequest) return object; - let message = new $root.vtctldata.GetBackupsRequest(); + let message = new $root.vtctldata.GetKeyspaceRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.limit != null) - message.limit = object.limit >>> 0; - if (object.detailed != null) - message.detailed = Boolean(object.detailed); - if (object.detailed_limit != null) - message.detailed_limit = object.detailed_limit >>> 0; return message; }; /** - * Creates a plain object from a GetBackupsRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static - * @param {vtctldata.GetBackupsRequest} message GetBackupsRequest + * @param {vtctldata.GetKeyspaceRequest} message GetKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetBackupsRequest.toObject = function toObject(message, options) { + GetKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { + if (options.defaults) object.keyspace = ""; - object.shard = ""; - object.limit = 0; - object.detailed = false; - object.detailed_limit = 0; - } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.limit != null && message.hasOwnProperty("limit")) - object.limit = message.limit; - if (message.detailed != null && message.hasOwnProperty("detailed")) - object.detailed = message.detailed; - if (message.detailed_limit != null && message.hasOwnProperty("detailed_limit")) - object.detailed_limit = message.detailed_limit; return object; }; /** - * Converts this GetBackupsRequest to JSON. + * Converts this GetKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @instance * @returns {Object.} JSON object */ - GetBackupsRequest.prototype.toJSON = function toJSON() { + GetKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetBackupsRequest + * Gets the default type url for GetKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.GetBackupsRequest + * @memberof vtctldata.GetKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetBackupsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetBackupsRequest"; + return typeUrlPrefix + "/vtctldata.GetKeyspaceRequest"; }; - return GetBackupsRequest; + return GetKeyspaceRequest; })(); - vtctldata.GetBackupsResponse = (function() { + vtctldata.GetKeyspaceResponse = (function() { /** - * Properties of a GetBackupsResponse. + * Properties of a GetKeyspaceResponse. * @memberof vtctldata - * @interface IGetBackupsResponse - * @property {Array.|null} [backups] GetBackupsResponse backups + * @interface IGetKeyspaceResponse + * @property {vtctldata.IKeyspace|null} [keyspace] GetKeyspaceResponse keyspace */ /** - * Constructs a new GetBackupsResponse. + * Constructs a new GetKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a GetBackupsResponse. - * @implements IGetBackupsResponse + * @classdesc Represents a GetKeyspaceResponse. + * @implements IGetKeyspaceResponse * @constructor - * @param {vtctldata.IGetBackupsResponse=} [properties] Properties to set + * @param {vtctldata.IGetKeyspaceResponse=} [properties] Properties to set */ - function GetBackupsResponse(properties) { - this.backups = []; + function GetKeyspaceResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -113199,78 +118598,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetBackupsResponse backups. - * @member {Array.} backups - * @memberof vtctldata.GetBackupsResponse + * GetKeyspaceResponse keyspace. + * @member {vtctldata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.GetKeyspaceResponse * @instance */ - GetBackupsResponse.prototype.backups = $util.emptyArray; + GetKeyspaceResponse.prototype.keyspace = null; /** - * Creates a new GetBackupsResponse instance using the specified properties. + * Creates a new GetKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static - * @param {vtctldata.IGetBackupsResponse=} [properties] Properties to set - * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse instance + * @param {vtctldata.IGetKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse instance */ - GetBackupsResponse.create = function create(properties) { - return new GetBackupsResponse(properties); + GetKeyspaceResponse.create = function create(properties) { + return new GetKeyspaceResponse(properties); }; /** - * Encodes the specified GetBackupsResponse message. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. + * Encodes the specified GetKeyspaceResponse message. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static - * @param {vtctldata.IGetBackupsResponse} message GetBackupsResponse message or plain object to encode + * @param {vtctldata.IGetKeyspaceResponse} message GetKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetBackupsResponse.encode = function encode(message, writer) { + GetKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.backups != null && message.backups.length) - for (let i = 0; i < message.backups.length; ++i) - $root.mysqlctl.BackupInfo.encode(message.backups[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetBackupsResponse message, length delimited. Does not implicitly {@link vtctldata.GetBackupsResponse.verify|verify} messages. + * Encodes the specified GetKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static - * @param {vtctldata.IGetBackupsResponse} message GetBackupsResponse message or plain object to encode + * @param {vtctldata.IGetKeyspaceResponse} message GetKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetBackupsResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetBackupsResponse message from the specified reader or buffer. + * Decodes a GetKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse + * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetBackupsResponse.decode = function decode(reader, length) { + GetKeyspaceResponse.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.GetBackupsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetKeyspaceResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.backups && message.backups.length)) - message.backups = []; - message.backups.push($root.mysqlctl.BackupInfo.decode(reader, reader.uint32())); + message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); break; } default: @@ -113282,139 +118678,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetBackupsResponse message from the specified reader or buffer, length delimited. + * Decodes a GetKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse + * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetBackupsResponse.decodeDelimited = function decodeDelimited(reader) { + GetKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetBackupsResponse message. + * Verifies a GetKeyspaceResponse message. * @function verify - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetBackupsResponse.verify = function verify(message) { + GetKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.backups != null && message.hasOwnProperty("backups")) { - if (!Array.isArray(message.backups)) - return "backups: array expected"; - for (let i = 0; i < message.backups.length; ++i) { - let error = $root.mysqlctl.BackupInfo.verify(message.backups[i]); - if (error) - return "backups." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.vtctldata.Keyspace.verify(message.keyspace); + if (error) + return "keyspace." + error; } return null; }; /** - * Creates a GetBackupsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetBackupsResponse} GetBackupsResponse + * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse */ - GetBackupsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetBackupsResponse) + GetKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspaceResponse) return object; - let message = new $root.vtctldata.GetBackupsResponse(); - if (object.backups) { - if (!Array.isArray(object.backups)) - throw TypeError(".vtctldata.GetBackupsResponse.backups: array expected"); - message.backups = []; - for (let i = 0; i < object.backups.length; ++i) { - if (typeof object.backups[i] !== "object") - throw TypeError(".vtctldata.GetBackupsResponse.backups: object expected"); - message.backups[i] = $root.mysqlctl.BackupInfo.fromObject(object.backups[i]); - } + let message = new $root.vtctldata.GetKeyspaceResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.GetKeyspaceResponse.keyspace: object expected"); + message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); } return message; }; /** - * Creates a plain object from a GetBackupsResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static - * @param {vtctldata.GetBackupsResponse} message GetBackupsResponse + * @param {vtctldata.GetKeyspaceResponse} message GetKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetBackupsResponse.toObject = function toObject(message, options) { + GetKeyspaceResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.backups = []; - if (message.backups && message.backups.length) { - object.backups = []; - for (let j = 0; j < message.backups.length; ++j) - object.backups[j] = $root.mysqlctl.BackupInfo.toObject(message.backups[j], options); - } + if (options.defaults) + object.keyspace = null; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); return object; }; /** - * Converts this GetBackupsResponse to JSON. + * Converts this GetKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @instance * @returns {Object.} JSON object */ - GetBackupsResponse.prototype.toJSON = function toJSON() { + GetKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetBackupsResponse + * Gets the default type url for GetKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.GetBackupsResponse + * @memberof vtctldata.GetKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetBackupsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetBackupsResponse"; + return typeUrlPrefix + "/vtctldata.GetKeyspaceResponse"; }; - return GetBackupsResponse; + return GetKeyspaceResponse; })(); - vtctldata.GetCellInfoRequest = (function() { + vtctldata.GetPermissionsRequest = (function() { /** - * Properties of a GetCellInfoRequest. + * Properties of a GetPermissionsRequest. * @memberof vtctldata - * @interface IGetCellInfoRequest - * @property {string|null} [cell] GetCellInfoRequest cell + * @interface IGetPermissionsRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] GetPermissionsRequest tablet_alias */ /** - * Constructs a new GetCellInfoRequest. + * Constructs a new GetPermissionsRequest. * @memberof vtctldata - * @classdesc Represents a GetCellInfoRequest. - * @implements IGetCellInfoRequest + * @classdesc Represents a GetPermissionsRequest. + * @implements IGetPermissionsRequest * @constructor - * @param {vtctldata.IGetCellInfoRequest=} [properties] Properties to set + * @param {vtctldata.IGetPermissionsRequest=} [properties] Properties to set */ - function GetCellInfoRequest(properties) { + function GetPermissionsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -113422,75 +118806,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetCellInfoRequest cell. - * @member {string} cell - * @memberof vtctldata.GetCellInfoRequest + * GetPermissionsRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.GetPermissionsRequest * @instance */ - GetCellInfoRequest.prototype.cell = ""; + GetPermissionsRequest.prototype.tablet_alias = null; /** - * Creates a new GetCellInfoRequest instance using the specified properties. + * Creates a new GetPermissionsRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static - * @param {vtctldata.IGetCellInfoRequest=} [properties] Properties to set - * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest instance + * @param {vtctldata.IGetPermissionsRequest=} [properties] Properties to set + * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest instance */ - GetCellInfoRequest.create = function create(properties) { - return new GetCellInfoRequest(properties); + GetPermissionsRequest.create = function create(properties) { + return new GetPermissionsRequest(properties); }; /** - * Encodes the specified GetCellInfoRequest message. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. + * Encodes the specified GetPermissionsRequest message. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static - * @param {vtctldata.IGetCellInfoRequest} message GetCellInfoRequest message or plain object to encode + * @param {vtctldata.IGetPermissionsRequest} message GetPermissionsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoRequest.encode = function encode(message, writer) { + GetPermissionsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoRequest.verify|verify} messages. + * Encodes the specified GetPermissionsRequest message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static - * @param {vtctldata.IGetCellInfoRequest} message GetCellInfoRequest message or plain object to encode + * @param {vtctldata.IGetPermissionsRequest} message GetPermissionsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetPermissionsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellInfoRequest message from the specified reader or buffer. + * Decodes a GetPermissionsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest + * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoRequest.decode = function decode(reader, length) { + GetPermissionsRequest.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.GetCellInfoRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetPermissionsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.cell = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -113502,122 +118886,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a GetPermissionsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest + * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { + GetPermissionsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellInfoRequest message. + * Verifies a GetPermissionsRequest message. * @function verify - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellInfoRequest.verify = function verify(message) { + GetPermissionsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a GetCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetPermissionsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellInfoRequest} GetCellInfoRequest + * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest */ - GetCellInfoRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellInfoRequest) + GetPermissionsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetPermissionsRequest) return object; - let message = new $root.vtctldata.GetCellInfoRequest(); - if (object.cell != null) - message.cell = String(object.cell); + let message = new $root.vtctldata.GetPermissionsRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.GetPermissionsRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a GetCellInfoRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetPermissionsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static - * @param {vtctldata.GetCellInfoRequest} message GetCellInfoRequest + * @param {vtctldata.GetPermissionsRequest} message GetPermissionsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellInfoRequest.toObject = function toObject(message, options) { + GetPermissionsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.cell = ""; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this GetCellInfoRequest to JSON. + * Converts this GetPermissionsRequest to JSON. * @function toJSON - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @instance * @returns {Object.} JSON object */ - GetCellInfoRequest.prototype.toJSON = function toJSON() { + GetPermissionsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellInfoRequest + * Gets the default type url for GetPermissionsRequest * @function getTypeUrl - * @memberof vtctldata.GetCellInfoRequest + * @memberof vtctldata.GetPermissionsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetPermissionsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellInfoRequest"; + return typeUrlPrefix + "/vtctldata.GetPermissionsRequest"; }; - return GetCellInfoRequest; + return GetPermissionsRequest; })(); - vtctldata.GetCellInfoResponse = (function() { + vtctldata.GetPermissionsResponse = (function() { /** - * Properties of a GetCellInfoResponse. + * Properties of a GetPermissionsResponse. * @memberof vtctldata - * @interface IGetCellInfoResponse - * @property {topodata.ICellInfo|null} [cell_info] GetCellInfoResponse cell_info + * @interface IGetPermissionsResponse + * @property {tabletmanagerdata.IPermissions|null} [permissions] GetPermissionsResponse permissions */ /** - * Constructs a new GetCellInfoResponse. + * Constructs a new GetPermissionsResponse. * @memberof vtctldata - * @classdesc Represents a GetCellInfoResponse. - * @implements IGetCellInfoResponse + * @classdesc Represents a GetPermissionsResponse. + * @implements IGetPermissionsResponse * @constructor - * @param {vtctldata.IGetCellInfoResponse=} [properties] Properties to set + * @param {vtctldata.IGetPermissionsResponse=} [properties] Properties to set */ - function GetCellInfoResponse(properties) { + function GetPermissionsResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -113625,75 +119014,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetCellInfoResponse cell_info. - * @member {topodata.ICellInfo|null|undefined} cell_info - * @memberof vtctldata.GetCellInfoResponse + * GetPermissionsResponse permissions. + * @member {tabletmanagerdata.IPermissions|null|undefined} permissions + * @memberof vtctldata.GetPermissionsResponse * @instance */ - GetCellInfoResponse.prototype.cell_info = null; + GetPermissionsResponse.prototype.permissions = null; /** - * Creates a new GetCellInfoResponse instance using the specified properties. + * Creates a new GetPermissionsResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static - * @param {vtctldata.IGetCellInfoResponse=} [properties] Properties to set - * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse instance + * @param {vtctldata.IGetPermissionsResponse=} [properties] Properties to set + * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse instance */ - GetCellInfoResponse.create = function create(properties) { - return new GetCellInfoResponse(properties); + GetPermissionsResponse.create = function create(properties) { + return new GetPermissionsResponse(properties); }; /** - * Encodes the specified GetCellInfoResponse message. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. + * Encodes the specified GetPermissionsResponse message. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static - * @param {vtctldata.IGetCellInfoResponse} message GetCellInfoResponse message or plain object to encode + * @param {vtctldata.IGetPermissionsResponse} message GetPermissionsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoResponse.encode = function encode(message, writer) { + GetPermissionsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) - $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.permissions != null && Object.hasOwnProperty.call(message, "permissions")) + $root.tabletmanagerdata.Permissions.encode(message.permissions, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoResponse.verify|verify} messages. + * Encodes the specified GetPermissionsResponse message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static - * @param {vtctldata.IGetCellInfoResponse} message GetCellInfoResponse message or plain object to encode + * @param {vtctldata.IGetPermissionsResponse} message GetPermissionsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetPermissionsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellInfoResponse message from the specified reader or buffer. + * Decodes a GetPermissionsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse + * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoResponse.decode = function decode(reader, length) { + GetPermissionsResponse.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.GetCellInfoResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetPermissionsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); + message.permissions = $root.tabletmanagerdata.Permissions.decode(reader, reader.uint32()); break; } default: @@ -113705,126 +119094,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a GetPermissionsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse + * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { + GetPermissionsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellInfoResponse message. + * Verifies a GetPermissionsResponse message. * @function verify - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellInfoResponse.verify = function verify(message) { + GetPermissionsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) { - let error = $root.topodata.CellInfo.verify(message.cell_info); + if (message.permissions != null && message.hasOwnProperty("permissions")) { + let error = $root.tabletmanagerdata.Permissions.verify(message.permissions); if (error) - return "cell_info." + error; + return "permissions." + error; } return null; }; /** - * Creates a GetCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetPermissionsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellInfoResponse} GetCellInfoResponse + * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse */ - GetCellInfoResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellInfoResponse) + GetPermissionsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetPermissionsResponse) return object; - let message = new $root.vtctldata.GetCellInfoResponse(); - if (object.cell_info != null) { - if (typeof object.cell_info !== "object") - throw TypeError(".vtctldata.GetCellInfoResponse.cell_info: object expected"); - message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + let message = new $root.vtctldata.GetPermissionsResponse(); + if (object.permissions != null) { + if (typeof object.permissions !== "object") + throw TypeError(".vtctldata.GetPermissionsResponse.permissions: object expected"); + message.permissions = $root.tabletmanagerdata.Permissions.fromObject(object.permissions); } return message; }; /** - * Creates a plain object from a GetCellInfoResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetPermissionsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static - * @param {vtctldata.GetCellInfoResponse} message GetCellInfoResponse + * @param {vtctldata.GetPermissionsResponse} message GetPermissionsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellInfoResponse.toObject = function toObject(message, options) { + GetPermissionsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.cell_info = null; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) - object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); + object.permissions = null; + if (message.permissions != null && message.hasOwnProperty("permissions")) + object.permissions = $root.tabletmanagerdata.Permissions.toObject(message.permissions, options); return object; }; /** - * Converts this GetCellInfoResponse to JSON. + * Converts this GetPermissionsResponse to JSON. * @function toJSON - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @instance * @returns {Object.} JSON object */ - GetCellInfoResponse.prototype.toJSON = function toJSON() { + GetPermissionsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellInfoResponse + * Gets the default type url for GetPermissionsResponse * @function getTypeUrl - * @memberof vtctldata.GetCellInfoResponse + * @memberof vtctldata.GetPermissionsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetPermissionsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellInfoResponse"; + return typeUrlPrefix + "/vtctldata.GetPermissionsResponse"; }; - return GetCellInfoResponse; + return GetPermissionsResponse; })(); - vtctldata.GetCellInfoNamesRequest = (function() { + vtctldata.GetRoutingRulesRequest = (function() { /** - * Properties of a GetCellInfoNamesRequest. + * Properties of a GetRoutingRulesRequest. * @memberof vtctldata - * @interface IGetCellInfoNamesRequest + * @interface IGetRoutingRulesRequest */ /** - * Constructs a new GetCellInfoNamesRequest. + * Constructs a new GetRoutingRulesRequest. * @memberof vtctldata - * @classdesc Represents a GetCellInfoNamesRequest. - * @implements IGetCellInfoNamesRequest + * @classdesc Represents a GetRoutingRulesRequest. + * @implements IGetRoutingRulesRequest * @constructor - * @param {vtctldata.IGetCellInfoNamesRequest=} [properties] Properties to set + * @param {vtctldata.IGetRoutingRulesRequest=} [properties] Properties to set */ - function GetCellInfoNamesRequest(properties) { + function GetRoutingRulesRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -113832,60 +119221,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new GetCellInfoNamesRequest instance using the specified properties. + * Creates a new GetRoutingRulesRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static - * @param {vtctldata.IGetCellInfoNamesRequest=} [properties] Properties to set - * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest instance + * @param {vtctldata.IGetRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest instance */ - GetCellInfoNamesRequest.create = function create(properties) { - return new GetCellInfoNamesRequest(properties); + GetRoutingRulesRequest.create = function create(properties) { + return new GetRoutingRulesRequest(properties); }; /** - * Encodes the specified GetCellInfoNamesRequest message. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. + * Encodes the specified GetRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static - * @param {vtctldata.IGetCellInfoNamesRequest} message GetCellInfoNamesRequest message or plain object to encode + * @param {vtctldata.IGetRoutingRulesRequest} message GetRoutingRulesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoNamesRequest.encode = function encode(message, writer) { + GetRoutingRulesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified GetCellInfoNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesRequest.verify|verify} messages. + * Encodes the specified GetRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static - * @param {vtctldata.IGetCellInfoNamesRequest} message GetCellInfoNamesRequest message or plain object to encode + * @param {vtctldata.IGetRoutingRulesRequest} message GetRoutingRulesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoNamesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer. + * Decodes a GetRoutingRulesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest + * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoNamesRequest.decode = function decode(reader, length) { + GetRoutingRulesRequest.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.GetCellInfoNamesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetRoutingRulesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -113898,110 +119287,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellInfoNamesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetRoutingRulesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest + * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoNamesRequest.decodeDelimited = function decodeDelimited(reader) { + GetRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellInfoNamesRequest message. + * Verifies a GetRoutingRulesRequest message. * @function verify - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellInfoNamesRequest.verify = function verify(message) { + GetRoutingRulesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a GetCellInfoNamesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellInfoNamesRequest} GetCellInfoNamesRequest + * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest */ - GetCellInfoNamesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellInfoNamesRequest) + GetRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetRoutingRulesRequest) return object; - return new $root.vtctldata.GetCellInfoNamesRequest(); + return new $root.vtctldata.GetRoutingRulesRequest(); }; /** - * Creates a plain object from a GetCellInfoNamesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetRoutingRulesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static - * @param {vtctldata.GetCellInfoNamesRequest} message GetCellInfoNamesRequest + * @param {vtctldata.GetRoutingRulesRequest} message GetRoutingRulesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellInfoNamesRequest.toObject = function toObject() { + GetRoutingRulesRequest.toObject = function toObject() { return {}; }; /** - * Converts this GetCellInfoNamesRequest to JSON. + * Converts this GetRoutingRulesRequest to JSON. * @function toJSON - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @instance * @returns {Object.} JSON object */ - GetCellInfoNamesRequest.prototype.toJSON = function toJSON() { + GetRoutingRulesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellInfoNamesRequest + * Gets the default type url for GetRoutingRulesRequest * @function getTypeUrl - * @memberof vtctldata.GetCellInfoNamesRequest + * @memberof vtctldata.GetRoutingRulesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellInfoNamesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellInfoNamesRequest"; + return typeUrlPrefix + "/vtctldata.GetRoutingRulesRequest"; }; - return GetCellInfoNamesRequest; + return GetRoutingRulesRequest; })(); - vtctldata.GetCellInfoNamesResponse = (function() { + vtctldata.GetRoutingRulesResponse = (function() { /** - * Properties of a GetCellInfoNamesResponse. + * Properties of a GetRoutingRulesResponse. * @memberof vtctldata - * @interface IGetCellInfoNamesResponse - * @property {Array.|null} [names] GetCellInfoNamesResponse names + * @interface IGetRoutingRulesResponse + * @property {vschema.IRoutingRules|null} [routing_rules] GetRoutingRulesResponse routing_rules */ /** - * Constructs a new GetCellInfoNamesResponse. + * Constructs a new GetRoutingRulesResponse. * @memberof vtctldata - * @classdesc Represents a GetCellInfoNamesResponse. - * @implements IGetCellInfoNamesResponse + * @classdesc Represents a GetRoutingRulesResponse. + * @implements IGetRoutingRulesResponse * @constructor - * @param {vtctldata.IGetCellInfoNamesResponse=} [properties] Properties to set + * @param {vtctldata.IGetRoutingRulesResponse=} [properties] Properties to set */ - function GetCellInfoNamesResponse(properties) { - this.names = []; + function GetRoutingRulesResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -114009,78 +119397,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetCellInfoNamesResponse names. - * @member {Array.} names - * @memberof vtctldata.GetCellInfoNamesResponse + * GetRoutingRulesResponse routing_rules. + * @member {vschema.IRoutingRules|null|undefined} routing_rules + * @memberof vtctldata.GetRoutingRulesResponse * @instance */ - GetCellInfoNamesResponse.prototype.names = $util.emptyArray; + GetRoutingRulesResponse.prototype.routing_rules = null; /** - * Creates a new GetCellInfoNamesResponse instance using the specified properties. + * Creates a new GetRoutingRulesResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static - * @param {vtctldata.IGetCellInfoNamesResponse=} [properties] Properties to set - * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse instance + * @param {vtctldata.IGetRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse instance */ - GetCellInfoNamesResponse.create = function create(properties) { - return new GetCellInfoNamesResponse(properties); + GetRoutingRulesResponse.create = function create(properties) { + return new GetRoutingRulesResponse(properties); }; /** - * Encodes the specified GetCellInfoNamesResponse message. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. + * Encodes the specified GetRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static - * @param {vtctldata.IGetCellInfoNamesResponse} message GetCellInfoNamesResponse message or plain object to encode + * @param {vtctldata.IGetRoutingRulesResponse} message GetRoutingRulesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoNamesResponse.encode = function encode(message, writer) { + GetRoutingRulesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.names != null && message.names.length) - for (let i = 0; i < message.names.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.names[i]); + if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) + $root.vschema.RoutingRules.encode(message.routing_rules, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetCellInfoNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellInfoNamesResponse.verify|verify} messages. + * Encodes the specified GetRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static - * @param {vtctldata.IGetCellInfoNamesResponse} message GetCellInfoNamesResponse message or plain object to encode + * @param {vtctldata.IGetRoutingRulesResponse} message GetRoutingRulesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellInfoNamesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer. + * Decodes a GetRoutingRulesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse + * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoNamesResponse.decode = function decode(reader, length) { + GetRoutingRulesResponse.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.GetCellInfoNamesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetRoutingRulesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.names && message.names.length)) - message.names = []; - message.names.push(reader.string()); + message.routing_rules = $root.vschema.RoutingRules.decode(reader, reader.uint32()); break; } default: @@ -114092,133 +119477,135 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellInfoNamesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetRoutingRulesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse + * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellInfoNamesResponse.decodeDelimited = function decodeDelimited(reader) { + GetRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellInfoNamesResponse message. + * Verifies a GetRoutingRulesResponse message. * @function verify - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellInfoNamesResponse.verify = function verify(message) { + GetRoutingRulesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.names != null && message.hasOwnProperty("names")) { - if (!Array.isArray(message.names)) - return "names: array expected"; - for (let i = 0; i < message.names.length; ++i) - if (!$util.isString(message.names[i])) - return "names: string[] expected"; + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { + let error = $root.vschema.RoutingRules.verify(message.routing_rules); + if (error) + return "routing_rules." + error; } return null; }; /** - * Creates a GetCellInfoNamesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellInfoNamesResponse} GetCellInfoNamesResponse + * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse */ - GetCellInfoNamesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellInfoNamesResponse) + GetRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetRoutingRulesResponse) return object; - let message = new $root.vtctldata.GetCellInfoNamesResponse(); - if (object.names) { - if (!Array.isArray(object.names)) - throw TypeError(".vtctldata.GetCellInfoNamesResponse.names: array expected"); - message.names = []; - for (let i = 0; i < object.names.length; ++i) - message.names[i] = String(object.names[i]); + let message = new $root.vtctldata.GetRoutingRulesResponse(); + if (object.routing_rules != null) { + if (typeof object.routing_rules !== "object") + throw TypeError(".vtctldata.GetRoutingRulesResponse.routing_rules: object expected"); + message.routing_rules = $root.vschema.RoutingRules.fromObject(object.routing_rules); } return message; }; /** - * Creates a plain object from a GetCellInfoNamesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetRoutingRulesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static - * @param {vtctldata.GetCellInfoNamesResponse} message GetCellInfoNamesResponse + * @param {vtctldata.GetRoutingRulesResponse} message GetRoutingRulesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellInfoNamesResponse.toObject = function toObject(message, options) { + GetRoutingRulesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.names = []; - if (message.names && message.names.length) { - object.names = []; - for (let j = 0; j < message.names.length; ++j) - object.names[j] = message.names[j]; - } + if (options.defaults) + object.routing_rules = null; + if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) + object.routing_rules = $root.vschema.RoutingRules.toObject(message.routing_rules, options); return object; }; /** - * Converts this GetCellInfoNamesResponse to JSON. + * Converts this GetRoutingRulesResponse to JSON. * @function toJSON - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @instance * @returns {Object.} JSON object */ - GetCellInfoNamesResponse.prototype.toJSON = function toJSON() { + GetRoutingRulesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellInfoNamesResponse + * Gets the default type url for GetRoutingRulesResponse * @function getTypeUrl - * @memberof vtctldata.GetCellInfoNamesResponse + * @memberof vtctldata.GetRoutingRulesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellInfoNamesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellInfoNamesResponse"; + return typeUrlPrefix + "/vtctldata.GetRoutingRulesResponse"; }; - return GetCellInfoNamesResponse; + return GetRoutingRulesResponse; })(); - vtctldata.GetCellsAliasesRequest = (function() { + vtctldata.GetSchemaRequest = (function() { /** - * Properties of a GetCellsAliasesRequest. + * Properties of a GetSchemaRequest. * @memberof vtctldata - * @interface IGetCellsAliasesRequest + * @interface IGetSchemaRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] GetSchemaRequest tablet_alias + * @property {Array.|null} [tables] GetSchemaRequest tables + * @property {Array.|null} [exclude_tables] GetSchemaRequest exclude_tables + * @property {boolean|null} [include_views] GetSchemaRequest include_views + * @property {boolean|null} [table_names_only] GetSchemaRequest table_names_only + * @property {boolean|null} [table_sizes_only] GetSchemaRequest table_sizes_only + * @property {boolean|null} [table_schema_only] GetSchemaRequest table_schema_only */ /** - * Constructs a new GetCellsAliasesRequest. + * Constructs a new GetSchemaRequest. * @memberof vtctldata - * @classdesc Represents a GetCellsAliasesRequest. - * @implements IGetCellsAliasesRequest + * @classdesc Represents a GetSchemaRequest. + * @implements IGetSchemaRequest * @constructor - * @param {vtctldata.IGetCellsAliasesRequest=} [properties] Properties to set + * @param {vtctldata.IGetSchemaRequest=} [properties] Properties to set */ - function GetCellsAliasesRequest(properties) { + function GetSchemaRequest(properties) { + this.tables = []; + this.exclude_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -114226,63 +119613,167 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new GetCellsAliasesRequest instance using the specified properties. + * GetSchemaRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.tablet_alias = null; + + /** + * GetSchemaRequest tables. + * @member {Array.} tables + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.tables = $util.emptyArray; + + /** + * GetSchemaRequest exclude_tables. + * @member {Array.} exclude_tables + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.exclude_tables = $util.emptyArray; + + /** + * GetSchemaRequest include_views. + * @member {boolean} include_views + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.include_views = false; + + /** + * GetSchemaRequest table_names_only. + * @member {boolean} table_names_only + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.table_names_only = false; + + /** + * GetSchemaRequest table_sizes_only. + * @member {boolean} table_sizes_only + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.table_sizes_only = false; + + /** + * GetSchemaRequest table_schema_only. + * @member {boolean} table_schema_only + * @memberof vtctldata.GetSchemaRequest + * @instance + */ + GetSchemaRequest.prototype.table_schema_only = false; + + /** + * Creates a new GetSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static - * @param {vtctldata.IGetCellsAliasesRequest=} [properties] Properties to set - * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest instance + * @param {vtctldata.IGetSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest instance */ - GetCellsAliasesRequest.create = function create(properties) { - return new GetCellsAliasesRequest(properties); + GetSchemaRequest.create = function create(properties) { + return new GetSchemaRequest(properties); }; /** - * Encodes the specified GetCellsAliasesRequest message. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. + * Encodes the specified GetSchemaRequest message. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static - * @param {vtctldata.IGetCellsAliasesRequest} message GetCellsAliasesRequest message or plain object to encode + * @param {vtctldata.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellsAliasesRequest.encode = function encode(message, writer) { + GetSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.tables[i]); + if (message.exclude_tables != null && message.exclude_tables.length) + for (let i = 0; i < message.exclude_tables.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.exclude_tables[i]); + if (message.include_views != null && Object.hasOwnProperty.call(message, "include_views")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_views); + if (message.table_names_only != null && Object.hasOwnProperty.call(message, "table_names_only")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.table_names_only); + if (message.table_sizes_only != null && Object.hasOwnProperty.call(message, "table_sizes_only")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.table_sizes_only); + if (message.table_schema_only != null && Object.hasOwnProperty.call(message, "table_schema_only")) + writer.uint32(/* id 7, wireType 0 =*/56).bool(message.table_schema_only); return writer; }; /** - * Encodes the specified GetCellsAliasesRequest message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesRequest.verify|verify} messages. + * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static - * @param {vtctldata.IGetCellsAliasesRequest} message GetCellsAliasesRequest message or plain object to encode + * @param {vtctldata.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellsAliasesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellsAliasesRequest message from the specified reader or buffer. + * Decodes a GetSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest + * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellsAliasesRequest.decode = function decode(reader, length) { + GetSchemaRequest.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.GetCellsAliasesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); + break; + } + case 3: { + if (!(message.exclude_tables && message.exclude_tables.length)) + message.exclude_tables = []; + message.exclude_tables.push(reader.string()); + break; + } + case 4: { + message.include_views = reader.bool(); + break; + } + case 5: { + message.table_names_only = reader.bool(); + break; + } + case 6: { + message.table_sizes_only = reader.bool(); + break; + } + case 7: { + message.table_schema_only = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -114292,110 +119783,202 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellsAliasesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest + * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellsAliasesRequest.decodeDelimited = function decodeDelimited(reader) { + GetSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellsAliasesRequest message. + * Verifies a GetSchemaRequest message. * @function verify - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellsAliasesRequest.verify = function verify(message) { + GetSchemaRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; + } + if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { + if (!Array.isArray(message.exclude_tables)) + return "exclude_tables: array expected"; + for (let i = 0; i < message.exclude_tables.length; ++i) + if (!$util.isString(message.exclude_tables[i])) + return "exclude_tables: string[] expected"; + } + if (message.include_views != null && message.hasOwnProperty("include_views")) + if (typeof message.include_views !== "boolean") + return "include_views: boolean expected"; + if (message.table_names_only != null && message.hasOwnProperty("table_names_only")) + if (typeof message.table_names_only !== "boolean") + return "table_names_only: boolean expected"; + if (message.table_sizes_only != null && message.hasOwnProperty("table_sizes_only")) + if (typeof message.table_sizes_only !== "boolean") + return "table_sizes_only: boolean expected"; + if (message.table_schema_only != null && message.hasOwnProperty("table_schema_only")) + if (typeof message.table_schema_only !== "boolean") + return "table_schema_only: boolean expected"; return null; }; /** - * Creates a GetCellsAliasesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellsAliasesRequest} GetCellsAliasesRequest + * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest */ - GetCellsAliasesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellsAliasesRequest) + GetSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSchemaRequest) return object; - return new $root.vtctldata.GetCellsAliasesRequest(); + let message = new $root.vtctldata.GetSchemaRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.GetSchemaRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".vtctldata.GetSchemaRequest.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); + } + if (object.exclude_tables) { + if (!Array.isArray(object.exclude_tables)) + throw TypeError(".vtctldata.GetSchemaRequest.exclude_tables: array expected"); + message.exclude_tables = []; + for (let i = 0; i < object.exclude_tables.length; ++i) + message.exclude_tables[i] = String(object.exclude_tables[i]); + } + if (object.include_views != null) + message.include_views = Boolean(object.include_views); + if (object.table_names_only != null) + message.table_names_only = Boolean(object.table_names_only); + if (object.table_sizes_only != null) + message.table_sizes_only = Boolean(object.table_sizes_only); + if (object.table_schema_only != null) + message.table_schema_only = Boolean(object.table_schema_only); + return message; }; /** - * Creates a plain object from a GetCellsAliasesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static - * @param {vtctldata.GetCellsAliasesRequest} message GetCellsAliasesRequest + * @param {vtctldata.GetSchemaRequest} message GetSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellsAliasesRequest.toObject = function toObject() { - return {}; + GetSchemaRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.tables = []; + object.exclude_tables = []; + } + if (options.defaults) { + object.tablet_alias = null; + object.include_views = false; + object.table_names_only = false; + object.table_sizes_only = false; + object.table_schema_only = false; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } + if (message.exclude_tables && message.exclude_tables.length) { + object.exclude_tables = []; + for (let j = 0; j < message.exclude_tables.length; ++j) + object.exclude_tables[j] = message.exclude_tables[j]; + } + if (message.include_views != null && message.hasOwnProperty("include_views")) + object.include_views = message.include_views; + if (message.table_names_only != null && message.hasOwnProperty("table_names_only")) + object.table_names_only = message.table_names_only; + if (message.table_sizes_only != null && message.hasOwnProperty("table_sizes_only")) + object.table_sizes_only = message.table_sizes_only; + if (message.table_schema_only != null && message.hasOwnProperty("table_schema_only")) + object.table_schema_only = message.table_schema_only; + return object; }; /** - * Converts this GetCellsAliasesRequest to JSON. + * Converts this GetSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @instance * @returns {Object.} JSON object */ - GetCellsAliasesRequest.prototype.toJSON = function toJSON() { + GetSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellsAliasesRequest + * Gets the default type url for GetSchemaRequest * @function getTypeUrl - * @memberof vtctldata.GetCellsAliasesRequest + * @memberof vtctldata.GetSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellsAliasesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellsAliasesRequest"; + return typeUrlPrefix + "/vtctldata.GetSchemaRequest"; }; - return GetCellsAliasesRequest; + return GetSchemaRequest; })(); - vtctldata.GetCellsAliasesResponse = (function() { + vtctldata.GetSchemaResponse = (function() { /** - * Properties of a GetCellsAliasesResponse. + * Properties of a GetSchemaResponse. * @memberof vtctldata - * @interface IGetCellsAliasesResponse - * @property {Object.|null} [aliases] GetCellsAliasesResponse aliases + * @interface IGetSchemaResponse + * @property {tabletmanagerdata.ISchemaDefinition|null} [schema] GetSchemaResponse schema */ /** - * Constructs a new GetCellsAliasesResponse. + * Constructs a new GetSchemaResponse. * @memberof vtctldata - * @classdesc Represents a GetCellsAliasesResponse. - * @implements IGetCellsAliasesResponse + * @classdesc Represents a GetSchemaResponse. + * @implements IGetSchemaResponse * @constructor - * @param {vtctldata.IGetCellsAliasesResponse=} [properties] Properties to set + * @param {vtctldata.IGetSchemaResponse=} [properties] Properties to set */ - function GetCellsAliasesResponse(properties) { - this.aliases = {}; + function GetSchemaResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -114403,97 +119986,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetCellsAliasesResponse aliases. - * @member {Object.} aliases - * @memberof vtctldata.GetCellsAliasesResponse + * GetSchemaResponse schema. + * @member {tabletmanagerdata.ISchemaDefinition|null|undefined} schema + * @memberof vtctldata.GetSchemaResponse * @instance */ - GetCellsAliasesResponse.prototype.aliases = $util.emptyObject; + GetSchemaResponse.prototype.schema = null; /** - * Creates a new GetCellsAliasesResponse instance using the specified properties. + * Creates a new GetSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static - * @param {vtctldata.IGetCellsAliasesResponse=} [properties] Properties to set - * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse instance + * @param {vtctldata.IGetSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse instance */ - GetCellsAliasesResponse.create = function create(properties) { - return new GetCellsAliasesResponse(properties); + GetSchemaResponse.create = function create(properties) { + return new GetSchemaResponse(properties); }; /** - * Encodes the specified GetCellsAliasesResponse message. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. + * Encodes the specified GetSchemaResponse message. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static - * @param {vtctldata.IGetCellsAliasesResponse} message GetCellsAliasesResponse message or plain object to encode + * @param {vtctldata.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellsAliasesResponse.encode = function encode(message, writer) { + GetSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.aliases != null && Object.hasOwnProperty.call(message, "aliases")) - for (let keys = Object.keys(message.aliases), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.topodata.CellsAlias.encode(message.aliases[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.schema != null && Object.hasOwnProperty.call(message, "schema")) + $root.tabletmanagerdata.SchemaDefinition.encode(message.schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetCellsAliasesResponse message, length delimited. Does not implicitly {@link vtctldata.GetCellsAliasesResponse.verify|verify} messages. + * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static - * @param {vtctldata.IGetCellsAliasesResponse} message GetCellsAliasesResponse message or plain object to encode + * @param {vtctldata.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetCellsAliasesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetCellsAliasesResponse message from the specified reader or buffer. + * Decodes a GetSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse + * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellsAliasesResponse.decode = function decode(reader, length) { + GetSchemaResponse.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.GetCellsAliasesResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSchemaResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.aliases === $util.emptyObject) - message.aliases = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.topodata.CellsAlias.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.aliases[key] = value; + message.schema = $root.tabletmanagerdata.SchemaDefinition.decode(reader, reader.uint32()); break; } default: @@ -114505,141 +120066,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetCellsAliasesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse + * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetCellsAliasesResponse.decodeDelimited = function decodeDelimited(reader) { + GetSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetCellsAliasesResponse message. + * Verifies a GetSchemaResponse message. * @function verify - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetCellsAliasesResponse.verify = function verify(message) { + GetSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.aliases != null && message.hasOwnProperty("aliases")) { - if (!$util.isObject(message.aliases)) - return "aliases: object expected"; - let key = Object.keys(message.aliases); - for (let i = 0; i < key.length; ++i) { - let error = $root.topodata.CellsAlias.verify(message.aliases[key[i]]); - if (error) - return "aliases." + error; - } + if (message.schema != null && message.hasOwnProperty("schema")) { + let error = $root.tabletmanagerdata.SchemaDefinition.verify(message.schema); + if (error) + return "schema." + error; } return null; }; /** - * Creates a GetCellsAliasesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetCellsAliasesResponse} GetCellsAliasesResponse + * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse */ - GetCellsAliasesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetCellsAliasesResponse) + GetSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSchemaResponse) return object; - let message = new $root.vtctldata.GetCellsAliasesResponse(); - if (object.aliases) { - if (typeof object.aliases !== "object") - throw TypeError(".vtctldata.GetCellsAliasesResponse.aliases: object expected"); - message.aliases = {}; - for (let keys = Object.keys(object.aliases), i = 0; i < keys.length; ++i) { - if (typeof object.aliases[keys[i]] !== "object") - throw TypeError(".vtctldata.GetCellsAliasesResponse.aliases: object expected"); - message.aliases[keys[i]] = $root.topodata.CellsAlias.fromObject(object.aliases[keys[i]]); - } + let message = new $root.vtctldata.GetSchemaResponse(); + if (object.schema != null) { + if (typeof object.schema !== "object") + throw TypeError(".vtctldata.GetSchemaResponse.schema: object expected"); + message.schema = $root.tabletmanagerdata.SchemaDefinition.fromObject(object.schema); } return message; }; /** - * Creates a plain object from a GetCellsAliasesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static - * @param {vtctldata.GetCellsAliasesResponse} message GetCellsAliasesResponse + * @param {vtctldata.GetSchemaResponse} message GetSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetCellsAliasesResponse.toObject = function toObject(message, options) { + GetSchemaResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.aliases = {}; - let keys2; - if (message.aliases && (keys2 = Object.keys(message.aliases)).length) { - object.aliases = {}; - for (let j = 0; j < keys2.length; ++j) - object.aliases[keys2[j]] = $root.topodata.CellsAlias.toObject(message.aliases[keys2[j]], options); - } + if (options.defaults) + object.schema = null; + if (message.schema != null && message.hasOwnProperty("schema")) + object.schema = $root.tabletmanagerdata.SchemaDefinition.toObject(message.schema, options); return object; }; /** - * Converts this GetCellsAliasesResponse to JSON. + * Converts this GetSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @instance * @returns {Object.} JSON object */ - GetCellsAliasesResponse.prototype.toJSON = function toJSON() { + GetSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetCellsAliasesResponse + * Gets the default type url for GetSchemaResponse * @function getTypeUrl - * @memberof vtctldata.GetCellsAliasesResponse + * @memberof vtctldata.GetSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetCellsAliasesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetCellsAliasesResponse"; + return typeUrlPrefix + "/vtctldata.GetSchemaResponse"; }; - return GetCellsAliasesResponse; + return GetSchemaResponse; })(); - vtctldata.GetFullStatusRequest = (function() { + vtctldata.GetShardRequest = (function() { /** - * Properties of a GetFullStatusRequest. + * Properties of a GetShardRequest. * @memberof vtctldata - * @interface IGetFullStatusRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] GetFullStatusRequest tablet_alias + * @interface IGetShardRequest + * @property {string|null} [keyspace] GetShardRequest keyspace + * @property {string|null} [shard_name] GetShardRequest shard_name */ /** - * Constructs a new GetFullStatusRequest. + * Constructs a new GetShardRequest. * @memberof vtctldata - * @classdesc Represents a GetFullStatusRequest. - * @implements IGetFullStatusRequest + * @classdesc Represents a GetShardRequest. + * @implements IGetShardRequest * @constructor - * @param {vtctldata.IGetFullStatusRequest=} [properties] Properties to set + * @param {vtctldata.IGetShardRequest=} [properties] Properties to set */ - function GetFullStatusRequest(properties) { + function GetShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -114647,75 +120195,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetFullStatusRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.GetFullStatusRequest + * GetShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetShardRequest * @instance */ - GetFullStatusRequest.prototype.tablet_alias = null; + GetShardRequest.prototype.keyspace = ""; /** - * Creates a new GetFullStatusRequest instance using the specified properties. + * GetShardRequest shard_name. + * @member {string} shard_name + * @memberof vtctldata.GetShardRequest + * @instance + */ + GetShardRequest.prototype.shard_name = ""; + + /** + * Creates a new GetShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static - * @param {vtctldata.IGetFullStatusRequest=} [properties] Properties to set - * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest instance + * @param {vtctldata.IGetShardRequest=} [properties] Properties to set + * @returns {vtctldata.GetShardRequest} GetShardRequest instance */ - GetFullStatusRequest.create = function create(properties) { - return new GetFullStatusRequest(properties); + GetShardRequest.create = function create(properties) { + return new GetShardRequest(properties); }; /** - * Encodes the specified GetFullStatusRequest message. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. + * Encodes the specified GetShardRequest message. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static - * @param {vtctldata.IGetFullStatusRequest} message GetFullStatusRequest message or plain object to encode + * @param {vtctldata.IGetShardRequest} message GetShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetFullStatusRequest.encode = function encode(message, writer) { + GetShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); return writer; }; /** - * Encodes the specified GetFullStatusRequest message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusRequest.verify|verify} messages. + * Encodes the specified GetShardRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static - * @param {vtctldata.IGetFullStatusRequest} message GetFullStatusRequest message or plain object to encode + * @param {vtctldata.IGetShardRequest} message GetShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetFullStatusRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetFullStatusRequest message from the specified reader or buffer. + * Decodes a GetShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest + * @returns {vtctldata.GetShardRequest} GetShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetFullStatusRequest.decode = function decode(reader, length) { + GetShardRequest.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.GetFullStatusRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard_name = reader.string(); break; } default: @@ -114727,127 +120289,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetFullStatusRequest message from the specified reader or buffer, length delimited. + * Decodes a GetShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest + * @returns {vtctldata.GetShardRequest} GetShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetFullStatusRequest.decodeDelimited = function decodeDelimited(reader) { + GetShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetFullStatusRequest message. + * Verifies a GetShardRequest message. * @function verify - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetFullStatusRequest.verify = function verify(message) { + GetShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + if (!$util.isString(message.shard_name)) + return "shard_name: string expected"; return null; }; /** - * Creates a GetFullStatusRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetFullStatusRequest} GetFullStatusRequest + * @returns {vtctldata.GetShardRequest} GetShardRequest */ - GetFullStatusRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetFullStatusRequest) + GetShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetShardRequest) return object; - let message = new $root.vtctldata.GetFullStatusRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.GetFullStatusRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.GetShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard_name != null) + message.shard_name = String(object.shard_name); return message; }; /** - * Creates a plain object from a GetFullStatusRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static - * @param {vtctldata.GetFullStatusRequest} message GetFullStatusRequest + * @param {vtctldata.GetShardRequest} message GetShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetFullStatusRequest.toObject = function toObject(message, options) { + GetShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) { + object.keyspace = ""; + object.shard_name = ""; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + object.shard_name = message.shard_name; return object; }; /** - * Converts this GetFullStatusRequest to JSON. + * Converts this GetShardRequest to JSON. * @function toJSON - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @instance * @returns {Object.} JSON object */ - GetFullStatusRequest.prototype.toJSON = function toJSON() { + GetShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetFullStatusRequest + * Gets the default type url for GetShardRequest * @function getTypeUrl - * @memberof vtctldata.GetFullStatusRequest + * @memberof vtctldata.GetShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetFullStatusRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetFullStatusRequest"; + return typeUrlPrefix + "/vtctldata.GetShardRequest"; }; - return GetFullStatusRequest; + return GetShardRequest; })(); - vtctldata.GetFullStatusResponse = (function() { + vtctldata.GetShardResponse = (function() { /** - * Properties of a GetFullStatusResponse. + * Properties of a GetShardResponse. * @memberof vtctldata - * @interface IGetFullStatusResponse - * @property {replicationdata.IFullStatus|null} [status] GetFullStatusResponse status + * @interface IGetShardResponse + * @property {vtctldata.IShard|null} [shard] GetShardResponse shard */ /** - * Constructs a new GetFullStatusResponse. + * Constructs a new GetShardResponse. * @memberof vtctldata - * @classdesc Represents a GetFullStatusResponse. - * @implements IGetFullStatusResponse + * @classdesc Represents a GetShardResponse. + * @implements IGetShardResponse * @constructor - * @param {vtctldata.IGetFullStatusResponse=} [properties] Properties to set + * @param {vtctldata.IGetShardResponse=} [properties] Properties to set */ - function GetFullStatusResponse(properties) { + function GetShardResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -114855,75 +120421,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetFullStatusResponse status. - * @member {replicationdata.IFullStatus|null|undefined} status - * @memberof vtctldata.GetFullStatusResponse + * GetShardResponse shard. + * @member {vtctldata.IShard|null|undefined} shard + * @memberof vtctldata.GetShardResponse * @instance */ - GetFullStatusResponse.prototype.status = null; + GetShardResponse.prototype.shard = null; /** - * Creates a new GetFullStatusResponse instance using the specified properties. + * Creates a new GetShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static - * @param {vtctldata.IGetFullStatusResponse=} [properties] Properties to set - * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse instance + * @param {vtctldata.IGetShardResponse=} [properties] Properties to set + * @returns {vtctldata.GetShardResponse} GetShardResponse instance */ - GetFullStatusResponse.create = function create(properties) { - return new GetFullStatusResponse(properties); + GetShardResponse.create = function create(properties) { + return new GetShardResponse(properties); }; /** - * Encodes the specified GetFullStatusResponse message. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. + * Encodes the specified GetShardResponse message. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static - * @param {vtctldata.IGetFullStatusResponse} message GetFullStatusResponse message or plain object to encode + * @param {vtctldata.IGetShardResponse} message GetShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetFullStatusResponse.encode = function encode(message, writer) { + GetShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.status != null && Object.hasOwnProperty.call(message, "status")) - $root.replicationdata.FullStatus.encode(message.status, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.vtctldata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetFullStatusResponse message, length delimited. Does not implicitly {@link vtctldata.GetFullStatusResponse.verify|verify} messages. + * Encodes the specified GetShardResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static - * @param {vtctldata.IGetFullStatusResponse} message GetFullStatusResponse message or plain object to encode + * @param {vtctldata.IGetShardResponse} message GetShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetFullStatusResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetFullStatusResponse message from the specified reader or buffer. + * Decodes a GetShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse + * @returns {vtctldata.GetShardResponse} GetShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetFullStatusResponse.decode = function decode(reader, length) { + GetShardResponse.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.GetFullStatusResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.status = $root.replicationdata.FullStatus.decode(reader, reader.uint32()); + message.shard = $root.vtctldata.Shard.decode(reader, reader.uint32()); break; } default: @@ -114935,126 +120501,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetFullStatusResponse message from the specified reader or buffer, length delimited. + * Decodes a GetShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse + * @returns {vtctldata.GetShardResponse} GetShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetFullStatusResponse.decodeDelimited = function decodeDelimited(reader) { + GetShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetFullStatusResponse message. + * Verifies a GetShardResponse message. * @function verify - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetFullStatusResponse.verify = function verify(message) { + GetShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.status != null && message.hasOwnProperty("status")) { - let error = $root.replicationdata.FullStatus.verify(message.status); + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.vtctldata.Shard.verify(message.shard); if (error) - return "status." + error; + return "shard." + error; } return null; }; /** - * Creates a GetFullStatusResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetFullStatusResponse} GetFullStatusResponse + * @returns {vtctldata.GetShardResponse} GetShardResponse */ - GetFullStatusResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetFullStatusResponse) + GetShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetShardResponse) return object; - let message = new $root.vtctldata.GetFullStatusResponse(); - if (object.status != null) { - if (typeof object.status !== "object") - throw TypeError(".vtctldata.GetFullStatusResponse.status: object expected"); - message.status = $root.replicationdata.FullStatus.fromObject(object.status); + let message = new $root.vtctldata.GetShardResponse(); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.GetShardResponse.shard: object expected"); + message.shard = $root.vtctldata.Shard.fromObject(object.shard); } return message; }; /** - * Creates a plain object from a GetFullStatusResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static - * @param {vtctldata.GetFullStatusResponse} message GetFullStatusResponse + * @param {vtctldata.GetShardResponse} message GetShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetFullStatusResponse.toObject = function toObject(message, options) { + GetShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.status = null; - if (message.status != null && message.hasOwnProperty("status")) - object.status = $root.replicationdata.FullStatus.toObject(message.status, options); + object.shard = null; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.vtctldata.Shard.toObject(message.shard, options); return object; }; /** - * Converts this GetFullStatusResponse to JSON. + * Converts this GetShardResponse to JSON. * @function toJSON - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @instance * @returns {Object.} JSON object */ - GetFullStatusResponse.prototype.toJSON = function toJSON() { + GetShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetFullStatusResponse + * Gets the default type url for GetShardResponse * @function getTypeUrl - * @memberof vtctldata.GetFullStatusResponse + * @memberof vtctldata.GetShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetFullStatusResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetFullStatusResponse"; + return typeUrlPrefix + "/vtctldata.GetShardResponse"; }; - return GetFullStatusResponse; + return GetShardResponse; })(); - vtctldata.GetKeyspacesRequest = (function() { + vtctldata.GetShardRoutingRulesRequest = (function() { /** - * Properties of a GetKeyspacesRequest. + * Properties of a GetShardRoutingRulesRequest. * @memberof vtctldata - * @interface IGetKeyspacesRequest + * @interface IGetShardRoutingRulesRequest */ /** - * Constructs a new GetKeyspacesRequest. + * Constructs a new GetShardRoutingRulesRequest. * @memberof vtctldata - * @classdesc Represents a GetKeyspacesRequest. - * @implements IGetKeyspacesRequest + * @classdesc Represents a GetShardRoutingRulesRequest. + * @implements IGetShardRoutingRulesRequest * @constructor - * @param {vtctldata.IGetKeyspacesRequest=} [properties] Properties to set + * @param {vtctldata.IGetShardRoutingRulesRequest=} [properties] Properties to set */ - function GetKeyspacesRequest(properties) { + function GetShardRoutingRulesRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -115062,60 +120628,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new GetKeyspacesRequest instance using the specified properties. + * Creates a new GetShardRoutingRulesRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static - * @param {vtctldata.IGetKeyspacesRequest=} [properties] Properties to set - * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest instance + * @param {vtctldata.IGetShardRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest instance */ - GetKeyspacesRequest.create = function create(properties) { - return new GetKeyspacesRequest(properties); + GetShardRoutingRulesRequest.create = function create(properties) { + return new GetShardRoutingRulesRequest(properties); }; /** - * Encodes the specified GetKeyspacesRequest message. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. + * Encodes the specified GetShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static - * @param {vtctldata.IGetKeyspacesRequest} message GetKeyspacesRequest message or plain object to encode + * @param {vtctldata.IGetShardRoutingRulesRequest} message GetShardRoutingRulesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspacesRequest.encode = function encode(message, writer) { + GetShardRoutingRulesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified GetKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesRequest.verify|verify} messages. + * Encodes the specified GetShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static - * @param {vtctldata.IGetKeyspacesRequest} message GetKeyspacesRequest message or plain object to encode + * @param {vtctldata.IGetShardRoutingRulesRequest} message GetShardRoutingRulesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspacesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetShardRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetKeyspacesRequest message from the specified reader or buffer. + * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest + * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspacesRequest.decode = function decode(reader, length) { + GetShardRoutingRulesRequest.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.GetKeyspacesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetShardRoutingRulesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -115128,110 +120694,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetKeyspacesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest + * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspacesRequest.decodeDelimited = function decodeDelimited(reader) { + GetShardRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetKeyspacesRequest message. + * Verifies a GetShardRoutingRulesRequest message. * @function verify - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetKeyspacesRequest.verify = function verify(message) { + GetShardRoutingRulesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a GetKeyspacesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetKeyspacesRequest} GetKeyspacesRequest + * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest */ - GetKeyspacesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetKeyspacesRequest) + GetShardRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetShardRoutingRulesRequest) return object; - return new $root.vtctldata.GetKeyspacesRequest(); + return new $root.vtctldata.GetShardRoutingRulesRequest(); }; /** - * Creates a plain object from a GetKeyspacesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetShardRoutingRulesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static - * @param {vtctldata.GetKeyspacesRequest} message GetKeyspacesRequest + * @param {vtctldata.GetShardRoutingRulesRequest} message GetShardRoutingRulesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetKeyspacesRequest.toObject = function toObject() { + GetShardRoutingRulesRequest.toObject = function toObject() { return {}; }; /** - * Converts this GetKeyspacesRequest to JSON. + * Converts this GetShardRoutingRulesRequest to JSON. * @function toJSON - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @instance * @returns {Object.} JSON object */ - GetKeyspacesRequest.prototype.toJSON = function toJSON() { + GetShardRoutingRulesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetKeyspacesRequest + * Gets the default type url for GetShardRoutingRulesRequest * @function getTypeUrl - * @memberof vtctldata.GetKeyspacesRequest + * @memberof vtctldata.GetShardRoutingRulesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetKeyspacesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetShardRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetKeyspacesRequest"; + return typeUrlPrefix + "/vtctldata.GetShardRoutingRulesRequest"; }; - return GetKeyspacesRequest; + return GetShardRoutingRulesRequest; })(); - vtctldata.GetKeyspacesResponse = (function() { + vtctldata.GetShardRoutingRulesResponse = (function() { /** - * Properties of a GetKeyspacesResponse. + * Properties of a GetShardRoutingRulesResponse. * @memberof vtctldata - * @interface IGetKeyspacesResponse - * @property {Array.|null} [keyspaces] GetKeyspacesResponse keyspaces + * @interface IGetShardRoutingRulesResponse + * @property {vschema.IShardRoutingRules|null} [shard_routing_rules] GetShardRoutingRulesResponse shard_routing_rules */ /** - * Constructs a new GetKeyspacesResponse. + * Constructs a new GetShardRoutingRulesResponse. * @memberof vtctldata - * @classdesc Represents a GetKeyspacesResponse. - * @implements IGetKeyspacesResponse + * @classdesc Represents a GetShardRoutingRulesResponse. + * @implements IGetShardRoutingRulesResponse * @constructor - * @param {vtctldata.IGetKeyspacesResponse=} [properties] Properties to set + * @param {vtctldata.IGetShardRoutingRulesResponse=} [properties] Properties to set */ - function GetKeyspacesResponse(properties) { - this.keyspaces = []; + function GetShardRoutingRulesResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -115239,78 +120804,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetKeyspacesResponse keyspaces. - * @member {Array.} keyspaces - * @memberof vtctldata.GetKeyspacesResponse + * GetShardRoutingRulesResponse shard_routing_rules. + * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules + * @memberof vtctldata.GetShardRoutingRulesResponse * @instance */ - GetKeyspacesResponse.prototype.keyspaces = $util.emptyArray; + GetShardRoutingRulesResponse.prototype.shard_routing_rules = null; /** - * Creates a new GetKeyspacesResponse instance using the specified properties. + * Creates a new GetShardRoutingRulesResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static - * @param {vtctldata.IGetKeyspacesResponse=} [properties] Properties to set - * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse instance + * @param {vtctldata.IGetShardRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse instance */ - GetKeyspacesResponse.create = function create(properties) { - return new GetKeyspacesResponse(properties); + GetShardRoutingRulesResponse.create = function create(properties) { + return new GetShardRoutingRulesResponse(properties); }; /** - * Encodes the specified GetKeyspacesResponse message. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. + * Encodes the specified GetShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static - * @param {vtctldata.IGetKeyspacesResponse} message GetKeyspacesResponse message or plain object to encode + * @param {vtctldata.IGetShardRoutingRulesResponse} message GetShardRoutingRulesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspacesResponse.encode = function encode(message, writer) { + GetShardRoutingRulesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspaces != null && message.keyspaces.length) - for (let i = 0; i < message.keyspaces.length; ++i) - $root.vtctldata.Keyspace.encode(message.keyspaces[i], writer.uint32(/* id 1, wireType 2 =*/10).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 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspacesResponse.verify|verify} messages. + * Encodes the specified GetShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static - * @param {vtctldata.IGetKeyspacesResponse} message GetKeyspacesResponse message or plain object to encode + * @param {vtctldata.IGetShardRoutingRulesResponse} message GetShardRoutingRulesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspacesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetShardRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetKeyspacesResponse message from the specified reader or buffer. + * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse + * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspacesResponse.decode = function decode(reader, length) { + GetShardRoutingRulesResponse.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.GetKeyspacesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetShardRoutingRulesResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.keyspaces && message.keyspaces.length)) - message.keyspaces = []; - message.keyspaces.push($root.vtctldata.Keyspace.decode(reader, reader.uint32())); + message.shard_routing_rules = $root.vschema.ShardRoutingRules.decode(reader, reader.uint32()); break; } default: @@ -115322,139 +120884,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetKeyspacesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse + * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspacesResponse.decodeDelimited = function decodeDelimited(reader) { + GetShardRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetKeyspacesResponse message. + * Verifies a GetShardRoutingRulesResponse message. * @function verify - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetKeyspacesResponse.verify = function verify(message) { + GetShardRoutingRulesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspaces != null && message.hasOwnProperty("keyspaces")) { - if (!Array.isArray(message.keyspaces)) - return "keyspaces: array expected"; - for (let i = 0; i < message.keyspaces.length; ++i) { - let error = $root.vtctldata.Keyspace.verify(message.keyspaces[i]); - if (error) - return "keyspaces." + error; - } + if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { + let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); + if (error) + return "shard_routing_rules." + error; } return null; }; /** - * Creates a GetKeyspacesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetKeyspacesResponse} GetKeyspacesResponse + * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse */ - GetKeyspacesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetKeyspacesResponse) + GetShardRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetShardRoutingRulesResponse) return object; - let message = new $root.vtctldata.GetKeyspacesResponse(); - if (object.keyspaces) { - if (!Array.isArray(object.keyspaces)) - throw TypeError(".vtctldata.GetKeyspacesResponse.keyspaces: array expected"); - message.keyspaces = []; - for (let i = 0; i < object.keyspaces.length; ++i) { - if (typeof object.keyspaces[i] !== "object") - throw TypeError(".vtctldata.GetKeyspacesResponse.keyspaces: object expected"); - message.keyspaces[i] = $root.vtctldata.Keyspace.fromObject(object.keyspaces[i]); - } + let message = new $root.vtctldata.GetShardRoutingRulesResponse(); + if (object.shard_routing_rules != null) { + if (typeof object.shard_routing_rules !== "object") + throw TypeError(".vtctldata.GetShardRoutingRulesResponse.shard_routing_rules: object expected"); + message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_routing_rules); } return message; }; /** - * Creates a plain object from a GetKeyspacesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetShardRoutingRulesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static - * @param {vtctldata.GetKeyspacesResponse} message GetKeyspacesResponse + * @param {vtctldata.GetShardRoutingRulesResponse} message GetShardRoutingRulesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetKeyspacesResponse.toObject = function toObject(message, options) { + GetShardRoutingRulesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.keyspaces = []; - if (message.keyspaces && message.keyspaces.length) { - object.keyspaces = []; - for (let j = 0; j < message.keyspaces.length; ++j) - object.keyspaces[j] = $root.vtctldata.Keyspace.toObject(message.keyspaces[j], options); - } + if (options.defaults) + object.shard_routing_rules = null; + if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) + object.shard_routing_rules = $root.vschema.ShardRoutingRules.toObject(message.shard_routing_rules, options); return object; }; /** - * Converts this GetKeyspacesResponse to JSON. + * Converts this GetShardRoutingRulesResponse to JSON. * @function toJSON - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @instance * @returns {Object.} JSON object */ - GetKeyspacesResponse.prototype.toJSON = function toJSON() { + GetShardRoutingRulesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetKeyspacesResponse + * Gets the default type url for GetShardRoutingRulesResponse * @function getTypeUrl - * @memberof vtctldata.GetKeyspacesResponse + * @memberof vtctldata.GetShardRoutingRulesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetKeyspacesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetShardRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetKeyspacesResponse"; + return typeUrlPrefix + "/vtctldata.GetShardRoutingRulesResponse"; }; - return GetKeyspacesResponse; + return GetShardRoutingRulesResponse; })(); - vtctldata.GetKeyspaceRequest = (function() { + vtctldata.GetSrvKeyspaceNamesRequest = (function() { /** - * Properties of a GetKeyspaceRequest. + * Properties of a GetSrvKeyspaceNamesRequest. * @memberof vtctldata - * @interface IGetKeyspaceRequest - * @property {string|null} [keyspace] GetKeyspaceRequest keyspace + * @interface IGetSrvKeyspaceNamesRequest + * @property {Array.|null} [cells] GetSrvKeyspaceNamesRequest cells */ /** - * Constructs a new GetKeyspaceRequest. + * Constructs a new GetSrvKeyspaceNamesRequest. * @memberof vtctldata - * @classdesc Represents a GetKeyspaceRequest. - * @implements IGetKeyspaceRequest + * @classdesc Represents a GetSrvKeyspaceNamesRequest. + * @implements IGetSrvKeyspaceNamesRequest * @constructor - * @param {vtctldata.IGetKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IGetSrvKeyspaceNamesRequest=} [properties] Properties to set */ - function GetKeyspaceRequest(properties) { + function GetSrvKeyspaceNamesRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -115462,75 +121013,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetKeyspaceRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.GetKeyspaceRequest + * GetSrvKeyspaceNamesRequest cells. + * @member {Array.} cells + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @instance */ - GetKeyspaceRequest.prototype.keyspace = ""; + GetSrvKeyspaceNamesRequest.prototype.cells = $util.emptyArray; /** - * Creates a new GetKeyspaceRequest instance using the specified properties. + * Creates a new GetSrvKeyspaceNamesRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static - * @param {vtctldata.IGetKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest instance + * @param {vtctldata.IGetSrvKeyspaceNamesRequest=} [properties] Properties to set + * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest instance */ - GetKeyspaceRequest.create = function create(properties) { - return new GetKeyspaceRequest(properties); + GetSrvKeyspaceNamesRequest.create = function create(properties) { + return new GetSrvKeyspaceNamesRequest(properties); }; /** - * Encodes the specified GetKeyspaceRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. + * Encodes the specified GetSrvKeyspaceNamesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static - * @param {vtctldata.IGetKeyspaceRequest} message GetKeyspaceRequest message or plain object to encode + * @param {vtctldata.IGetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspaceRequest.encode = function encode(message, writer) { + GetSrvKeyspaceNamesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.cells[i]); return writer; }; /** - * Encodes the specified GetKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRequest.verify|verify} messages. + * Encodes the specified GetSrvKeyspaceNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static - * @param {vtctldata.IGetKeyspaceRequest} message GetKeyspaceRequest message or plain object to encode + * @param {vtctldata.IGetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvKeyspaceNamesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetKeyspaceRequest message from the specified reader or buffer. + * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest + * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspaceRequest.decode = function decode(reader, length) { + GetSrvKeyspaceNamesRequest.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.GetKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvKeyspaceNamesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } default: @@ -115542,122 +121096,135 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest + * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + GetSrvKeyspaceNamesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetKeyspaceRequest message. + * Verifies a GetSrvKeyspaceNamesRequest message. * @function verify - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetKeyspaceRequest.verify = function verify(message) { + GetSrvKeyspaceNamesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } return null; }; /** - * Creates a GetKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspaceNamesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetKeyspaceRequest} GetKeyspaceRequest + * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest */ - GetKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetKeyspaceRequest) + GetSrvKeyspaceNamesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesRequest) return object; - let message = new $root.vtctldata.GetKeyspaceRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); + let message = new $root.vtctldata.GetSrvKeyspaceNamesRequest(); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.GetSrvKeyspaceNamesRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } return message; }; /** - * Creates a plain object from a GetKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvKeyspaceNamesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static - * @param {vtctldata.GetKeyspaceRequest} message GetKeyspaceRequest + * @param {vtctldata.GetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetKeyspaceRequest.toObject = function toObject(message, options) { + GetSrvKeyspaceNamesRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.keyspace = ""; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; + if (options.arrays || options.defaults) + object.cells = []; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } return object; }; /** - * Converts this GetKeyspaceRequest to JSON. + * Converts this GetSrvKeyspaceNamesRequest to JSON. * @function toJSON - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @instance * @returns {Object.} JSON object */ - GetKeyspaceRequest.prototype.toJSON = function toJSON() { + GetSrvKeyspaceNamesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetKeyspaceRequest + * Gets the default type url for GetSrvKeyspaceNamesRequest * @function getTypeUrl - * @memberof vtctldata.GetKeyspaceRequest + * @memberof vtctldata.GetSrvKeyspaceNamesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvKeyspaceNamesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesRequest"; }; - return GetKeyspaceRequest; + return GetSrvKeyspaceNamesRequest; })(); - vtctldata.GetKeyspaceResponse = (function() { + vtctldata.GetSrvKeyspaceNamesResponse = (function() { /** - * Properties of a GetKeyspaceResponse. + * Properties of a GetSrvKeyspaceNamesResponse. * @memberof vtctldata - * @interface IGetKeyspaceResponse - * @property {vtctldata.IKeyspace|null} [keyspace] GetKeyspaceResponse keyspace + * @interface IGetSrvKeyspaceNamesResponse + * @property {Object.|null} [names] GetSrvKeyspaceNamesResponse names */ /** - * Constructs a new GetKeyspaceResponse. + * Constructs a new GetSrvKeyspaceNamesResponse. * @memberof vtctldata - * @classdesc Represents a GetKeyspaceResponse. - * @implements IGetKeyspaceResponse + * @classdesc Represents a GetSrvKeyspaceNamesResponse. + * @implements IGetSrvKeyspaceNamesResponse * @constructor - * @param {vtctldata.IGetKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IGetSrvKeyspaceNamesResponse=} [properties] Properties to set */ - function GetKeyspaceResponse(properties) { + function GetSrvKeyspaceNamesResponse(properties) { + this.names = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -115665,75 +121232,97 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetKeyspaceResponse keyspace. - * @member {vtctldata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.GetKeyspaceResponse + * GetSrvKeyspaceNamesResponse names. + * @member {Object.} names + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @instance */ - GetKeyspaceResponse.prototype.keyspace = null; + GetSrvKeyspaceNamesResponse.prototype.names = $util.emptyObject; /** - * Creates a new GetKeyspaceResponse instance using the specified properties. + * Creates a new GetSrvKeyspaceNamesResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static - * @param {vtctldata.IGetKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse instance + * @param {vtctldata.IGetSrvKeyspaceNamesResponse=} [properties] Properties to set + * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse instance */ - GetKeyspaceResponse.create = function create(properties) { - return new GetKeyspaceResponse(properties); + GetSrvKeyspaceNamesResponse.create = function create(properties) { + return new GetSrvKeyspaceNamesResponse(properties); }; /** - * Encodes the specified GetKeyspaceResponse message. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. + * Encodes the specified GetSrvKeyspaceNamesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static - * @param {vtctldata.IGetKeyspaceResponse} message GetKeyspaceResponse message or plain object to encode + * @param {vtctldata.IGetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspaceResponse.encode = function encode(message, writer) { + GetSrvKeyspaceNamesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.vtctldata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.names != null && Object.hasOwnProperty.call(message, "names")) + for (let keys = Object.keys(message.names), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.encode(message.names[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified GetKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceResponse.verify|verify} messages. + * Encodes the specified GetSrvKeyspaceNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static - * @param {vtctldata.IGetKeyspaceResponse} message GetKeyspaceResponse message or plain object to encode + * @param {vtctldata.IGetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvKeyspaceNamesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetKeyspaceResponse message from the specified reader or buffer. + * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse + * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspaceResponse.decode = function decode(reader, length) { + GetSrvKeyspaceNamesResponse.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.GetKeyspaceResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvKeyspaceNamesResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = $root.vtctldata.Keyspace.decode(reader, reader.uint32()); + if (message.names === $util.emptyObject) + message.names = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.names[key] = value; break; } default: @@ -115745,127 +121334,362 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse + * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + GetSrvKeyspaceNamesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetKeyspaceResponse message. + * Verifies a GetSrvKeyspaceNamesResponse message. * @function verify - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetKeyspaceResponse.verify = function verify(message) { + GetSrvKeyspaceNamesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.vtctldata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; + if (message.names != null && message.hasOwnProperty("names")) { + if (!$util.isObject(message.names)) + return "names: object expected"; + let key = Object.keys(message.names); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify(message.names[key[i]]); + if (error) + return "names." + error; + } } return null; }; /** - * Creates a GetKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspaceNamesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetKeyspaceResponse} GetKeyspaceResponse + * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse */ - GetKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetKeyspaceResponse) + GetSrvKeyspaceNamesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesResponse) return object; - let message = new $root.vtctldata.GetKeyspaceResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.GetKeyspaceResponse.keyspace: object expected"); - message.keyspace = $root.vtctldata.Keyspace.fromObject(object.keyspace); + let message = new $root.vtctldata.GetSrvKeyspaceNamesResponse(); + if (object.names) { + if (typeof object.names !== "object") + throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.names: object expected"); + message.names = {}; + for (let keys = Object.keys(object.names), i = 0; i < keys.length; ++i) { + if (typeof object.names[keys[i]] !== "object") + throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.names: object expected"); + message.names[keys[i]] = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.fromObject(object.names[keys[i]]); + } } return message; }; /** - * Creates a plain object from a GetKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvKeyspaceNamesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static - * @param {vtctldata.GetKeyspaceResponse} message GetKeyspaceResponse + * @param {vtctldata.GetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetKeyspaceResponse.toObject = function toObject(message, options) { + GetSrvKeyspaceNamesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.keyspace = null; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.vtctldata.Keyspace.toObject(message.keyspace, options); + if (options.objects || options.defaults) + object.names = {}; + let keys2; + if (message.names && (keys2 = Object.keys(message.names)).length) { + object.names = {}; + for (let j = 0; j < keys2.length; ++j) + object.names[keys2[j]] = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.toObject(message.names[keys2[j]], options); + } return object; }; /** - * Converts this GetKeyspaceResponse to JSON. + * Converts this GetSrvKeyspaceNamesResponse to JSON. * @function toJSON - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @instance * @returns {Object.} JSON object */ - GetKeyspaceResponse.prototype.toJSON = function toJSON() { + GetSrvKeyspaceNamesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetKeyspaceResponse + * Gets the default type url for GetSrvKeyspaceNamesResponse * @function getTypeUrl - * @memberof vtctldata.GetKeyspaceResponse + * @memberof vtctldata.GetSrvKeyspaceNamesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvKeyspaceNamesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesResponse"; }; - return GetKeyspaceResponse; + GetSrvKeyspaceNamesResponse.NameList = (function() { + + /** + * Properties of a NameList. + * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @interface INameList + * @property {Array.|null} [names] NameList names + */ + + /** + * Constructs a new NameList. + * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @classdesc Represents a NameList. + * @implements INameList + * @constructor + * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList=} [properties] Properties to set + */ + function NameList(properties) { + this.names = []; + 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]]; + } + + /** + * NameList names. + * @member {Array.} names + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @instance + */ + NameList.prototype.names = $util.emptyArray; + + /** + * Creates a new NameList instance using the specified properties. + * @function create + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList=} [properties] Properties to set + * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList instance + */ + NameList.create = function create(properties) { + return new NameList(properties); + }; + + /** + * Encodes the specified NameList message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. + * @function encode + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList} message NameList message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + NameList.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.names != null && message.names.length) + for (let i = 0; i < message.names.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.names[i]); + return writer; + }; + + /** + * Encodes the specified NameList message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList} message NameList message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + NameList.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a NameList message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + NameList.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.GetSrvKeyspaceNamesResponse.NameList(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.names && message.names.length)) + message.names = []; + message.names.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a NameList message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + NameList.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a NameList message. + * @function verify + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + NameList.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.names != null && message.hasOwnProperty("names")) { + if (!Array.isArray(message.names)) + return "names: array expected"; + for (let i = 0; i < message.names.length; ++i) + if (!$util.isString(message.names[i])) + return "names: string[] expected"; + } + return null; + }; + + /** + * Creates a NameList message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList + */ + NameList.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList) + return object; + let message = new $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList(); + if (object.names) { + if (!Array.isArray(object.names)) + throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.NameList.names: array expected"); + message.names = []; + for (let i = 0; i < object.names.length; ++i) + message.names[i] = String(object.names[i]); + } + return message; + }; + + /** + * Creates a plain object from a NameList message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {vtctldata.GetSrvKeyspaceNamesResponse.NameList} message NameList + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + NameList.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.names = []; + if (message.names && message.names.length) { + object.names = []; + for (let j = 0; j < message.names.length; ++j) + object.names[j] = message.names[j]; + } + return object; + }; + + /** + * Converts this NameList to JSON. + * @function toJSON + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @instance + * @returns {Object.} JSON object + */ + NameList.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for NameList + * @function getTypeUrl + * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + NameList.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesResponse.NameList"; + }; + + return NameList; + })(); + + return GetSrvKeyspaceNamesResponse; })(); - vtctldata.GetPermissionsRequest = (function() { + vtctldata.GetSrvKeyspacesRequest = (function() { /** - * Properties of a GetPermissionsRequest. + * Properties of a GetSrvKeyspacesRequest. * @memberof vtctldata - * @interface IGetPermissionsRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] GetPermissionsRequest tablet_alias + * @interface IGetSrvKeyspacesRequest + * @property {string|null} [keyspace] GetSrvKeyspacesRequest keyspace + * @property {Array.|null} [cells] GetSrvKeyspacesRequest cells */ /** - * Constructs a new GetPermissionsRequest. + * Constructs a new GetSrvKeyspacesRequest. * @memberof vtctldata - * @classdesc Represents a GetPermissionsRequest. - * @implements IGetPermissionsRequest + * @classdesc Represents a GetSrvKeyspacesRequest. + * @implements IGetSrvKeyspacesRequest * @constructor - * @param {vtctldata.IGetPermissionsRequest=} [properties] Properties to set + * @param {vtctldata.IGetSrvKeyspacesRequest=} [properties] Properties to set */ - function GetPermissionsRequest(properties) { + function GetSrvKeyspacesRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -115873,75 +121697,92 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetPermissionsRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.GetPermissionsRequest + * GetSrvKeyspacesRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetSrvKeyspacesRequest * @instance */ - GetPermissionsRequest.prototype.tablet_alias = null; + GetSrvKeyspacesRequest.prototype.keyspace = ""; /** - * Creates a new GetPermissionsRequest instance using the specified properties. + * GetSrvKeyspacesRequest cells. + * @member {Array.} cells + * @memberof vtctldata.GetSrvKeyspacesRequest + * @instance + */ + GetSrvKeyspacesRequest.prototype.cells = $util.emptyArray; + + /** + * Creates a new GetSrvKeyspacesRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static - * @param {vtctldata.IGetPermissionsRequest=} [properties] Properties to set - * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest instance + * @param {vtctldata.IGetSrvKeyspacesRequest=} [properties] Properties to set + * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest instance */ - GetPermissionsRequest.create = function create(properties) { - return new GetPermissionsRequest(properties); + GetSrvKeyspacesRequest.create = function create(properties) { + return new GetSrvKeyspacesRequest(properties); }; /** - * Encodes the specified GetPermissionsRequest message. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. + * Encodes the specified GetSrvKeyspacesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static - * @param {vtctldata.IGetPermissionsRequest} message GetPermissionsRequest message or plain object to encode + * @param {vtctldata.IGetSrvKeyspacesRequest} message GetSrvKeyspacesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetPermissionsRequest.encode = function encode(message, writer) { + GetSrvKeyspacesRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); return writer; }; /** - * Encodes the specified GetPermissionsRequest message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsRequest.verify|verify} messages. + * Encodes the specified GetSrvKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static - * @param {vtctldata.IGetPermissionsRequest} message GetPermissionsRequest message or plain object to encode + * @param {vtctldata.IGetSrvKeyspacesRequest} message GetSrvKeyspacesRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetPermissionsRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvKeyspacesRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetPermissionsRequest message from the specified reader or buffer. + * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest + * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetPermissionsRequest.decode = function decode(reader, length) { + GetSrvKeyspacesRequest.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.GetPermissionsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvKeyspacesRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } default: @@ -115953,127 +121794,144 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetPermissionsRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest + * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetPermissionsRequest.decodeDelimited = function decodeDelimited(reader) { + GetSrvKeyspacesRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetPermissionsRequest message. + * Verifies a GetSrvKeyspacesRequest message. * @function verify - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetPermissionsRequest.verify = function verify(message) { + GetSrvKeyspacesRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; } return null; }; /** - * Creates a GetPermissionsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspacesRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetPermissionsRequest} GetPermissionsRequest + * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest */ - GetPermissionsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetPermissionsRequest) + GetSrvKeyspacesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvKeyspacesRequest) return object; - let message = new $root.vtctldata.GetPermissionsRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.GetPermissionsRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + let message = new $root.vtctldata.GetSrvKeyspacesRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.GetSrvKeyspacesRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); } return message; }; /** - * Creates a plain object from a GetPermissionsRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvKeyspacesRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static - * @param {vtctldata.GetPermissionsRequest} message GetPermissionsRequest + * @param {vtctldata.GetSrvKeyspacesRequest} message GetSrvKeyspacesRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetPermissionsRequest.toObject = function toObject(message, options) { + GetSrvKeyspacesRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.cells = []; if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + object.keyspace = ""; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } return object; }; /** - * Converts this GetPermissionsRequest to JSON. + * Converts this GetSrvKeyspacesRequest to JSON. * @function toJSON - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @instance * @returns {Object.} JSON object */ - GetPermissionsRequest.prototype.toJSON = function toJSON() { + GetSrvKeyspacesRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetPermissionsRequest + * Gets the default type url for GetSrvKeyspacesRequest * @function getTypeUrl - * @memberof vtctldata.GetPermissionsRequest + * @memberof vtctldata.GetSrvKeyspacesRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetPermissionsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvKeyspacesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetPermissionsRequest"; + return typeUrlPrefix + "/vtctldata.GetSrvKeyspacesRequest"; }; - return GetPermissionsRequest; + return GetSrvKeyspacesRequest; })(); - vtctldata.GetPermissionsResponse = (function() { + vtctldata.GetSrvKeyspacesResponse = (function() { /** - * Properties of a GetPermissionsResponse. + * Properties of a GetSrvKeyspacesResponse. * @memberof vtctldata - * @interface IGetPermissionsResponse - * @property {tabletmanagerdata.IPermissions|null} [permissions] GetPermissionsResponse permissions + * @interface IGetSrvKeyspacesResponse + * @property {Object.|null} [srv_keyspaces] GetSrvKeyspacesResponse srv_keyspaces */ /** - * Constructs a new GetPermissionsResponse. + * Constructs a new GetSrvKeyspacesResponse. * @memberof vtctldata - * @classdesc Represents a GetPermissionsResponse. - * @implements IGetPermissionsResponse + * @classdesc Represents a GetSrvKeyspacesResponse. + * @implements IGetSrvKeyspacesResponse * @constructor - * @param {vtctldata.IGetPermissionsResponse=} [properties] Properties to set + * @param {vtctldata.IGetSrvKeyspacesResponse=} [properties] Properties to set */ - function GetPermissionsResponse(properties) { + function GetSrvKeyspacesResponse(properties) { + this.srv_keyspaces = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -116081,75 +121939,97 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetPermissionsResponse permissions. - * @member {tabletmanagerdata.IPermissions|null|undefined} permissions - * @memberof vtctldata.GetPermissionsResponse + * GetSrvKeyspacesResponse srv_keyspaces. + * @member {Object.} srv_keyspaces + * @memberof vtctldata.GetSrvKeyspacesResponse * @instance */ - GetPermissionsResponse.prototype.permissions = null; + GetSrvKeyspacesResponse.prototype.srv_keyspaces = $util.emptyObject; /** - * Creates a new GetPermissionsResponse instance using the specified properties. + * Creates a new GetSrvKeyspacesResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static - * @param {vtctldata.IGetPermissionsResponse=} [properties] Properties to set - * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse instance + * @param {vtctldata.IGetSrvKeyspacesResponse=} [properties] Properties to set + * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse instance */ - GetPermissionsResponse.create = function create(properties) { - return new GetPermissionsResponse(properties); + GetSrvKeyspacesResponse.create = function create(properties) { + return new GetSrvKeyspacesResponse(properties); }; /** - * Encodes the specified GetPermissionsResponse message. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. + * Encodes the specified GetSrvKeyspacesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static - * @param {vtctldata.IGetPermissionsResponse} message GetPermissionsResponse message or plain object to encode + * @param {vtctldata.IGetSrvKeyspacesResponse} message GetSrvKeyspacesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetPermissionsResponse.encode = function encode(message, writer) { + GetSrvKeyspacesResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.permissions != null && Object.hasOwnProperty.call(message, "permissions")) - $root.tabletmanagerdata.Permissions.encode(message.permissions, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.srv_keyspaces != null && Object.hasOwnProperty.call(message, "srv_keyspaces")) + for (let keys = Object.keys(message.srv_keyspaces), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.topodata.SrvKeyspace.encode(message.srv_keyspaces[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified GetPermissionsResponse message, length delimited. Does not implicitly {@link vtctldata.GetPermissionsResponse.verify|verify} messages. + * Encodes the specified GetSrvKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static - * @param {vtctldata.IGetPermissionsResponse} message GetPermissionsResponse message or plain object to encode + * @param {vtctldata.IGetSrvKeyspacesResponse} message GetSrvKeyspacesResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetPermissionsResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvKeyspacesResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetPermissionsResponse message from the specified reader or buffer. + * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse + * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetPermissionsResponse.decode = function decode(reader, length) { + GetSrvKeyspacesResponse.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.GetPermissionsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvKeyspacesResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.permissions = $root.tabletmanagerdata.Permissions.decode(reader, reader.uint32()); + if (message.srv_keyspaces === $util.emptyObject) + message.srv_keyspaces = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.topodata.SrvKeyspace.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.srv_keyspaces[key] = value; break; } default: @@ -116161,126 +122041,149 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetPermissionsResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse + * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetPermissionsResponse.decodeDelimited = function decodeDelimited(reader) { + GetSrvKeyspacesResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetPermissionsResponse message. + * Verifies a GetSrvKeyspacesResponse message. * @function verify - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetPermissionsResponse.verify = function verify(message) { + GetSrvKeyspacesResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.permissions != null && message.hasOwnProperty("permissions")) { - let error = $root.tabletmanagerdata.Permissions.verify(message.permissions); - if (error) - return "permissions." + error; + if (message.srv_keyspaces != null && message.hasOwnProperty("srv_keyspaces")) { + if (!$util.isObject(message.srv_keyspaces)) + return "srv_keyspaces: object expected"; + let key = Object.keys(message.srv_keyspaces); + for (let i = 0; i < key.length; ++i) { + let error = $root.topodata.SrvKeyspace.verify(message.srv_keyspaces[key[i]]); + if (error) + return "srv_keyspaces." + error; + } } return null; }; /** - * Creates a GetPermissionsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvKeyspacesResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetPermissionsResponse} GetPermissionsResponse + * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse */ - GetPermissionsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetPermissionsResponse) + GetSrvKeyspacesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvKeyspacesResponse) return object; - let message = new $root.vtctldata.GetPermissionsResponse(); - if (object.permissions != null) { - if (typeof object.permissions !== "object") - throw TypeError(".vtctldata.GetPermissionsResponse.permissions: object expected"); - message.permissions = $root.tabletmanagerdata.Permissions.fromObject(object.permissions); + let message = new $root.vtctldata.GetSrvKeyspacesResponse(); + if (object.srv_keyspaces) { + if (typeof object.srv_keyspaces !== "object") + throw TypeError(".vtctldata.GetSrvKeyspacesResponse.srv_keyspaces: object expected"); + message.srv_keyspaces = {}; + for (let keys = Object.keys(object.srv_keyspaces), i = 0; i < keys.length; ++i) { + if (typeof object.srv_keyspaces[keys[i]] !== "object") + throw TypeError(".vtctldata.GetSrvKeyspacesResponse.srv_keyspaces: object expected"); + message.srv_keyspaces[keys[i]] = $root.topodata.SrvKeyspace.fromObject(object.srv_keyspaces[keys[i]]); + } } return message; }; /** - * Creates a plain object from a GetPermissionsResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvKeyspacesResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static - * @param {vtctldata.GetPermissionsResponse} message GetPermissionsResponse + * @param {vtctldata.GetSrvKeyspacesResponse} message GetSrvKeyspacesResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetPermissionsResponse.toObject = function toObject(message, options) { + GetSrvKeyspacesResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.permissions = null; - if (message.permissions != null && message.hasOwnProperty("permissions")) - object.permissions = $root.tabletmanagerdata.Permissions.toObject(message.permissions, options); + if (options.objects || options.defaults) + object.srv_keyspaces = {}; + let keys2; + if (message.srv_keyspaces && (keys2 = Object.keys(message.srv_keyspaces)).length) { + object.srv_keyspaces = {}; + for (let j = 0; j < keys2.length; ++j) + object.srv_keyspaces[keys2[j]] = $root.topodata.SrvKeyspace.toObject(message.srv_keyspaces[keys2[j]], options); + } return object; }; /** - * Converts this GetPermissionsResponse to JSON. + * Converts this GetSrvKeyspacesResponse to JSON. * @function toJSON - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @instance * @returns {Object.} JSON object */ - GetPermissionsResponse.prototype.toJSON = function toJSON() { + GetSrvKeyspacesResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetPermissionsResponse + * Gets the default type url for GetSrvKeyspacesResponse * @function getTypeUrl - * @memberof vtctldata.GetPermissionsResponse + * @memberof vtctldata.GetSrvKeyspacesResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetPermissionsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvKeyspacesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetPermissionsResponse"; + return typeUrlPrefix + "/vtctldata.GetSrvKeyspacesResponse"; }; - return GetPermissionsResponse; + return GetSrvKeyspacesResponse; })(); - vtctldata.GetRoutingRulesRequest = (function() { + vtctldata.UpdateThrottlerConfigRequest = (function() { /** - * Properties of a GetRoutingRulesRequest. + * Properties of an UpdateThrottlerConfigRequest. * @memberof vtctldata - * @interface IGetRoutingRulesRequest + * @interface IUpdateThrottlerConfigRequest + * @property {string|null} [keyspace] UpdateThrottlerConfigRequest keyspace + * @property {boolean|null} [enable] UpdateThrottlerConfigRequest enable + * @property {boolean|null} [disable] UpdateThrottlerConfigRequest disable + * @property {number|null} [threshold] UpdateThrottlerConfigRequest threshold + * @property {string|null} [custom_query] UpdateThrottlerConfigRequest custom_query + * @property {boolean|null} [custom_query_set] UpdateThrottlerConfigRequest custom_query_set + * @property {boolean|null} [check_as_check_self] UpdateThrottlerConfigRequest check_as_check_self + * @property {boolean|null} [check_as_check_shard] UpdateThrottlerConfigRequest check_as_check_shard + * @property {topodata.IThrottledAppRule|null} [throttled_app] UpdateThrottlerConfigRequest throttled_app */ /** - * Constructs a new GetRoutingRulesRequest. + * Constructs a new UpdateThrottlerConfigRequest. * @memberof vtctldata - * @classdesc Represents a GetRoutingRulesRequest. - * @implements IGetRoutingRulesRequest + * @classdesc Represents an UpdateThrottlerConfigRequest. + * @implements IUpdateThrottlerConfigRequest * @constructor - * @param {vtctldata.IGetRoutingRulesRequest=} [properties] Properties to set + * @param {vtctldata.IUpdateThrottlerConfigRequest=} [properties] Properties to set */ - function GetRoutingRulesRequest(properties) { + function UpdateThrottlerConfigRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -116288,63 +122191,189 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new GetRoutingRulesRequest instance using the specified properties. + * UpdateThrottlerConfigRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.keyspace = ""; + + /** + * UpdateThrottlerConfigRequest enable. + * @member {boolean} enable + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.enable = false; + + /** + * UpdateThrottlerConfigRequest disable. + * @member {boolean} disable + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.disable = false; + + /** + * UpdateThrottlerConfigRequest threshold. + * @member {number} threshold + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.threshold = 0; + + /** + * UpdateThrottlerConfigRequest custom_query. + * @member {string} custom_query + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.custom_query = ""; + + /** + * UpdateThrottlerConfigRequest custom_query_set. + * @member {boolean} custom_query_set + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.custom_query_set = false; + + /** + * UpdateThrottlerConfigRequest check_as_check_self. + * @member {boolean} check_as_check_self + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.check_as_check_self = false; + + /** + * UpdateThrottlerConfigRequest check_as_check_shard. + * @member {boolean} check_as_check_shard + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.check_as_check_shard = false; + + /** + * UpdateThrottlerConfigRequest throttled_app. + * @member {topodata.IThrottledAppRule|null|undefined} throttled_app + * @memberof vtctldata.UpdateThrottlerConfigRequest + * @instance + */ + UpdateThrottlerConfigRequest.prototype.throttled_app = null; + + /** + * Creates a new UpdateThrottlerConfigRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static - * @param {vtctldata.IGetRoutingRulesRequest=} [properties] Properties to set - * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest instance + * @param {vtctldata.IUpdateThrottlerConfigRequest=} [properties] Properties to set + * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest instance */ - GetRoutingRulesRequest.create = function create(properties) { - return new GetRoutingRulesRequest(properties); + UpdateThrottlerConfigRequest.create = function create(properties) { + return new UpdateThrottlerConfigRequest(properties); }; /** - * Encodes the specified GetRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. + * Encodes the specified UpdateThrottlerConfigRequest message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static - * @param {vtctldata.IGetRoutingRulesRequest} message GetRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IUpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetRoutingRulesRequest.encode = function encode(message, writer) { + UpdateThrottlerConfigRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.enable != null && Object.hasOwnProperty.call(message, "enable")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.enable); + if (message.disable != null && Object.hasOwnProperty.call(message, "disable")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.disable); + if (message.threshold != null && Object.hasOwnProperty.call(message, "threshold")) + writer.uint32(/* id 4, wireType 1 =*/33).double(message.threshold); + if (message.custom_query != null && Object.hasOwnProperty.call(message, "custom_query")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.custom_query); + if (message.custom_query_set != null && Object.hasOwnProperty.call(message, "custom_query_set")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.custom_query_set); + if (message.check_as_check_self != null && Object.hasOwnProperty.call(message, "check_as_check_self")) + writer.uint32(/* id 7, wireType 0 =*/56).bool(message.check_as_check_self); + if (message.check_as_check_shard != null && Object.hasOwnProperty.call(message, "check_as_check_shard")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.check_as_check_shard); + if (message.throttled_app != null && Object.hasOwnProperty.call(message, "throttled_app")) + $root.topodata.ThrottledAppRule.encode(message.throttled_app, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesRequest.verify|verify} messages. + * Encodes the specified UpdateThrottlerConfigRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static - * @param {vtctldata.IGetRoutingRulesRequest} message GetRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IUpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + UpdateThrottlerConfigRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetRoutingRulesRequest message from the specified reader or buffer. + * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest + * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetRoutingRulesRequest.decode = function decode(reader, length) { + UpdateThrottlerConfigRequest.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.GetRoutingRulesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateThrottlerConfigRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.enable = reader.bool(); + break; + } + case 3: { + message.disable = reader.bool(); + break; + } + case 4: { + message.threshold = reader.double(); + break; + } + case 5: { + message.custom_query = reader.string(); + break; + } + case 6: { + message.custom_query_set = reader.bool(); + break; + } + case 7: { + message.check_as_check_self = reader.bool(); + break; + } + case 8: { + message.check_as_check_shard = reader.bool(); + break; + } + case 9: { + message.throttled_app = $root.topodata.ThrottledAppRule.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -116354,109 +122383,191 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest + * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + UpdateThrottlerConfigRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetRoutingRulesRequest message. + * Verifies an UpdateThrottlerConfigRequest message. * @function verify - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetRoutingRulesRequest.verify = function verify(message) { + UpdateThrottlerConfigRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.enable != null && message.hasOwnProperty("enable")) + if (typeof message.enable !== "boolean") + return "enable: boolean expected"; + if (message.disable != null && message.hasOwnProperty("disable")) + if (typeof message.disable !== "boolean") + return "disable: boolean expected"; + if (message.threshold != null && message.hasOwnProperty("threshold")) + if (typeof message.threshold !== "number") + return "threshold: number expected"; + if (message.custom_query != null && message.hasOwnProperty("custom_query")) + if (!$util.isString(message.custom_query)) + return "custom_query: string expected"; + if (message.custom_query_set != null && message.hasOwnProperty("custom_query_set")) + if (typeof message.custom_query_set !== "boolean") + return "custom_query_set: boolean expected"; + if (message.check_as_check_self != null && message.hasOwnProperty("check_as_check_self")) + if (typeof message.check_as_check_self !== "boolean") + return "check_as_check_self: boolean expected"; + if (message.check_as_check_shard != null && message.hasOwnProperty("check_as_check_shard")) + if (typeof message.check_as_check_shard !== "boolean") + return "check_as_check_shard: boolean expected"; + if (message.throttled_app != null && message.hasOwnProperty("throttled_app")) { + let error = $root.topodata.ThrottledAppRule.verify(message.throttled_app); + if (error) + return "throttled_app." + error; + } return null; }; /** - * Creates a GetRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateThrottlerConfigRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetRoutingRulesRequest} GetRoutingRulesRequest + * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest */ - GetRoutingRulesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetRoutingRulesRequest) + UpdateThrottlerConfigRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateThrottlerConfigRequest) return object; - return new $root.vtctldata.GetRoutingRulesRequest(); + let message = new $root.vtctldata.UpdateThrottlerConfigRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.enable != null) + message.enable = Boolean(object.enable); + if (object.disable != null) + message.disable = Boolean(object.disable); + if (object.threshold != null) + message.threshold = Number(object.threshold); + if (object.custom_query != null) + message.custom_query = String(object.custom_query); + if (object.custom_query_set != null) + message.custom_query_set = Boolean(object.custom_query_set); + if (object.check_as_check_self != null) + message.check_as_check_self = Boolean(object.check_as_check_self); + if (object.check_as_check_shard != null) + message.check_as_check_shard = Boolean(object.check_as_check_shard); + if (object.throttled_app != null) { + if (typeof object.throttled_app !== "object") + throw TypeError(".vtctldata.UpdateThrottlerConfigRequest.throttled_app: object expected"); + message.throttled_app = $root.topodata.ThrottledAppRule.fromObject(object.throttled_app); + } + return message; }; /** - * Creates a plain object from a GetRoutingRulesRequest message. Also converts values to other types if specified. + * Creates a plain object from an UpdateThrottlerConfigRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static - * @param {vtctldata.GetRoutingRulesRequest} message GetRoutingRulesRequest + * @param {vtctldata.UpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetRoutingRulesRequest.toObject = function toObject() { - return {}; + UpdateThrottlerConfigRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.keyspace = ""; + object.enable = false; + object.disable = false; + object.threshold = 0; + object.custom_query = ""; + object.custom_query_set = false; + object.check_as_check_self = false; + object.check_as_check_shard = false; + object.throttled_app = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.enable != null && message.hasOwnProperty("enable")) + object.enable = message.enable; + if (message.disable != null && message.hasOwnProperty("disable")) + object.disable = message.disable; + if (message.threshold != null && message.hasOwnProperty("threshold")) + object.threshold = options.json && !isFinite(message.threshold) ? String(message.threshold) : message.threshold; + if (message.custom_query != null && message.hasOwnProperty("custom_query")) + object.custom_query = message.custom_query; + if (message.custom_query_set != null && message.hasOwnProperty("custom_query_set")) + object.custom_query_set = message.custom_query_set; + if (message.check_as_check_self != null && message.hasOwnProperty("check_as_check_self")) + object.check_as_check_self = message.check_as_check_self; + if (message.check_as_check_shard != null && message.hasOwnProperty("check_as_check_shard")) + object.check_as_check_shard = message.check_as_check_shard; + if (message.throttled_app != null && message.hasOwnProperty("throttled_app")) + object.throttled_app = $root.topodata.ThrottledAppRule.toObject(message.throttled_app, options); + return object; }; /** - * Converts this GetRoutingRulesRequest to JSON. + * Converts this UpdateThrottlerConfigRequest to JSON. * @function toJSON - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @instance * @returns {Object.} JSON object */ - GetRoutingRulesRequest.prototype.toJSON = function toJSON() { + UpdateThrottlerConfigRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetRoutingRulesRequest + * Gets the default type url for UpdateThrottlerConfigRequest * @function getTypeUrl - * @memberof vtctldata.GetRoutingRulesRequest + * @memberof vtctldata.UpdateThrottlerConfigRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateThrottlerConfigRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetRoutingRulesRequest"; + return typeUrlPrefix + "/vtctldata.UpdateThrottlerConfigRequest"; }; - return GetRoutingRulesRequest; + return UpdateThrottlerConfigRequest; })(); - vtctldata.GetRoutingRulesResponse = (function() { + vtctldata.UpdateThrottlerConfigResponse = (function() { /** - * Properties of a GetRoutingRulesResponse. + * Properties of an UpdateThrottlerConfigResponse. * @memberof vtctldata - * @interface IGetRoutingRulesResponse - * @property {vschema.IRoutingRules|null} [routing_rules] GetRoutingRulesResponse routing_rules + * @interface IUpdateThrottlerConfigResponse */ /** - * Constructs a new GetRoutingRulesResponse. + * Constructs a new UpdateThrottlerConfigResponse. * @memberof vtctldata - * @classdesc Represents a GetRoutingRulesResponse. - * @implements IGetRoutingRulesResponse + * @classdesc Represents an UpdateThrottlerConfigResponse. + * @implements IUpdateThrottlerConfigResponse * @constructor - * @param {vtctldata.IGetRoutingRulesResponse=} [properties] Properties to set + * @param {vtctldata.IUpdateThrottlerConfigResponse=} [properties] Properties to set */ - function GetRoutingRulesResponse(properties) { + function UpdateThrottlerConfigResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -116464,77 +122575,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetRoutingRulesResponse routing_rules. - * @member {vschema.IRoutingRules|null|undefined} routing_rules - * @memberof vtctldata.GetRoutingRulesResponse - * @instance - */ - GetRoutingRulesResponse.prototype.routing_rules = null; - - /** - * Creates a new GetRoutingRulesResponse instance using the specified properties. + * Creates a new UpdateThrottlerConfigResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static - * @param {vtctldata.IGetRoutingRulesResponse=} [properties] Properties to set - * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse instance + * @param {vtctldata.IUpdateThrottlerConfigResponse=} [properties] Properties to set + * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse instance */ - GetRoutingRulesResponse.create = function create(properties) { - return new GetRoutingRulesResponse(properties); + UpdateThrottlerConfigResponse.create = function create(properties) { + return new UpdateThrottlerConfigResponse(properties); }; /** - * Encodes the specified GetRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. + * Encodes the specified UpdateThrottlerConfigResponse message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static - * @param {vtctldata.IGetRoutingRulesResponse} message GetRoutingRulesResponse message or plain object to encode + * @param {vtctldata.IUpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetRoutingRulesResponse.encode = function encode(message, writer) { + UpdateThrottlerConfigResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.routing_rules != null && Object.hasOwnProperty.call(message, "routing_rules")) - $root.vschema.RoutingRules.encode(message.routing_rules, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetRoutingRulesResponse.verify|verify} messages. + * Encodes the specified UpdateThrottlerConfigResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static - * @param {vtctldata.IGetRoutingRulesResponse} message GetRoutingRulesResponse message or plain object to encode + * @param {vtctldata.IUpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + UpdateThrottlerConfigResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetRoutingRulesResponse message from the specified reader or buffer. + * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse + * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetRoutingRulesResponse.decode = function decode(reader, length) { + UpdateThrottlerConfigResponse.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.GetRoutingRulesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateThrottlerConfigResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.routing_rules = $root.vschema.RoutingRules.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -116544,135 +122641,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse + * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + UpdateThrottlerConfigResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetRoutingRulesResponse message. + * Verifies an UpdateThrottlerConfigResponse message. * @function verify - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetRoutingRulesResponse.verify = function verify(message) { + UpdateThrottlerConfigResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) { - let error = $root.vschema.RoutingRules.verify(message.routing_rules); - if (error) - return "routing_rules." + error; - } return null; }; /** - * Creates a GetRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateThrottlerConfigResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetRoutingRulesResponse} GetRoutingRulesResponse + * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse */ - GetRoutingRulesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetRoutingRulesResponse) + UpdateThrottlerConfigResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateThrottlerConfigResponse) return object; - let message = new $root.vtctldata.GetRoutingRulesResponse(); - if (object.routing_rules != null) { - if (typeof object.routing_rules !== "object") - throw TypeError(".vtctldata.GetRoutingRulesResponse.routing_rules: object expected"); - message.routing_rules = $root.vschema.RoutingRules.fromObject(object.routing_rules); - } - return message; + return new $root.vtctldata.UpdateThrottlerConfigResponse(); }; /** - * Creates a plain object from a GetRoutingRulesResponse message. Also converts values to other types if specified. + * Creates a plain object from an UpdateThrottlerConfigResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static - * @param {vtctldata.GetRoutingRulesResponse} message GetRoutingRulesResponse + * @param {vtctldata.UpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetRoutingRulesResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.routing_rules = null; - if (message.routing_rules != null && message.hasOwnProperty("routing_rules")) - object.routing_rules = $root.vschema.RoutingRules.toObject(message.routing_rules, options); - return object; + UpdateThrottlerConfigResponse.toObject = function toObject() { + return {}; }; /** - * Converts this GetRoutingRulesResponse to JSON. + * Converts this UpdateThrottlerConfigResponse to JSON. * @function toJSON - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @instance * @returns {Object.} JSON object */ - GetRoutingRulesResponse.prototype.toJSON = function toJSON() { + UpdateThrottlerConfigResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetRoutingRulesResponse + * Gets the default type url for UpdateThrottlerConfigResponse * @function getTypeUrl - * @memberof vtctldata.GetRoutingRulesResponse + * @memberof vtctldata.UpdateThrottlerConfigResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateThrottlerConfigResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetRoutingRulesResponse"; + return typeUrlPrefix + "/vtctldata.UpdateThrottlerConfigResponse"; }; - return GetRoutingRulesResponse; + return UpdateThrottlerConfigResponse; })(); - vtctldata.GetSchemaRequest = (function() { + vtctldata.GetSrvVSchemaRequest = (function() { /** - * Properties of a GetSchemaRequest. + * Properties of a GetSrvVSchemaRequest. * @memberof vtctldata - * @interface IGetSchemaRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] GetSchemaRequest tablet_alias - * @property {Array.|null} [tables] GetSchemaRequest tables - * @property {Array.|null} [exclude_tables] GetSchemaRequest exclude_tables - * @property {boolean|null} [include_views] GetSchemaRequest include_views - * @property {boolean|null} [table_names_only] GetSchemaRequest table_names_only - * @property {boolean|null} [table_sizes_only] GetSchemaRequest table_sizes_only - * @property {boolean|null} [table_schema_only] GetSchemaRequest table_schema_only + * @interface IGetSrvVSchemaRequest + * @property {string|null} [cell] GetSrvVSchemaRequest cell */ /** - * Constructs a new GetSchemaRequest. + * Constructs a new GetSrvVSchemaRequest. * @memberof vtctldata - * @classdesc Represents a GetSchemaRequest. - * @implements IGetSchemaRequest + * @classdesc Represents a GetSrvVSchemaRequest. + * @implements IGetSrvVSchemaRequest * @constructor - * @param {vtctldata.IGetSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IGetSrvVSchemaRequest=} [properties] Properties to set */ - function GetSchemaRequest(properties) { - this.tables = []; - this.exclude_tables = []; + function GetSrvVSchemaRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -116680,165 +122751,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSchemaRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.tablet_alias = null; - - /** - * GetSchemaRequest tables. - * @member {Array.} tables - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.tables = $util.emptyArray; - - /** - * GetSchemaRequest exclude_tables. - * @member {Array.} exclude_tables - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.exclude_tables = $util.emptyArray; - - /** - * GetSchemaRequest include_views. - * @member {boolean} include_views - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.include_views = false; - - /** - * GetSchemaRequest table_names_only. - * @member {boolean} table_names_only - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.table_names_only = false; - - /** - * GetSchemaRequest table_sizes_only. - * @member {boolean} table_sizes_only - * @memberof vtctldata.GetSchemaRequest - * @instance - */ - GetSchemaRequest.prototype.table_sizes_only = false; - - /** - * GetSchemaRequest table_schema_only. - * @member {boolean} table_schema_only - * @memberof vtctldata.GetSchemaRequest + * GetSrvVSchemaRequest cell. + * @member {string} cell + * @memberof vtctldata.GetSrvVSchemaRequest * @instance */ - GetSchemaRequest.prototype.table_schema_only = false; + GetSrvVSchemaRequest.prototype.cell = ""; /** - * Creates a new GetSchemaRequest instance using the specified properties. + * Creates a new GetSrvVSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static - * @param {vtctldata.IGetSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest instance + * @param {vtctldata.IGetSrvVSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest instance */ - GetSchemaRequest.create = function create(properties) { - return new GetSchemaRequest(properties); + GetSrvVSchemaRequest.create = function create(properties) { + return new GetSrvVSchemaRequest(properties); }; /** - * Encodes the specified GetSchemaRequest message. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. + * Encodes the specified GetSrvVSchemaRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static - * @param {vtctldata.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode + * @param {vtctldata.IGetSrvVSchemaRequest} message GetSrvVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaRequest.encode = function encode(message, writer) { + GetSrvVSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.tables[i]); - if (message.exclude_tables != null && message.exclude_tables.length) - for (let i = 0; i < message.exclude_tables.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.exclude_tables[i]); - if (message.include_views != null && Object.hasOwnProperty.call(message, "include_views")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_views); - if (message.table_names_only != null && Object.hasOwnProperty.call(message, "table_names_only")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.table_names_only); - if (message.table_sizes_only != null && Object.hasOwnProperty.call(message, "table_sizes_only")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.table_sizes_only); - if (message.table_schema_only != null && Object.hasOwnProperty.call(message, "table_schema_only")) - writer.uint32(/* id 7, wireType 0 =*/56).bool(message.table_schema_only); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); return writer; }; /** - * Encodes the specified GetSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSchemaRequest.verify|verify} messages. + * Encodes the specified GetSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static - * @param {vtctldata.IGetSchemaRequest} message GetSchemaRequest message or plain object to encode + * @param {vtctldata.IGetSrvVSchemaRequest} message GetSrvVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer. + * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest + * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaRequest.decode = function decode(reader, length) { + GetSrvVSchemaRequest.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.GetSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvVSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); - break; - } - case 3: { - if (!(message.exclude_tables && message.exclude_tables.length)) - message.exclude_tables = []; - message.exclude_tables.push(reader.string()); - break; - } - case 4: { - message.include_views = reader.bool(); - break; - } - case 5: { - message.table_names_only = reader.bool(); - break; - } - case 6: { - message.table_sizes_only = reader.bool(); - break; - } - case 7: { - message.table_schema_only = reader.bool(); + message.cell = reader.string(); break; } default: @@ -116850,202 +122831,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest + * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + GetSrvVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSchemaRequest message. + * Verifies a GetSrvVSchemaRequest message. * @function verify - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSchemaRequest.verify = function verify(message) { + GetSrvVSchemaRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; - } - if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { - if (!Array.isArray(message.exclude_tables)) - return "exclude_tables: array expected"; - for (let i = 0; i < message.exclude_tables.length; ++i) - if (!$util.isString(message.exclude_tables[i])) - return "exclude_tables: string[] expected"; - } - if (message.include_views != null && message.hasOwnProperty("include_views")) - if (typeof message.include_views !== "boolean") - return "include_views: boolean expected"; - if (message.table_names_only != null && message.hasOwnProperty("table_names_only")) - if (typeof message.table_names_only !== "boolean") - return "table_names_only: boolean expected"; - if (message.table_sizes_only != null && message.hasOwnProperty("table_sizes_only")) - if (typeof message.table_sizes_only !== "boolean") - return "table_sizes_only: boolean expected"; - if (message.table_schema_only != null && message.hasOwnProperty("table_schema_only")) - if (typeof message.table_schema_only !== "boolean") - return "table_schema_only: boolean expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; return null; }; /** - * Creates a GetSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSchemaRequest} GetSchemaRequest + * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest */ - GetSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSchemaRequest) + GetSrvVSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvVSchemaRequest) return object; - let message = new $root.vtctldata.GetSchemaRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.GetSchemaRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".vtctldata.GetSchemaRequest.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); - } - if (object.exclude_tables) { - if (!Array.isArray(object.exclude_tables)) - throw TypeError(".vtctldata.GetSchemaRequest.exclude_tables: array expected"); - message.exclude_tables = []; - for (let i = 0; i < object.exclude_tables.length; ++i) - message.exclude_tables[i] = String(object.exclude_tables[i]); - } - if (object.include_views != null) - message.include_views = Boolean(object.include_views); - if (object.table_names_only != null) - message.table_names_only = Boolean(object.table_names_only); - if (object.table_sizes_only != null) - message.table_sizes_only = Boolean(object.table_sizes_only); - if (object.table_schema_only != null) - message.table_schema_only = Boolean(object.table_schema_only); + let message = new $root.vtctldata.GetSrvVSchemaRequest(); + if (object.cell != null) + message.cell = String(object.cell); return message; }; /** - * Creates a plain object from a GetSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvVSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static - * @param {vtctldata.GetSchemaRequest} message GetSchemaRequest + * @param {vtctldata.GetSrvVSchemaRequest} message GetSrvVSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSchemaRequest.toObject = function toObject(message, options) { + GetSrvVSchemaRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.tables = []; - object.exclude_tables = []; - } - if (options.defaults) { - object.tablet_alias = null; - object.include_views = false; - object.table_names_only = false; - object.table_sizes_only = false; - object.table_schema_only = false; - } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; - } - if (message.exclude_tables && message.exclude_tables.length) { - object.exclude_tables = []; - for (let j = 0; j < message.exclude_tables.length; ++j) - object.exclude_tables[j] = message.exclude_tables[j]; - } - if (message.include_views != null && message.hasOwnProperty("include_views")) - object.include_views = message.include_views; - if (message.table_names_only != null && message.hasOwnProperty("table_names_only")) - object.table_names_only = message.table_names_only; - if (message.table_sizes_only != null && message.hasOwnProperty("table_sizes_only")) - object.table_sizes_only = message.table_sizes_only; - if (message.table_schema_only != null && message.hasOwnProperty("table_schema_only")) - object.table_schema_only = message.table_schema_only; + if (options.defaults) + object.cell = ""; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; return object; }; /** - * Converts this GetSchemaRequest to JSON. + * Converts this GetSrvVSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @instance * @returns {Object.} JSON object */ - GetSchemaRequest.prototype.toJSON = function toJSON() { + GetSrvVSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSchemaRequest + * Gets the default type url for GetSrvVSchemaRequest * @function getTypeUrl - * @memberof vtctldata.GetSchemaRequest + * @memberof vtctldata.GetSrvVSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSchemaRequest"; + return typeUrlPrefix + "/vtctldata.GetSrvVSchemaRequest"; }; - return GetSchemaRequest; + return GetSrvVSchemaRequest; })(); - vtctldata.GetSchemaResponse = (function() { + vtctldata.GetSrvVSchemaResponse = (function() { /** - * Properties of a GetSchemaResponse. + * Properties of a GetSrvVSchemaResponse. * @memberof vtctldata - * @interface IGetSchemaResponse - * @property {tabletmanagerdata.ISchemaDefinition|null} [schema] GetSchemaResponse schema + * @interface IGetSrvVSchemaResponse + * @property {vschema.ISrvVSchema|null} [srv_v_schema] GetSrvVSchemaResponse srv_v_schema */ /** - * Constructs a new GetSchemaResponse. + * Constructs a new GetSrvVSchemaResponse. * @memberof vtctldata - * @classdesc Represents a GetSchemaResponse. - * @implements IGetSchemaResponse + * @classdesc Represents a GetSrvVSchemaResponse. + * @implements IGetSrvVSchemaResponse * @constructor - * @param {vtctldata.IGetSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IGetSrvVSchemaResponse=} [properties] Properties to set */ - function GetSchemaResponse(properties) { + function GetSrvVSchemaResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -117053,75 +122954,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSchemaResponse schema. - * @member {tabletmanagerdata.ISchemaDefinition|null|undefined} schema - * @memberof vtctldata.GetSchemaResponse + * GetSrvVSchemaResponse srv_v_schema. + * @member {vschema.ISrvVSchema|null|undefined} srv_v_schema + * @memberof vtctldata.GetSrvVSchemaResponse * @instance */ - GetSchemaResponse.prototype.schema = null; + GetSrvVSchemaResponse.prototype.srv_v_schema = null; /** - * Creates a new GetSchemaResponse instance using the specified properties. + * Creates a new GetSrvVSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static - * @param {vtctldata.IGetSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse instance + * @param {vtctldata.IGetSrvVSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse instance */ - GetSchemaResponse.create = function create(properties) { - return new GetSchemaResponse(properties); + GetSrvVSchemaResponse.create = function create(properties) { + return new GetSrvVSchemaResponse(properties); }; /** - * Encodes the specified GetSchemaResponse message. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. + * Encodes the specified GetSrvVSchemaResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static - * @param {vtctldata.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode + * @param {vtctldata.IGetSrvVSchemaResponse} message GetSrvVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaResponse.encode = function encode(message, writer) { + GetSrvVSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.schema != null && Object.hasOwnProperty.call(message, "schema")) - $root.tabletmanagerdata.SchemaDefinition.encode(message.schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.srv_v_schema != null && Object.hasOwnProperty.call(message, "srv_v_schema")) + $root.vschema.SrvVSchema.encode(message.srv_v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSchemaResponse.verify|verify} messages. + * Encodes the specified GetSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static - * @param {vtctldata.IGetSchemaResponse} message GetSchemaResponse message or plain object to encode + * @param {vtctldata.IGetSrvVSchemaResponse} message GetSrvVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer. + * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse + * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaResponse.decode = function decode(reader, length) { + GetSrvVSchemaResponse.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.GetSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvVSchemaResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.schema = $root.tabletmanagerdata.SchemaDefinition.decode(reader, reader.uint32()); + message.srv_v_schema = $root.vschema.SrvVSchema.decode(reader, reader.uint32()); break; } default: @@ -117133,128 +123034,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse + * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + GetSrvVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSchemaResponse message. + * Verifies a GetSrvVSchemaResponse message. * @function verify - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSchemaResponse.verify = function verify(message) { + GetSrvVSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.schema != null && message.hasOwnProperty("schema")) { - let error = $root.tabletmanagerdata.SchemaDefinition.verify(message.schema); + if (message.srv_v_schema != null && message.hasOwnProperty("srv_v_schema")) { + let error = $root.vschema.SrvVSchema.verify(message.srv_v_schema); if (error) - return "schema." + error; + return "srv_v_schema." + error; } return null; }; /** - * Creates a GetSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSchemaResponse} GetSchemaResponse + * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse */ - GetSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSchemaResponse) + GetSrvVSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvVSchemaResponse) return object; - let message = new $root.vtctldata.GetSchemaResponse(); - if (object.schema != null) { - if (typeof object.schema !== "object") - throw TypeError(".vtctldata.GetSchemaResponse.schema: object expected"); - message.schema = $root.tabletmanagerdata.SchemaDefinition.fromObject(object.schema); + let message = new $root.vtctldata.GetSrvVSchemaResponse(); + if (object.srv_v_schema != null) { + if (typeof object.srv_v_schema !== "object") + throw TypeError(".vtctldata.GetSrvVSchemaResponse.srv_v_schema: object expected"); + message.srv_v_schema = $root.vschema.SrvVSchema.fromObject(object.srv_v_schema); } return message; }; /** - * Creates a plain object from a GetSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvVSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static - * @param {vtctldata.GetSchemaResponse} message GetSchemaResponse + * @param {vtctldata.GetSrvVSchemaResponse} message GetSrvVSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSchemaResponse.toObject = function toObject(message, options) { + GetSrvVSchemaResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.schema = null; - if (message.schema != null && message.hasOwnProperty("schema")) - object.schema = $root.tabletmanagerdata.SchemaDefinition.toObject(message.schema, options); + object.srv_v_schema = null; + if (message.srv_v_schema != null && message.hasOwnProperty("srv_v_schema")) + object.srv_v_schema = $root.vschema.SrvVSchema.toObject(message.srv_v_schema, options); return object; }; /** - * Converts this GetSchemaResponse to JSON. + * Converts this GetSrvVSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @instance * @returns {Object.} JSON object */ - GetSchemaResponse.prototype.toJSON = function toJSON() { + GetSrvVSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSchemaResponse + * Gets the default type url for GetSrvVSchemaResponse * @function getTypeUrl - * @memberof vtctldata.GetSchemaResponse + * @memberof vtctldata.GetSrvVSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSchemaResponse"; + return typeUrlPrefix + "/vtctldata.GetSrvVSchemaResponse"; }; - return GetSchemaResponse; + return GetSrvVSchemaResponse; })(); - vtctldata.GetShardRequest = (function() { + vtctldata.GetSrvVSchemasRequest = (function() { /** - * Properties of a GetShardRequest. + * Properties of a GetSrvVSchemasRequest. * @memberof vtctldata - * @interface IGetShardRequest - * @property {string|null} [keyspace] GetShardRequest keyspace - * @property {string|null} [shard_name] GetShardRequest shard_name + * @interface IGetSrvVSchemasRequest + * @property {Array.|null} [cells] GetSrvVSchemasRequest cells */ /** - * Constructs a new GetShardRequest. + * Constructs a new GetSrvVSchemasRequest. * @memberof vtctldata - * @classdesc Represents a GetShardRequest. - * @implements IGetShardRequest + * @classdesc Represents a GetSrvVSchemasRequest. + * @implements IGetSrvVSchemasRequest * @constructor - * @param {vtctldata.IGetShardRequest=} [properties] Properties to set + * @param {vtctldata.IGetSrvVSchemasRequest=} [properties] Properties to set */ - function GetShardRequest(properties) { + function GetSrvVSchemasRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -117262,89 +123163,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.GetShardRequest - * @instance - */ - GetShardRequest.prototype.keyspace = ""; - - /** - * GetShardRequest shard_name. - * @member {string} shard_name - * @memberof vtctldata.GetShardRequest + * GetSrvVSchemasRequest cells. + * @member {Array.} cells + * @memberof vtctldata.GetSrvVSchemasRequest * @instance */ - GetShardRequest.prototype.shard_name = ""; + GetSrvVSchemasRequest.prototype.cells = $util.emptyArray; /** - * Creates a new GetShardRequest instance using the specified properties. + * Creates a new GetSrvVSchemasRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static - * @param {vtctldata.IGetShardRequest=} [properties] Properties to set - * @returns {vtctldata.GetShardRequest} GetShardRequest instance + * @param {vtctldata.IGetSrvVSchemasRequest=} [properties] Properties to set + * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest instance */ - GetShardRequest.create = function create(properties) { - return new GetShardRequest(properties); + GetSrvVSchemasRequest.create = function create(properties) { + return new GetSrvVSchemasRequest(properties); }; /** - * Encodes the specified GetShardRequest message. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. + * Encodes the specified GetSrvVSchemasRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static - * @param {vtctldata.IGetShardRequest} message GetShardRequest message or plain object to encode + * @param {vtctldata.IGetSrvVSchemasRequest} message GetSrvVSchemasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRequest.encode = function encode(message, writer) { + GetSrvVSchemasRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); return writer; }; /** - * Encodes the specified GetShardRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRequest.verify|verify} messages. + * Encodes the specified GetSrvVSchemasRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static - * @param {vtctldata.IGetShardRequest} message GetShardRequest message or plain object to encode + * @param {vtctldata.IGetSrvVSchemasRequest} message GetSrvVSchemasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvVSchemasRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetShardRequest message from the specified reader or buffer. + * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetShardRequest} GetShardRequest + * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRequest.decode = function decode(reader, length) { + GetSrvVSchemasRequest.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.GetShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvVSchemasRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } case 2: { - message.shard_name = reader.string(); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } default: @@ -117356,131 +123246,135 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetShardRequest message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetShardRequest} GetShardRequest + * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRequest.decodeDelimited = function decodeDelimited(reader) { + GetSrvVSchemasRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetShardRequest message. + * Verifies a GetSrvVSchemasRequest message. * @function verify - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetShardRequest.verify = function verify(message) { + GetSrvVSchemasRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - if (!$util.isString(message.shard_name)) - return "shard_name: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } return null; }; /** - * Creates a GetShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemasRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetShardRequest} GetShardRequest + * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest */ - GetShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetShardRequest) + GetSrvVSchemasRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvVSchemasRequest) return object; - let message = new $root.vtctldata.GetShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard_name != null) - message.shard_name = String(object.shard_name); + let message = new $root.vtctldata.GetSrvVSchemasRequest(); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.GetSrvVSchemasRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } return message; }; /** - * Creates a plain object from a GetShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvVSchemasRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static - * @param {vtctldata.GetShardRequest} message GetShardRequest + * @param {vtctldata.GetSrvVSchemasRequest} message GetSrvVSchemasRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetShardRequest.toObject = function toObject(message, options) { + GetSrvVSchemasRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard_name = ""; + if (options.arrays || options.defaults) + object.cells = []; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - object.shard_name = message.shard_name; return object; }; /** - * Converts this GetShardRequest to JSON. + * Converts this GetSrvVSchemasRequest to JSON. * @function toJSON - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @instance * @returns {Object.} JSON object */ - GetShardRequest.prototype.toJSON = function toJSON() { + GetSrvVSchemasRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetShardRequest + * Gets the default type url for GetSrvVSchemasRequest * @function getTypeUrl - * @memberof vtctldata.GetShardRequest + * @memberof vtctldata.GetSrvVSchemasRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvVSchemasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetShardRequest"; + return typeUrlPrefix + "/vtctldata.GetSrvVSchemasRequest"; }; - return GetShardRequest; + return GetSrvVSchemasRequest; })(); - vtctldata.GetShardResponse = (function() { + vtctldata.GetSrvVSchemasResponse = (function() { /** - * Properties of a GetShardResponse. + * Properties of a GetSrvVSchemasResponse. * @memberof vtctldata - * @interface IGetShardResponse - * @property {vtctldata.IShard|null} [shard] GetShardResponse shard + * @interface IGetSrvVSchemasResponse + * @property {Object.|null} [srv_v_schemas] GetSrvVSchemasResponse srv_v_schemas */ /** - * Constructs a new GetShardResponse. + * Constructs a new GetSrvVSchemasResponse. * @memberof vtctldata - * @classdesc Represents a GetShardResponse. - * @implements IGetShardResponse + * @classdesc Represents a GetSrvVSchemasResponse. + * @implements IGetSrvVSchemasResponse * @constructor - * @param {vtctldata.IGetShardResponse=} [properties] Properties to set + * @param {vtctldata.IGetSrvVSchemasResponse=} [properties] Properties to set */ - function GetShardResponse(properties) { + function GetSrvVSchemasResponse(properties) { + this.srv_v_schemas = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -117488,75 +123382,97 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetShardResponse shard. - * @member {vtctldata.IShard|null|undefined} shard - * @memberof vtctldata.GetShardResponse + * GetSrvVSchemasResponse srv_v_schemas. + * @member {Object.} srv_v_schemas + * @memberof vtctldata.GetSrvVSchemasResponse * @instance */ - GetShardResponse.prototype.shard = null; + GetSrvVSchemasResponse.prototype.srv_v_schemas = $util.emptyObject; /** - * Creates a new GetShardResponse instance using the specified properties. + * Creates a new GetSrvVSchemasResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static - * @param {vtctldata.IGetShardResponse=} [properties] Properties to set - * @returns {vtctldata.GetShardResponse} GetShardResponse instance + * @param {vtctldata.IGetSrvVSchemasResponse=} [properties] Properties to set + * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse instance */ - GetShardResponse.create = function create(properties) { - return new GetShardResponse(properties); + GetSrvVSchemasResponse.create = function create(properties) { + return new GetSrvVSchemasResponse(properties); }; /** - * Encodes the specified GetShardResponse message. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. + * Encodes the specified GetSrvVSchemasResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static - * @param {vtctldata.IGetShardResponse} message GetShardResponse message or plain object to encode + * @param {vtctldata.IGetSrvVSchemasResponse} message GetSrvVSchemasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardResponse.encode = function encode(message, writer) { + GetSrvVSchemasResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.vtctldata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.srv_v_schemas != null && Object.hasOwnProperty.call(message, "srv_v_schemas")) + for (let keys = Object.keys(message.srv_v_schemas), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vschema.SrvVSchema.encode(message.srv_v_schemas[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified GetShardResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardResponse.verify|verify} messages. + * Encodes the specified GetSrvVSchemasResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static - * @param {vtctldata.IGetShardResponse} message GetShardResponse message or plain object to encode + * @param {vtctldata.IGetSrvVSchemasResponse} message GetSrvVSchemasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetSrvVSchemasResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetShardResponse message from the specified reader or buffer. + * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetShardResponse} GetShardResponse + * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardResponse.decode = function decode(reader, length) { + GetSrvVSchemasResponse.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.GetShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetSrvVSchemasResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard = $root.vtctldata.Shard.decode(reader, reader.uint32()); + if (message.srv_v_schemas === $util.emptyObject) + message.srv_v_schemas = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vschema.SrvVSchema.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.srv_v_schemas[key] = value; break; } default: @@ -117568,126 +123484,141 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetShardResponse message from the specified reader or buffer, length delimited. + * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetShardResponse} GetShardResponse + * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardResponse.decodeDelimited = function decodeDelimited(reader) { + GetSrvVSchemasResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetShardResponse message. + * Verifies a GetSrvVSchemasResponse message. * @function verify - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetShardResponse.verify = function verify(message) { + GetSrvVSchemasResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.vtctldata.Shard.verify(message.shard); - if (error) - return "shard." + error; + if (message.srv_v_schemas != null && message.hasOwnProperty("srv_v_schemas")) { + if (!$util.isObject(message.srv_v_schemas)) + return "srv_v_schemas: object expected"; + let key = Object.keys(message.srv_v_schemas); + for (let i = 0; i < key.length; ++i) { + let error = $root.vschema.SrvVSchema.verify(message.srv_v_schemas[key[i]]); + if (error) + return "srv_v_schemas." + error; + } } return null; }; /** - * Creates a GetShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetSrvVSchemasResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetShardResponse} GetShardResponse + * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse */ - GetShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetShardResponse) + GetSrvVSchemasResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetSrvVSchemasResponse) return object; - let message = new $root.vtctldata.GetShardResponse(); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.GetShardResponse.shard: object expected"); - message.shard = $root.vtctldata.Shard.fromObject(object.shard); + let message = new $root.vtctldata.GetSrvVSchemasResponse(); + if (object.srv_v_schemas) { + if (typeof object.srv_v_schemas !== "object") + throw TypeError(".vtctldata.GetSrvVSchemasResponse.srv_v_schemas: object expected"); + message.srv_v_schemas = {}; + for (let keys = Object.keys(object.srv_v_schemas), i = 0; i < keys.length; ++i) { + if (typeof object.srv_v_schemas[keys[i]] !== "object") + throw TypeError(".vtctldata.GetSrvVSchemasResponse.srv_v_schemas: object expected"); + message.srv_v_schemas[keys[i]] = $root.vschema.SrvVSchema.fromObject(object.srv_v_schemas[keys[i]]); + } } return message; }; /** - * Creates a plain object from a GetShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetSrvVSchemasResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static - * @param {vtctldata.GetShardResponse} message GetShardResponse + * @param {vtctldata.GetSrvVSchemasResponse} message GetSrvVSchemasResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetShardResponse.toObject = function toObject(message, options) { + GetSrvVSchemasResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.shard = null; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.vtctldata.Shard.toObject(message.shard, options); + if (options.objects || options.defaults) + object.srv_v_schemas = {}; + let keys2; + if (message.srv_v_schemas && (keys2 = Object.keys(message.srv_v_schemas)).length) { + object.srv_v_schemas = {}; + for (let j = 0; j < keys2.length; ++j) + object.srv_v_schemas[keys2[j]] = $root.vschema.SrvVSchema.toObject(message.srv_v_schemas[keys2[j]], options); + } return object; }; /** - * Converts this GetShardResponse to JSON. + * Converts this GetSrvVSchemasResponse to JSON. * @function toJSON - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @instance * @returns {Object.} JSON object */ - GetShardResponse.prototype.toJSON = function toJSON() { + GetSrvVSchemasResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetShardResponse + * Gets the default type url for GetSrvVSchemasResponse * @function getTypeUrl - * @memberof vtctldata.GetShardResponse + * @memberof vtctldata.GetSrvVSchemasResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetSrvVSchemasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetShardResponse"; + return typeUrlPrefix + "/vtctldata.GetSrvVSchemasResponse"; }; - return GetShardResponse; + return GetSrvVSchemasResponse; })(); - vtctldata.GetShardRoutingRulesRequest = (function() { + vtctldata.GetTabletRequest = (function() { /** - * Properties of a GetShardRoutingRulesRequest. + * Properties of a GetTabletRequest. * @memberof vtctldata - * @interface IGetShardRoutingRulesRequest + * @interface IGetTabletRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] GetTabletRequest tablet_alias */ /** - * Constructs a new GetShardRoutingRulesRequest. + * Constructs a new GetTabletRequest. * @memberof vtctldata - * @classdesc Represents a GetShardRoutingRulesRequest. - * @implements IGetShardRoutingRulesRequest + * @classdesc Represents a GetTabletRequest. + * @implements IGetTabletRequest * @constructor - * @param {vtctldata.IGetShardRoutingRulesRequest=} [properties] Properties to set + * @param {vtctldata.IGetTabletRequest=} [properties] Properties to set */ - function GetShardRoutingRulesRequest(properties) { + function GetTabletRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -117695,63 +123626,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new GetShardRoutingRulesRequest instance using the specified properties. + * GetTabletRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.GetTabletRequest + * @instance + */ + GetTabletRequest.prototype.tablet_alias = null; + + /** + * Creates a new GetTabletRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static - * @param {vtctldata.IGetShardRoutingRulesRequest=} [properties] Properties to set - * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest instance + * @param {vtctldata.IGetTabletRequest=} [properties] Properties to set + * @returns {vtctldata.GetTabletRequest} GetTabletRequest instance */ - GetShardRoutingRulesRequest.create = function create(properties) { - return new GetShardRoutingRulesRequest(properties); + GetTabletRequest.create = function create(properties) { + return new GetTabletRequest(properties); }; /** - * Encodes the specified GetShardRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. + * Encodes the specified GetTabletRequest message. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static - * @param {vtctldata.IGetShardRoutingRulesRequest} message GetShardRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IGetTabletRequest} message GetTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRoutingRulesRequest.encode = function encode(message, writer) { + GetTabletRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetShardRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesRequest.verify|verify} messages. + * Encodes the specified GetTabletRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static - * @param {vtctldata.IGetShardRoutingRulesRequest} message GetShardRoutingRulesRequest message or plain object to encode + * @param {vtctldata.IGetTabletRequest} message GetTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer. + * Decodes a GetTabletRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest + * @returns {vtctldata.GetTabletRequest} GetTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRoutingRulesRequest.decode = function decode(reader, length) { + GetTabletRequest.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.GetShardRoutingRulesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTabletRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -117761,109 +123706,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetShardRoutingRulesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTabletRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest + * @returns {vtctldata.GetTabletRequest} GetTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + GetTabletRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetShardRoutingRulesRequest message. + * Verifies a GetTabletRequest message. * @function verify - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetShardRoutingRulesRequest.verify = function verify(message) { + GetTabletRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a GetShardRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetShardRoutingRulesRequest} GetShardRoutingRulesRequest + * @returns {vtctldata.GetTabletRequest} GetTabletRequest */ - GetShardRoutingRulesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetShardRoutingRulesRequest) + GetTabletRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTabletRequest) return object; - return new $root.vtctldata.GetShardRoutingRulesRequest(); + let message = new $root.vtctldata.GetTabletRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.GetTabletRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + return message; }; /** - * Creates a plain object from a GetShardRoutingRulesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetTabletRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static - * @param {vtctldata.GetShardRoutingRulesRequest} message GetShardRoutingRulesRequest + * @param {vtctldata.GetTabletRequest} message GetTabletRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetShardRoutingRulesRequest.toObject = function toObject() { - return {}; + GetTabletRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + return object; }; /** - * Converts this GetShardRoutingRulesRequest to JSON. + * Converts this GetTabletRequest to JSON. * @function toJSON - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @instance * @returns {Object.} JSON object */ - GetShardRoutingRulesRequest.prototype.toJSON = function toJSON() { + GetTabletRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetShardRoutingRulesRequest + * Gets the default type url for GetTabletRequest * @function getTypeUrl - * @memberof vtctldata.GetShardRoutingRulesRequest + * @memberof vtctldata.GetTabletRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetShardRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetShardRoutingRulesRequest"; + return typeUrlPrefix + "/vtctldata.GetTabletRequest"; }; - return GetShardRoutingRulesRequest; + return GetTabletRequest; })(); - vtctldata.GetShardRoutingRulesResponse = (function() { + vtctldata.GetTabletResponse = (function() { /** - * Properties of a GetShardRoutingRulesResponse. + * Properties of a GetTabletResponse. * @memberof vtctldata - * @interface IGetShardRoutingRulesResponse - * @property {vschema.IShardRoutingRules|null} [shard_routing_rules] GetShardRoutingRulesResponse shard_routing_rules + * @interface IGetTabletResponse + * @property {topodata.ITablet|null} [tablet] GetTabletResponse tablet */ /** - * Constructs a new GetShardRoutingRulesResponse. + * Constructs a new GetTabletResponse. * @memberof vtctldata - * @classdesc Represents a GetShardRoutingRulesResponse. - * @implements IGetShardRoutingRulesResponse + * @classdesc Represents a GetTabletResponse. + * @implements IGetTabletResponse * @constructor - * @param {vtctldata.IGetShardRoutingRulesResponse=} [properties] Properties to set + * @param {vtctldata.IGetTabletResponse=} [properties] Properties to set */ - function GetShardRoutingRulesResponse(properties) { + function GetTabletResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -117871,75 +123834,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetShardRoutingRulesResponse shard_routing_rules. - * @member {vschema.IShardRoutingRules|null|undefined} shard_routing_rules - * @memberof vtctldata.GetShardRoutingRulesResponse + * GetTabletResponse tablet. + * @member {topodata.ITablet|null|undefined} tablet + * @memberof vtctldata.GetTabletResponse * @instance */ - GetShardRoutingRulesResponse.prototype.shard_routing_rules = null; + GetTabletResponse.prototype.tablet = null; /** - * Creates a new GetShardRoutingRulesResponse instance using the specified properties. + * Creates a new GetTabletResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static - * @param {vtctldata.IGetShardRoutingRulesResponse=} [properties] Properties to set - * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse instance + * @param {vtctldata.IGetTabletResponse=} [properties] Properties to set + * @returns {vtctldata.GetTabletResponse} GetTabletResponse instance */ - GetShardRoutingRulesResponse.create = function create(properties) { - return new GetShardRoutingRulesResponse(properties); + GetTabletResponse.create = function create(properties) { + return new GetTabletResponse(properties); }; /** - * Encodes the specified GetShardRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. + * Encodes the specified GetTabletResponse message. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static - * @param {vtctldata.IGetShardRoutingRulesResponse} message GetShardRoutingRulesResponse message or plain object to encode + * @param {vtctldata.IGetTabletResponse} message GetTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRoutingRulesResponse.encode = function encode(message, writer) { + GetTabletResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard_routing_rules != null && Object.hasOwnProperty.call(message, "shard_routing_rules")) - $root.vschema.ShardRoutingRules.encode(message.shard_routing_rules, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.Tablet.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetShardRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetShardRoutingRulesResponse.verify|verify} messages. + * Encodes the specified GetTabletResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static - * @param {vtctldata.IGetShardRoutingRulesResponse} message GetShardRoutingRulesResponse message or plain object to encode + * @param {vtctldata.IGetTabletResponse} message GetTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetShardRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer. + * Decodes a GetTabletResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse + * @returns {vtctldata.GetTabletResponse} GetTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRoutingRulesResponse.decode = function decode(reader, length) { + GetTabletResponse.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.GetShardRoutingRulesResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTabletResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard_routing_rules = $root.vschema.ShardRoutingRules.decode(reader, reader.uint32()); + message.tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); break; } default: @@ -117951,128 +123914,134 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetShardRoutingRulesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTabletResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse + * @returns {vtctldata.GetTabletResponse} GetTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetShardRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + GetTabletResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetShardRoutingRulesResponse message. + * Verifies a GetTabletResponse message. * @function verify - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetShardRoutingRulesResponse.verify = function verify(message) { + GetTabletResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) { - let error = $root.vschema.ShardRoutingRules.verify(message.shard_routing_rules); + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.Tablet.verify(message.tablet); if (error) - return "shard_routing_rules." + error; + return "tablet." + error; } return null; }; /** - * Creates a GetShardRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetShardRoutingRulesResponse} GetShardRoutingRulesResponse + * @returns {vtctldata.GetTabletResponse} GetTabletResponse */ - GetShardRoutingRulesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetShardRoutingRulesResponse) + GetTabletResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTabletResponse) return object; - let message = new $root.vtctldata.GetShardRoutingRulesResponse(); - if (object.shard_routing_rules != null) { - if (typeof object.shard_routing_rules !== "object") - throw TypeError(".vtctldata.GetShardRoutingRulesResponse.shard_routing_rules: object expected"); - message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_routing_rules); + let message = new $root.vtctldata.GetTabletResponse(); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.GetTabletResponse.tablet: object expected"); + message.tablet = $root.topodata.Tablet.fromObject(object.tablet); } return message; }; /** - * Creates a plain object from a GetShardRoutingRulesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetTabletResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static - * @param {vtctldata.GetShardRoutingRulesResponse} message GetShardRoutingRulesResponse + * @param {vtctldata.GetTabletResponse} message GetTabletResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetShardRoutingRulesResponse.toObject = function toObject(message, options) { + GetTabletResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.shard_routing_rules = null; - if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) - object.shard_routing_rules = $root.vschema.ShardRoutingRules.toObject(message.shard_routing_rules, options); + object.tablet = null; + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.Tablet.toObject(message.tablet, options); return object; }; /** - * Converts this GetShardRoutingRulesResponse to JSON. + * Converts this GetTabletResponse to JSON. * @function toJSON - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @instance * @returns {Object.} JSON object */ - GetShardRoutingRulesResponse.prototype.toJSON = function toJSON() { + GetTabletResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetShardRoutingRulesResponse + * Gets the default type url for GetTabletResponse * @function getTypeUrl - * @memberof vtctldata.GetShardRoutingRulesResponse + * @memberof vtctldata.GetTabletResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetShardRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetShardRoutingRulesResponse"; + return typeUrlPrefix + "/vtctldata.GetTabletResponse"; }; - return GetShardRoutingRulesResponse; + return GetTabletResponse; })(); - vtctldata.GetSrvKeyspaceNamesRequest = (function() { + vtctldata.GetTabletsRequest = (function() { /** - * Properties of a GetSrvKeyspaceNamesRequest. + * Properties of a GetTabletsRequest. * @memberof vtctldata - * @interface IGetSrvKeyspaceNamesRequest - * @property {Array.|null} [cells] GetSrvKeyspaceNamesRequest cells + * @interface IGetTabletsRequest + * @property {string|null} [keyspace] GetTabletsRequest keyspace + * @property {string|null} [shard] GetTabletsRequest shard + * @property {Array.|null} [cells] GetTabletsRequest cells + * @property {boolean|null} [strict] GetTabletsRequest strict + * @property {Array.|null} [tablet_aliases] GetTabletsRequest tablet_aliases + * @property {topodata.TabletType|null} [tablet_type] GetTabletsRequest tablet_type */ /** - * Constructs a new GetSrvKeyspaceNamesRequest. + * Constructs a new GetTabletsRequest. * @memberof vtctldata - * @classdesc Represents a GetSrvKeyspaceNamesRequest. - * @implements IGetSrvKeyspaceNamesRequest + * @classdesc Represents a GetTabletsRequest. + * @implements IGetTabletsRequest * @constructor - * @param {vtctldata.IGetSrvKeyspaceNamesRequest=} [properties] Properties to set + * @param {vtctldata.IGetTabletsRequest=} [properties] Properties to set */ - function GetSrvKeyspaceNamesRequest(properties) { + function GetTabletsRequest(properties) { this.cells = []; + this.tablet_aliases = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -118080,80 +124049,153 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvKeyspaceNamesRequest cells. + * GetTabletsRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetTabletsRequest + * @instance + */ + GetTabletsRequest.prototype.keyspace = ""; + + /** + * GetTabletsRequest shard. + * @member {string} shard + * @memberof vtctldata.GetTabletsRequest + * @instance + */ + GetTabletsRequest.prototype.shard = ""; + + /** + * GetTabletsRequest cells. * @member {Array.} cells - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @instance */ - GetSrvKeyspaceNamesRequest.prototype.cells = $util.emptyArray; + GetTabletsRequest.prototype.cells = $util.emptyArray; /** - * Creates a new GetSrvKeyspaceNamesRequest instance using the specified properties. + * GetTabletsRequest strict. + * @member {boolean} strict + * @memberof vtctldata.GetTabletsRequest + * @instance + */ + GetTabletsRequest.prototype.strict = false; + + /** + * GetTabletsRequest tablet_aliases. + * @member {Array.} tablet_aliases + * @memberof vtctldata.GetTabletsRequest + * @instance + */ + GetTabletsRequest.prototype.tablet_aliases = $util.emptyArray; + + /** + * GetTabletsRequest tablet_type. + * @member {topodata.TabletType} tablet_type + * @memberof vtctldata.GetTabletsRequest + * @instance + */ + GetTabletsRequest.prototype.tablet_type = 0; + + /** + * Creates a new GetTabletsRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static - * @param {vtctldata.IGetSrvKeyspaceNamesRequest=} [properties] Properties to set - * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest instance + * @param {vtctldata.IGetTabletsRequest=} [properties] Properties to set + * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest instance */ - GetSrvKeyspaceNamesRequest.create = function create(properties) { - return new GetSrvKeyspaceNamesRequest(properties); + GetTabletsRequest.create = function create(properties) { + return new GetTabletsRequest(properties); }; /** - * Encodes the specified GetSrvKeyspaceNamesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. + * Encodes the specified GetTabletsRequest message. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static - * @param {vtctldata.IGetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest message or plain object to encode + * @param {vtctldata.IGetTabletsRequest} message GetTabletsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspaceNamesRequest.encode = function encode(message, writer) { + GetTabletsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); if (message.cells != null && message.cells.length) for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.cells[i]); + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); + if (message.strict != null && Object.hasOwnProperty.call(message, "strict")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.strict); + if (message.tablet_aliases != null && message.tablet_aliases.length) + for (let i = 0; i < message.tablet_aliases.length; ++i) + $root.topodata.TabletAlias.encode(message.tablet_aliases[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.tablet_type); return writer; }; /** - * Encodes the specified GetSrvKeyspaceNamesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesRequest.verify|verify} messages. + * Encodes the specified GetTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static - * @param {vtctldata.IGetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest message or plain object to encode + * @param {vtctldata.IGetTabletsRequest} message GetTabletsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspaceNamesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetTabletsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer. + * Decodes a GetTabletsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest + * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspaceNamesRequest.decode = function decode(reader, length) { + GetTabletsRequest.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.GetSrvKeyspaceNamesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTabletsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { if (!(message.cells && message.cells.length)) message.cells = []; message.cells.push(reader.string()); break; } + case 4: { + message.strict = reader.bool(); + break; + } + case 5: { + if (!(message.tablet_aliases && message.tablet_aliases.length)) + message.tablet_aliases = []; + message.tablet_aliases.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); + break; + } + case 6: { + message.tablet_type = reader.int32(); + break; + } default: reader.skipType(tag & 7); break; @@ -118163,32 +124205,38 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvKeyspaceNamesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTabletsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest + * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspaceNamesRequest.decodeDelimited = function decodeDelimited(reader) { + GetTabletsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvKeyspaceNamesRequest message. + * Verifies a GetTabletsRequest message. * @function verify - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvKeyspaceNamesRequest.verify = function verify(message) { + GetTabletsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; if (message.cells != null && message.hasOwnProperty("cells")) { if (!Array.isArray(message.cells)) return "cells: array expected"; @@ -118196,102 +124244,220 @@ export const vtctldata = $root.vtctldata = (() => { if (!$util.isString(message.cells[i])) return "cells: string[] expected"; } + if (message.strict != null && message.hasOwnProperty("strict")) + if (typeof message.strict !== "boolean") + return "strict: boolean expected"; + if (message.tablet_aliases != null && message.hasOwnProperty("tablet_aliases")) { + if (!Array.isArray(message.tablet_aliases)) + return "tablet_aliases: array expected"; + for (let i = 0; i < message.tablet_aliases.length; ++i) { + let error = $root.topodata.TabletAlias.verify(message.tablet_aliases[i]); + if (error) + return "tablet_aliases." + error; + } + } + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + switch (message.tablet_type) { + default: + return "tablet_type: enum value expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } return null; }; /** - * Creates a GetSrvKeyspaceNamesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvKeyspaceNamesRequest} GetSrvKeyspaceNamesRequest + * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest */ - GetSrvKeyspaceNamesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesRequest) + GetTabletsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTabletsRequest) return object; - let message = new $root.vtctldata.GetSrvKeyspaceNamesRequest(); + let message = new $root.vtctldata.GetTabletsRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); if (object.cells) { if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.GetSrvKeyspaceNamesRequest.cells: array expected"); + throw TypeError(".vtctldata.GetTabletsRequest.cells: array expected"); message.cells = []; for (let i = 0; i < object.cells.length; ++i) message.cells[i] = String(object.cells[i]); } + if (object.strict != null) + message.strict = Boolean(object.strict); + if (object.tablet_aliases) { + if (!Array.isArray(object.tablet_aliases)) + throw TypeError(".vtctldata.GetTabletsRequest.tablet_aliases: array expected"); + message.tablet_aliases = []; + for (let i = 0; i < object.tablet_aliases.length; ++i) { + if (typeof object.tablet_aliases[i] !== "object") + throw TypeError(".vtctldata.GetTabletsRequest.tablet_aliases: object expected"); + message.tablet_aliases[i] = $root.topodata.TabletAlias.fromObject(object.tablet_aliases[i]); + } + } + switch (object.tablet_type) { + default: + if (typeof object.tablet_type === "number") { + message.tablet_type = object.tablet_type; + break; + } + break; + case "UNKNOWN": + case 0: + message.tablet_type = 0; + break; + case "PRIMARY": + case 1: + message.tablet_type = 1; + break; + case "MASTER": + case 1: + message.tablet_type = 1; + break; + case "REPLICA": + case 2: + message.tablet_type = 2; + break; + case "RDONLY": + case 3: + message.tablet_type = 3; + break; + case "BATCH": + case 3: + message.tablet_type = 3; + break; + case "SPARE": + case 4: + message.tablet_type = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_type = 5; + break; + case "BACKUP": + case 6: + message.tablet_type = 6; + break; + case "RESTORE": + case 7: + message.tablet_type = 7; + break; + case "DRAINED": + case 8: + message.tablet_type = 8; + break; + } return message; }; /** - * Creates a plain object from a GetSrvKeyspaceNamesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetTabletsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static - * @param {vtctldata.GetSrvKeyspaceNamesRequest} message GetSrvKeyspaceNamesRequest + * @param {vtctldata.GetTabletsRequest} message GetTabletsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvKeyspaceNamesRequest.toObject = function toObject(message, options) { + GetTabletsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) + if (options.arrays || options.defaults) { object.cells = []; + object.tablet_aliases = []; + } + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.strict = false; + object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; if (message.cells && message.cells.length) { object.cells = []; for (let j = 0; j < message.cells.length; ++j) object.cells[j] = message.cells[j]; } + if (message.strict != null && message.hasOwnProperty("strict")) + object.strict = message.strict; + if (message.tablet_aliases && message.tablet_aliases.length) { + object.tablet_aliases = []; + for (let j = 0; j < message.tablet_aliases.length; ++j) + object.tablet_aliases[j] = $root.topodata.TabletAlias.toObject(message.tablet_aliases[j], options); + } + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; return object; }; /** - * Converts this GetSrvKeyspaceNamesRequest to JSON. + * Converts this GetTabletsRequest to JSON. * @function toJSON - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @instance * @returns {Object.} JSON object */ - GetSrvKeyspaceNamesRequest.prototype.toJSON = function toJSON() { + GetTabletsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvKeyspaceNamesRequest + * Gets the default type url for GetTabletsRequest * @function getTypeUrl - * @memberof vtctldata.GetSrvKeyspaceNamesRequest + * @memberof vtctldata.GetTabletsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvKeyspaceNamesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTabletsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesRequest"; + return typeUrlPrefix + "/vtctldata.GetTabletsRequest"; }; - return GetSrvKeyspaceNamesRequest; + return GetTabletsRequest; })(); - vtctldata.GetSrvKeyspaceNamesResponse = (function() { + vtctldata.GetTabletsResponse = (function() { /** - * Properties of a GetSrvKeyspaceNamesResponse. + * Properties of a GetTabletsResponse. * @memberof vtctldata - * @interface IGetSrvKeyspaceNamesResponse - * @property {Object.|null} [names] GetSrvKeyspaceNamesResponse names + * @interface IGetTabletsResponse + * @property {Array.|null} [tablets] GetTabletsResponse tablets */ /** - * Constructs a new GetSrvKeyspaceNamesResponse. + * Constructs a new GetTabletsResponse. * @memberof vtctldata - * @classdesc Represents a GetSrvKeyspaceNamesResponse. - * @implements IGetSrvKeyspaceNamesResponse + * @classdesc Represents a GetTabletsResponse. + * @implements IGetTabletsResponse * @constructor - * @param {vtctldata.IGetSrvKeyspaceNamesResponse=} [properties] Properties to set + * @param {vtctldata.IGetTabletsResponse=} [properties] Properties to set */ - function GetSrvKeyspaceNamesResponse(properties) { - this.names = {}; + function GetTabletsResponse(properties) { + this.tablets = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -118299,97 +124465,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvKeyspaceNamesResponse names. - * @member {Object.} names - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * GetTabletsResponse tablets. + * @member {Array.} tablets + * @memberof vtctldata.GetTabletsResponse * @instance */ - GetSrvKeyspaceNamesResponse.prototype.names = $util.emptyObject; + GetTabletsResponse.prototype.tablets = $util.emptyArray; /** - * Creates a new GetSrvKeyspaceNamesResponse instance using the specified properties. + * Creates a new GetTabletsResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static - * @param {vtctldata.IGetSrvKeyspaceNamesResponse=} [properties] Properties to set - * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse instance + * @param {vtctldata.IGetTabletsResponse=} [properties] Properties to set + * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse instance */ - GetSrvKeyspaceNamesResponse.create = function create(properties) { - return new GetSrvKeyspaceNamesResponse(properties); + GetTabletsResponse.create = function create(properties) { + return new GetTabletsResponse(properties); }; /** - * Encodes the specified GetSrvKeyspaceNamesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. + * Encodes the specified GetTabletsResponse message. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static - * @param {vtctldata.IGetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse message or plain object to encode + * @param {vtctldata.IGetTabletsResponse} message GetTabletsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspaceNamesResponse.encode = function encode(message, writer) { + GetTabletsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.names != null && Object.hasOwnProperty.call(message, "names")) - for (let keys = Object.keys(message.names), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.encode(message.names[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.tablets != null && message.tablets.length) + for (let i = 0; i < message.tablets.length; ++i) + $root.topodata.Tablet.encode(message.tablets[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetSrvKeyspaceNamesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.verify|verify} messages. + * Encodes the specified GetTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static - * @param {vtctldata.IGetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse message or plain object to encode + * @param {vtctldata.IGetTabletsResponse} message GetTabletsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspaceNamesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetTabletsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer. + * Decodes a GetTabletsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse + * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspaceNamesResponse.decode = function decode(reader, length) { + GetTabletsResponse.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.GetSrvKeyspaceNamesResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTabletsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.names === $util.emptyObject) - message.names = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.names[key] = value; + if (!(message.tablets && message.tablets.length)) + message.tablets = []; + message.tablets.push($root.topodata.Tablet.decode(reader, reader.uint32())); break; } default: @@ -118401,362 +124548,139 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvKeyspaceNamesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTabletsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse + * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspaceNamesResponse.decodeDelimited = function decodeDelimited(reader) { + GetTabletsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvKeyspaceNamesResponse message. + * Verifies a GetTabletsResponse message. * @function verify - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvKeyspaceNamesResponse.verify = function verify(message) { + GetTabletsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.names != null && message.hasOwnProperty("names")) { - if (!$util.isObject(message.names)) - return "names: object expected"; - let key = Object.keys(message.names); - for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify(message.names[key[i]]); + if (message.tablets != null && message.hasOwnProperty("tablets")) { + if (!Array.isArray(message.tablets)) + return "tablets: array expected"; + for (let i = 0; i < message.tablets.length; ++i) { + let error = $root.topodata.Tablet.verify(message.tablets[i]); if (error) - return "names." + error; + return "tablets." + error; } } return null; }; /** - * Creates a GetSrvKeyspaceNamesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTabletsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvKeyspaceNamesResponse} GetSrvKeyspaceNamesResponse + * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse */ - GetSrvKeyspaceNamesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesResponse) + GetTabletsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTabletsResponse) return object; - let message = new $root.vtctldata.GetSrvKeyspaceNamesResponse(); - if (object.names) { - if (typeof object.names !== "object") - throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.names: object expected"); - message.names = {}; - for (let keys = Object.keys(object.names), i = 0; i < keys.length; ++i) { - if (typeof object.names[keys[i]] !== "object") - throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.names: object expected"); - message.names[keys[i]] = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.fromObject(object.names[keys[i]]); + let message = new $root.vtctldata.GetTabletsResponse(); + if (object.tablets) { + if (!Array.isArray(object.tablets)) + throw TypeError(".vtctldata.GetTabletsResponse.tablets: array expected"); + message.tablets = []; + for (let i = 0; i < object.tablets.length; ++i) { + if (typeof object.tablets[i] !== "object") + throw TypeError(".vtctldata.GetTabletsResponse.tablets: object expected"); + message.tablets[i] = $root.topodata.Tablet.fromObject(object.tablets[i]); } } return message; }; /** - * Creates a plain object from a GetSrvKeyspaceNamesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetTabletsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static - * @param {vtctldata.GetSrvKeyspaceNamesResponse} message GetSrvKeyspaceNamesResponse + * @param {vtctldata.GetTabletsResponse} message GetTabletsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvKeyspaceNamesResponse.toObject = function toObject(message, options) { + GetTabletsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.names = {}; - let keys2; - if (message.names && (keys2 = Object.keys(message.names)).length) { - object.names = {}; - for (let j = 0; j < keys2.length; ++j) - object.names[keys2[j]] = $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList.toObject(message.names[keys2[j]], options); + if (options.arrays || options.defaults) + object.tablets = []; + if (message.tablets && message.tablets.length) { + object.tablets = []; + for (let j = 0; j < message.tablets.length; ++j) + object.tablets[j] = $root.topodata.Tablet.toObject(message.tablets[j], options); } return object; }; /** - * Converts this GetSrvKeyspaceNamesResponse to JSON. + * Converts this GetTabletsResponse to JSON. * @function toJSON - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @instance * @returns {Object.} JSON object */ - GetSrvKeyspaceNamesResponse.prototype.toJSON = function toJSON() { + GetTabletsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvKeyspaceNamesResponse + * Gets the default type url for GetTabletsResponse * @function getTypeUrl - * @memberof vtctldata.GetSrvKeyspaceNamesResponse + * @memberof vtctldata.GetTabletsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvKeyspaceNamesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTabletsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesResponse"; + return typeUrlPrefix + "/vtctldata.GetTabletsResponse"; }; - GetSrvKeyspaceNamesResponse.NameList = (function() { - - /** - * Properties of a NameList. - * @memberof vtctldata.GetSrvKeyspaceNamesResponse - * @interface INameList - * @property {Array.|null} [names] NameList names - */ - - /** - * Constructs a new NameList. - * @memberof vtctldata.GetSrvKeyspaceNamesResponse - * @classdesc Represents a NameList. - * @implements INameList - * @constructor - * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList=} [properties] Properties to set - */ - function NameList(properties) { - this.names = []; - 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]]; - } - - /** - * NameList names. - * @member {Array.} names - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @instance - */ - NameList.prototype.names = $util.emptyArray; - - /** - * Creates a new NameList instance using the specified properties. - * @function create - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList=} [properties] Properties to set - * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList instance - */ - NameList.create = function create(properties) { - return new NameList(properties); - }; - - /** - * Encodes the specified NameList message. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. - * @function encode - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList} message NameList message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - NameList.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.names != null && message.names.length) - for (let i = 0; i < message.names.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.names[i]); - return writer; - }; - - /** - * Encodes the specified NameList message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspaceNamesResponse.NameList.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {vtctldata.GetSrvKeyspaceNamesResponse.INameList} message NameList message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - NameList.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; - - /** - * Decodes a NameList message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - NameList.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.GetSrvKeyspaceNamesResponse.NameList(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - if (!(message.names && message.names.length)) - message.names = []; - message.names.push(reader.string()); - break; - } - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Decodes a NameList message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - NameList.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a NameList message. - * @function verify - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - NameList.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.names != null && message.hasOwnProperty("names")) { - if (!Array.isArray(message.names)) - return "names: array expected"; - for (let i = 0; i < message.names.length; ++i) - if (!$util.isString(message.names[i])) - return "names: string[] expected"; - } - return null; - }; - - /** - * Creates a NameList message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvKeyspaceNamesResponse.NameList} NameList - */ - NameList.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList) - return object; - let message = new $root.vtctldata.GetSrvKeyspaceNamesResponse.NameList(); - if (object.names) { - if (!Array.isArray(object.names)) - throw TypeError(".vtctldata.GetSrvKeyspaceNamesResponse.NameList.names: array expected"); - message.names = []; - for (let i = 0; i < object.names.length; ++i) - message.names[i] = String(object.names[i]); - } - return message; - }; - - /** - * Creates a plain object from a NameList message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {vtctldata.GetSrvKeyspaceNamesResponse.NameList} message NameList - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - NameList.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.names = []; - if (message.names && message.names.length) { - object.names = []; - for (let j = 0; j < message.names.length; ++j) - object.names[j] = message.names[j]; - } - return object; - }; - - /** - * Converts this NameList to JSON. - * @function toJSON - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @instance - * @returns {Object.} JSON object - */ - NameList.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; - - /** - * Gets the default type url for NameList - * @function getTypeUrl - * @memberof vtctldata.GetSrvKeyspaceNamesResponse.NameList - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - NameList.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.GetSrvKeyspaceNamesResponse.NameList"; - }; - - return NameList; - })(); - - return GetSrvKeyspaceNamesResponse; + return GetTabletsResponse; })(); - vtctldata.GetSrvKeyspacesRequest = (function() { + vtctldata.GetTopologyPathRequest = (function() { /** - * Properties of a GetSrvKeyspacesRequest. + * Properties of a GetTopologyPathRequest. * @memberof vtctldata - * @interface IGetSrvKeyspacesRequest - * @property {string|null} [keyspace] GetSrvKeyspacesRequest keyspace - * @property {Array.|null} [cells] GetSrvKeyspacesRequest cells + * @interface IGetTopologyPathRequest + * @property {string|null} [path] GetTopologyPathRequest path */ /** - * Constructs a new GetSrvKeyspacesRequest. + * Constructs a new GetTopologyPathRequest. * @memberof vtctldata - * @classdesc Represents a GetSrvKeyspacesRequest. - * @implements IGetSrvKeyspacesRequest + * @classdesc Represents a GetTopologyPathRequest. + * @implements IGetTopologyPathRequest * @constructor - * @param {vtctldata.IGetSrvKeyspacesRequest=} [properties] Properties to set + * @param {vtctldata.IGetTopologyPathRequest=} [properties] Properties to set */ - function GetSrvKeyspacesRequest(properties) { - this.cells = []; + function GetTopologyPathRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -118764,92 +124688,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvKeyspacesRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.GetSrvKeyspacesRequest - * @instance - */ - GetSrvKeyspacesRequest.prototype.keyspace = ""; - - /** - * GetSrvKeyspacesRequest cells. - * @member {Array.} cells - * @memberof vtctldata.GetSrvKeyspacesRequest + * GetTopologyPathRequest path. + * @member {string} path + * @memberof vtctldata.GetTopologyPathRequest * @instance */ - GetSrvKeyspacesRequest.prototype.cells = $util.emptyArray; + GetTopologyPathRequest.prototype.path = ""; /** - * Creates a new GetSrvKeyspacesRequest instance using the specified properties. + * Creates a new GetTopologyPathRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static - * @param {vtctldata.IGetSrvKeyspacesRequest=} [properties] Properties to set - * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest instance + * @param {vtctldata.IGetTopologyPathRequest=} [properties] Properties to set + * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest instance */ - GetSrvKeyspacesRequest.create = function create(properties) { - return new GetSrvKeyspacesRequest(properties); + GetTopologyPathRequest.create = function create(properties) { + return new GetTopologyPathRequest(properties); }; /** - * Encodes the specified GetSrvKeyspacesRequest message. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. + * Encodes the specified GetTopologyPathRequest message. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static - * @param {vtctldata.IGetSrvKeyspacesRequest} message GetSrvKeyspacesRequest message or plain object to encode + * @param {vtctldata.IGetTopologyPathRequest} message GetTopologyPathRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspacesRequest.encode = function encode(message, writer) { + GetTopologyPathRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.path != null && Object.hasOwnProperty.call(message, "path")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.path); return writer; }; /** - * Encodes the specified GetSrvKeyspacesRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesRequest.verify|verify} messages. + * Encodes the specified GetTopologyPathRequest message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static - * @param {vtctldata.IGetSrvKeyspacesRequest} message GetSrvKeyspacesRequest message or plain object to encode + * @param {vtctldata.IGetTopologyPathRequest} message GetTopologyPathRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspacesRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetTopologyPathRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer. + * Decodes a GetTopologyPathRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest + * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspacesRequest.decode = function decode(reader, length) { + GetTopologyPathRequest.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.GetSrvKeyspacesRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTopologyPathRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + message.path = reader.string(); break; } default: @@ -118861,144 +124768,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvKeyspacesRequest message from the specified reader or buffer, length delimited. + * Decodes a GetTopologyPathRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest + * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspacesRequest.decodeDelimited = function decodeDelimited(reader) { + GetTopologyPathRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvKeyspacesRequest message. + * Verifies a GetTopologyPathRequest message. * @function verify - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvKeyspacesRequest.verify = function verify(message) { + GetTopologyPathRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } + if (message.path != null && message.hasOwnProperty("path")) + if (!$util.isString(message.path)) + return "path: string expected"; return null; }; /** - * Creates a GetSrvKeyspacesRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetTopologyPathRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvKeyspacesRequest} GetSrvKeyspacesRequest + * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest */ - GetSrvKeyspacesRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvKeyspacesRequest) + GetTopologyPathRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTopologyPathRequest) return object; - let message = new $root.vtctldata.GetSrvKeyspacesRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.GetSrvKeyspacesRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } + let message = new $root.vtctldata.GetTopologyPathRequest(); + if (object.path != null) + message.path = String(object.path); return message; }; /** - * Creates a plain object from a GetSrvKeyspacesRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetTopologyPathRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static - * @param {vtctldata.GetSrvKeyspacesRequest} message GetSrvKeyspacesRequest + * @param {vtctldata.GetTopologyPathRequest} message GetTopologyPathRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvKeyspacesRequest.toObject = function toObject(message, options) { + GetTopologyPathRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; if (options.defaults) - object.keyspace = ""; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } + object.path = ""; + if (message.path != null && message.hasOwnProperty("path")) + object.path = message.path; return object; }; /** - * Converts this GetSrvKeyspacesRequest to JSON. + * Converts this GetTopologyPathRequest to JSON. * @function toJSON - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @instance * @returns {Object.} JSON object */ - GetSrvKeyspacesRequest.prototype.toJSON = function toJSON() { + GetTopologyPathRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvKeyspacesRequest + * Gets the default type url for GetTopologyPathRequest * @function getTypeUrl - * @memberof vtctldata.GetSrvKeyspacesRequest + * @memberof vtctldata.GetTopologyPathRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvKeyspacesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTopologyPathRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvKeyspacesRequest"; + return typeUrlPrefix + "/vtctldata.GetTopologyPathRequest"; }; - return GetSrvKeyspacesRequest; + return GetTopologyPathRequest; })(); - vtctldata.GetSrvKeyspacesResponse = (function() { + vtctldata.GetTopologyPathResponse = (function() { /** - * Properties of a GetSrvKeyspacesResponse. + * Properties of a GetTopologyPathResponse. * @memberof vtctldata - * @interface IGetSrvKeyspacesResponse - * @property {Object.|null} [srv_keyspaces] GetSrvKeyspacesResponse srv_keyspaces + * @interface IGetTopologyPathResponse + * @property {vtctldata.ITopologyCell|null} [cell] GetTopologyPathResponse cell */ /** - * Constructs a new GetSrvKeyspacesResponse. + * Constructs a new GetTopologyPathResponse. * @memberof vtctldata - * @classdesc Represents a GetSrvKeyspacesResponse. - * @implements IGetSrvKeyspacesResponse + * @classdesc Represents a GetTopologyPathResponse. + * @implements IGetTopologyPathResponse * @constructor - * @param {vtctldata.IGetSrvKeyspacesResponse=} [properties] Properties to set + * @param {vtctldata.IGetTopologyPathResponse=} [properties] Properties to set */ - function GetSrvKeyspacesResponse(properties) { - this.srv_keyspaces = {}; + function GetTopologyPathResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -119006,97 +124891,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvKeyspacesResponse srv_keyspaces. - * @member {Object.} srv_keyspaces - * @memberof vtctldata.GetSrvKeyspacesResponse + * GetTopologyPathResponse cell. + * @member {vtctldata.ITopologyCell|null|undefined} cell + * @memberof vtctldata.GetTopologyPathResponse * @instance */ - GetSrvKeyspacesResponse.prototype.srv_keyspaces = $util.emptyObject; + GetTopologyPathResponse.prototype.cell = null; /** - * Creates a new GetSrvKeyspacesResponse instance using the specified properties. + * Creates a new GetTopologyPathResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static - * @param {vtctldata.IGetSrvKeyspacesResponse=} [properties] Properties to set - * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse instance + * @param {vtctldata.IGetTopologyPathResponse=} [properties] Properties to set + * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse instance */ - GetSrvKeyspacesResponse.create = function create(properties) { - return new GetSrvKeyspacesResponse(properties); + GetTopologyPathResponse.create = function create(properties) { + return new GetTopologyPathResponse(properties); }; /** - * Encodes the specified GetSrvKeyspacesResponse message. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. + * Encodes the specified GetTopologyPathResponse message. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static - * @param {vtctldata.IGetSrvKeyspacesResponse} message GetSrvKeyspacesResponse message or plain object to encode + * @param {vtctldata.IGetTopologyPathResponse} message GetTopologyPathResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspacesResponse.encode = function encode(message, writer) { + GetTopologyPathResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.srv_keyspaces != null && Object.hasOwnProperty.call(message, "srv_keyspaces")) - for (let keys = Object.keys(message.srv_keyspaces), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.topodata.SrvKeyspace.encode(message.srv_keyspaces[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + $root.vtctldata.TopologyCell.encode(message.cell, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetSrvKeyspacesResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvKeyspacesResponse.verify|verify} messages. + * Encodes the specified GetTopologyPathResponse message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static - * @param {vtctldata.IGetSrvKeyspacesResponse} message GetSrvKeyspacesResponse message or plain object to encode + * @param {vtctldata.IGetTopologyPathResponse} message GetTopologyPathResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvKeyspacesResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetTopologyPathResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer. + * Decodes a GetTopologyPathResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse + * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspacesResponse.decode = function decode(reader, length) { + GetTopologyPathResponse.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.GetSrvKeyspacesResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetTopologyPathResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.srv_keyspaces === $util.emptyObject) - message.srv_keyspaces = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.topodata.SrvKeyspace.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.srv_keyspaces[key] = value; + message.cell = $root.vtctldata.TopologyCell.decode(reader, reader.uint32()); break; } default: @@ -119108,149 +124971,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvKeyspacesResponse message from the specified reader or buffer, length delimited. + * Decodes a GetTopologyPathResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse + * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvKeyspacesResponse.decodeDelimited = function decodeDelimited(reader) { + GetTopologyPathResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvKeyspacesResponse message. + * Verifies a GetTopologyPathResponse message. * @function verify - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvKeyspacesResponse.verify = function verify(message) { + GetTopologyPathResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.srv_keyspaces != null && message.hasOwnProperty("srv_keyspaces")) { - if (!$util.isObject(message.srv_keyspaces)) - return "srv_keyspaces: object expected"; - let key = Object.keys(message.srv_keyspaces); - for (let i = 0; i < key.length; ++i) { - let error = $root.topodata.SrvKeyspace.verify(message.srv_keyspaces[key[i]]); - if (error) - return "srv_keyspaces." + error; - } + if (message.cell != null && message.hasOwnProperty("cell")) { + let error = $root.vtctldata.TopologyCell.verify(message.cell); + if (error) + return "cell." + error; } return null; }; /** - * Creates a GetSrvKeyspacesResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetTopologyPathResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvKeyspacesResponse} GetSrvKeyspacesResponse + * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse */ - GetSrvKeyspacesResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvKeyspacesResponse) + GetTopologyPathResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetTopologyPathResponse) return object; - let message = new $root.vtctldata.GetSrvKeyspacesResponse(); - if (object.srv_keyspaces) { - if (typeof object.srv_keyspaces !== "object") - throw TypeError(".vtctldata.GetSrvKeyspacesResponse.srv_keyspaces: object expected"); - message.srv_keyspaces = {}; - for (let keys = Object.keys(object.srv_keyspaces), i = 0; i < keys.length; ++i) { - if (typeof object.srv_keyspaces[keys[i]] !== "object") - throw TypeError(".vtctldata.GetSrvKeyspacesResponse.srv_keyspaces: object expected"); - message.srv_keyspaces[keys[i]] = $root.topodata.SrvKeyspace.fromObject(object.srv_keyspaces[keys[i]]); - } + let message = new $root.vtctldata.GetTopologyPathResponse(); + if (object.cell != null) { + if (typeof object.cell !== "object") + throw TypeError(".vtctldata.GetTopologyPathResponse.cell: object expected"); + message.cell = $root.vtctldata.TopologyCell.fromObject(object.cell); } return message; }; /** - * Creates a plain object from a GetSrvKeyspacesResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetTopologyPathResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static - * @param {vtctldata.GetSrvKeyspacesResponse} message GetSrvKeyspacesResponse + * @param {vtctldata.GetTopologyPathResponse} message GetTopologyPathResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvKeyspacesResponse.toObject = function toObject(message, options) { + GetTopologyPathResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.srv_keyspaces = {}; - let keys2; - if (message.srv_keyspaces && (keys2 = Object.keys(message.srv_keyspaces)).length) { - object.srv_keyspaces = {}; - for (let j = 0; j < keys2.length; ++j) - object.srv_keyspaces[keys2[j]] = $root.topodata.SrvKeyspace.toObject(message.srv_keyspaces[keys2[j]], options); - } + if (options.defaults) + object.cell = null; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = $root.vtctldata.TopologyCell.toObject(message.cell, options); return object; }; /** - * Converts this GetSrvKeyspacesResponse to JSON. + * Converts this GetTopologyPathResponse to JSON. * @function toJSON - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @instance * @returns {Object.} JSON object */ - GetSrvKeyspacesResponse.prototype.toJSON = function toJSON() { + GetTopologyPathResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvKeyspacesResponse + * Gets the default type url for GetTopologyPathResponse * @function getTypeUrl - * @memberof vtctldata.GetSrvKeyspacesResponse + * @memberof vtctldata.GetTopologyPathResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvKeyspacesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetTopologyPathResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvKeyspacesResponse"; + return typeUrlPrefix + "/vtctldata.GetTopologyPathResponse"; }; - return GetSrvKeyspacesResponse; + return GetTopologyPathResponse; })(); - vtctldata.UpdateThrottlerConfigRequest = (function() { + vtctldata.TopologyCell = (function() { /** - * Properties of an UpdateThrottlerConfigRequest. + * Properties of a TopologyCell. * @memberof vtctldata - * @interface IUpdateThrottlerConfigRequest - * @property {string|null} [keyspace] UpdateThrottlerConfigRequest keyspace - * @property {boolean|null} [enable] UpdateThrottlerConfigRequest enable - * @property {boolean|null} [disable] UpdateThrottlerConfigRequest disable - * @property {number|null} [threshold] UpdateThrottlerConfigRequest threshold - * @property {string|null} [custom_query] UpdateThrottlerConfigRequest custom_query - * @property {boolean|null} [custom_query_set] UpdateThrottlerConfigRequest custom_query_set - * @property {boolean|null} [check_as_check_self] UpdateThrottlerConfigRequest check_as_check_self - * @property {boolean|null} [check_as_check_shard] UpdateThrottlerConfigRequest check_as_check_shard - * @property {topodata.IThrottledAppRule|null} [throttled_app] UpdateThrottlerConfigRequest throttled_app + * @interface ITopologyCell + * @property {string|null} [name] TopologyCell name + * @property {string|null} [path] TopologyCell path + * @property {string|null} [data] TopologyCell data + * @property {Array.|null} [children] TopologyCell children */ /** - * Constructs a new UpdateThrottlerConfigRequest. + * Constructs a new TopologyCell. * @memberof vtctldata - * @classdesc Represents an UpdateThrottlerConfigRequest. - * @implements IUpdateThrottlerConfigRequest + * @classdesc Represents a TopologyCell. + * @implements ITopologyCell * @constructor - * @param {vtctldata.IUpdateThrottlerConfigRequest=} [properties] Properties to set + * @param {vtctldata.ITopologyCell=} [properties] Properties to set */ - function UpdateThrottlerConfigRequest(properties) { + function TopologyCell(properties) { + this.children = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -119258,187 +125103,120 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * UpdateThrottlerConfigRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.UpdateThrottlerConfigRequest - * @instance - */ - UpdateThrottlerConfigRequest.prototype.keyspace = ""; - - /** - * UpdateThrottlerConfigRequest enable. - * @member {boolean} enable - * @memberof vtctldata.UpdateThrottlerConfigRequest - * @instance - */ - UpdateThrottlerConfigRequest.prototype.enable = false; - - /** - * UpdateThrottlerConfigRequest disable. - * @member {boolean} disable - * @memberof vtctldata.UpdateThrottlerConfigRequest - * @instance - */ - UpdateThrottlerConfigRequest.prototype.disable = false; - - /** - * UpdateThrottlerConfigRequest threshold. - * @member {number} threshold - * @memberof vtctldata.UpdateThrottlerConfigRequest - * @instance - */ - UpdateThrottlerConfigRequest.prototype.threshold = 0; - - /** - * UpdateThrottlerConfigRequest custom_query. - * @member {string} custom_query - * @memberof vtctldata.UpdateThrottlerConfigRequest - * @instance - */ - UpdateThrottlerConfigRequest.prototype.custom_query = ""; - - /** - * UpdateThrottlerConfigRequest custom_query_set. - * @member {boolean} custom_query_set - * @memberof vtctldata.UpdateThrottlerConfigRequest + * TopologyCell name. + * @member {string} name + * @memberof vtctldata.TopologyCell * @instance */ - UpdateThrottlerConfigRequest.prototype.custom_query_set = false; + TopologyCell.prototype.name = ""; /** - * UpdateThrottlerConfigRequest check_as_check_self. - * @member {boolean} check_as_check_self - * @memberof vtctldata.UpdateThrottlerConfigRequest + * TopologyCell path. + * @member {string} path + * @memberof vtctldata.TopologyCell * @instance */ - UpdateThrottlerConfigRequest.prototype.check_as_check_self = false; + TopologyCell.prototype.path = ""; /** - * UpdateThrottlerConfigRequest check_as_check_shard. - * @member {boolean} check_as_check_shard - * @memberof vtctldata.UpdateThrottlerConfigRequest + * TopologyCell data. + * @member {string} data + * @memberof vtctldata.TopologyCell * @instance */ - UpdateThrottlerConfigRequest.prototype.check_as_check_shard = false; + TopologyCell.prototype.data = ""; /** - * UpdateThrottlerConfigRequest throttled_app. - * @member {topodata.IThrottledAppRule|null|undefined} throttled_app - * @memberof vtctldata.UpdateThrottlerConfigRequest + * TopologyCell children. + * @member {Array.} children + * @memberof vtctldata.TopologyCell * @instance */ - UpdateThrottlerConfigRequest.prototype.throttled_app = null; + TopologyCell.prototype.children = $util.emptyArray; /** - * Creates a new UpdateThrottlerConfigRequest instance using the specified properties. + * Creates a new TopologyCell instance using the specified properties. * @function create - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static - * @param {vtctldata.IUpdateThrottlerConfigRequest=} [properties] Properties to set - * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest instance + * @param {vtctldata.ITopologyCell=} [properties] Properties to set + * @returns {vtctldata.TopologyCell} TopologyCell instance */ - UpdateThrottlerConfigRequest.create = function create(properties) { - return new UpdateThrottlerConfigRequest(properties); + TopologyCell.create = function create(properties) { + return new TopologyCell(properties); }; /** - * Encodes the specified UpdateThrottlerConfigRequest message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. + * Encodes the specified TopologyCell message. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static - * @param {vtctldata.IUpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest message or plain object to encode + * @param {vtctldata.ITopologyCell} message TopologyCell message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateThrottlerConfigRequest.encode = function encode(message, writer) { + TopologyCell.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.enable != null && Object.hasOwnProperty.call(message, "enable")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.enable); - if (message.disable != null && Object.hasOwnProperty.call(message, "disable")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.disable); - if (message.threshold != null && Object.hasOwnProperty.call(message, "threshold")) - writer.uint32(/* id 4, wireType 1 =*/33).double(message.threshold); - if (message.custom_query != null && Object.hasOwnProperty.call(message, "custom_query")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.custom_query); - if (message.custom_query_set != null && Object.hasOwnProperty.call(message, "custom_query_set")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.custom_query_set); - if (message.check_as_check_self != null && Object.hasOwnProperty.call(message, "check_as_check_self")) - writer.uint32(/* id 7, wireType 0 =*/56).bool(message.check_as_check_self); - if (message.check_as_check_shard != null && Object.hasOwnProperty.call(message, "check_as_check_shard")) - writer.uint32(/* id 8, wireType 0 =*/64).bool(message.check_as_check_shard); - if (message.throttled_app != null && Object.hasOwnProperty.call(message, "throttled_app")) - $root.topodata.ThrottledAppRule.encode(message.throttled_app, writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.path != null && Object.hasOwnProperty.call(message, "path")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.path); + if (message.data != null && Object.hasOwnProperty.call(message, "data")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.data); + if (message.children != null && message.children.length) + for (let i = 0; i < message.children.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.children[i]); return writer; }; /** - * Encodes the specified UpdateThrottlerConfigRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigRequest.verify|verify} messages. + * Encodes the specified TopologyCell message, length delimited. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static - * @param {vtctldata.IUpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest message or plain object to encode + * @param {vtctldata.ITopologyCell} message TopologyCell message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateThrottlerConfigRequest.encodeDelimited = function encodeDelimited(message, writer) { + TopologyCell.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer. + * Decodes a TopologyCell message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest + * @returns {vtctldata.TopologyCell} TopologyCell * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateThrottlerConfigRequest.decode = function decode(reader, length) { + TopologyCell.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.UpdateThrottlerConfigRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.TopologyCell(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.name = reader.string(); break; } case 2: { - message.enable = reader.bool(); + message.path = reader.string(); break; } case 3: { - message.disable = reader.bool(); + message.data = reader.string(); break; } case 4: { - message.threshold = reader.double(); - break; - } - case 5: { - message.custom_query = reader.string(); - break; - } - case 6: { - message.custom_query_set = reader.bool(); - break; - } - case 7: { - message.check_as_check_self = reader.bool(); - break; - } - case 8: { - message.check_as_check_shard = reader.bool(); - break; - } - case 9: { - message.throttled_app = $root.topodata.ThrottledAppRule.decode(reader, reader.uint32()); + if (!(message.children && message.children.length)) + message.children = []; + message.children.push(reader.string()); break; } default: @@ -119450,191 +125228,160 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateThrottlerConfigRequest message from the specified reader or buffer, length delimited. + * Decodes a TopologyCell message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest + * @returns {vtctldata.TopologyCell} TopologyCell * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateThrottlerConfigRequest.decodeDelimited = function decodeDelimited(reader) { + TopologyCell.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateThrottlerConfigRequest message. + * Verifies a TopologyCell message. * @function verify - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateThrottlerConfigRequest.verify = function verify(message) { + TopologyCell.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.enable != null && message.hasOwnProperty("enable")) - if (typeof message.enable !== "boolean") - return "enable: boolean expected"; - if (message.disable != null && message.hasOwnProperty("disable")) - if (typeof message.disable !== "boolean") - return "disable: boolean expected"; - if (message.threshold != null && message.hasOwnProperty("threshold")) - if (typeof message.threshold !== "number") - return "threshold: number expected"; - if (message.custom_query != null && message.hasOwnProperty("custom_query")) - if (!$util.isString(message.custom_query)) - return "custom_query: string expected"; - if (message.custom_query_set != null && message.hasOwnProperty("custom_query_set")) - if (typeof message.custom_query_set !== "boolean") - return "custom_query_set: boolean expected"; - if (message.check_as_check_self != null && message.hasOwnProperty("check_as_check_self")) - if (typeof message.check_as_check_self !== "boolean") - return "check_as_check_self: boolean expected"; - if (message.check_as_check_shard != null && message.hasOwnProperty("check_as_check_shard")) - if (typeof message.check_as_check_shard !== "boolean") - return "check_as_check_shard: boolean expected"; - if (message.throttled_app != null && message.hasOwnProperty("throttled_app")) { - let error = $root.topodata.ThrottledAppRule.verify(message.throttled_app); - if (error) - return "throttled_app." + error; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.path != null && message.hasOwnProperty("path")) + if (!$util.isString(message.path)) + return "path: string expected"; + if (message.data != null && message.hasOwnProperty("data")) + if (!$util.isString(message.data)) + return "data: string expected"; + if (message.children != null && message.hasOwnProperty("children")) { + if (!Array.isArray(message.children)) + return "children: array expected"; + for (let i = 0; i < message.children.length; ++i) + if (!$util.isString(message.children[i])) + return "children: string[] expected"; } return null; }; /** - * Creates an UpdateThrottlerConfigRequest message from a plain object. Also converts values to their respective internal types. + * Creates a TopologyCell message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateThrottlerConfigRequest} UpdateThrottlerConfigRequest + * @returns {vtctldata.TopologyCell} TopologyCell */ - UpdateThrottlerConfigRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateThrottlerConfigRequest) + TopologyCell.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.TopologyCell) return object; - let message = new $root.vtctldata.UpdateThrottlerConfigRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.enable != null) - message.enable = Boolean(object.enable); - if (object.disable != null) - message.disable = Boolean(object.disable); - if (object.threshold != null) - message.threshold = Number(object.threshold); - if (object.custom_query != null) - message.custom_query = String(object.custom_query); - if (object.custom_query_set != null) - message.custom_query_set = Boolean(object.custom_query_set); - if (object.check_as_check_self != null) - message.check_as_check_self = Boolean(object.check_as_check_self); - if (object.check_as_check_shard != null) - message.check_as_check_shard = Boolean(object.check_as_check_shard); - if (object.throttled_app != null) { - if (typeof object.throttled_app !== "object") - throw TypeError(".vtctldata.UpdateThrottlerConfigRequest.throttled_app: object expected"); - message.throttled_app = $root.topodata.ThrottledAppRule.fromObject(object.throttled_app); + let message = new $root.vtctldata.TopologyCell(); + if (object.name != null) + message.name = String(object.name); + if (object.path != null) + message.path = String(object.path); + if (object.data != null) + message.data = String(object.data); + if (object.children) { + if (!Array.isArray(object.children)) + throw TypeError(".vtctldata.TopologyCell.children: array expected"); + message.children = []; + for (let i = 0; i < object.children.length; ++i) + message.children[i] = String(object.children[i]); } return message; }; /** - * Creates a plain object from an UpdateThrottlerConfigRequest message. Also converts values to other types if specified. + * Creates a plain object from a TopologyCell message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static - * @param {vtctldata.UpdateThrottlerConfigRequest} message UpdateThrottlerConfigRequest + * @param {vtctldata.TopologyCell} message TopologyCell * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateThrottlerConfigRequest.toObject = function toObject(message, options) { + TopologyCell.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.children = []; if (options.defaults) { - object.keyspace = ""; - object.enable = false; - object.disable = false; - object.threshold = 0; - object.custom_query = ""; - object.custom_query_set = false; - object.check_as_check_self = false; - object.check_as_check_shard = false; - object.throttled_app = null; + object.name = ""; + object.path = ""; + object.data = ""; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.path != null && message.hasOwnProperty("path")) + object.path = message.path; + if (message.data != null && message.hasOwnProperty("data")) + object.data = message.data; + if (message.children && message.children.length) { + object.children = []; + for (let j = 0; j < message.children.length; ++j) + object.children[j] = message.children[j]; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.enable != null && message.hasOwnProperty("enable")) - object.enable = message.enable; - if (message.disable != null && message.hasOwnProperty("disable")) - object.disable = message.disable; - if (message.threshold != null && message.hasOwnProperty("threshold")) - object.threshold = options.json && !isFinite(message.threshold) ? String(message.threshold) : message.threshold; - if (message.custom_query != null && message.hasOwnProperty("custom_query")) - object.custom_query = message.custom_query; - if (message.custom_query_set != null && message.hasOwnProperty("custom_query_set")) - object.custom_query_set = message.custom_query_set; - if (message.check_as_check_self != null && message.hasOwnProperty("check_as_check_self")) - object.check_as_check_self = message.check_as_check_self; - if (message.check_as_check_shard != null && message.hasOwnProperty("check_as_check_shard")) - object.check_as_check_shard = message.check_as_check_shard; - if (message.throttled_app != null && message.hasOwnProperty("throttled_app")) - object.throttled_app = $root.topodata.ThrottledAppRule.toObject(message.throttled_app, options); return object; }; /** - * Converts this UpdateThrottlerConfigRequest to JSON. + * Converts this TopologyCell to JSON. * @function toJSON - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @instance * @returns {Object.} JSON object */ - UpdateThrottlerConfigRequest.prototype.toJSON = function toJSON() { + TopologyCell.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateThrottlerConfigRequest + * Gets the default type url for TopologyCell * @function getTypeUrl - * @memberof vtctldata.UpdateThrottlerConfigRequest + * @memberof vtctldata.TopologyCell * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateThrottlerConfigRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + TopologyCell.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateThrottlerConfigRequest"; + return typeUrlPrefix + "/vtctldata.TopologyCell"; }; - return UpdateThrottlerConfigRequest; + return TopologyCell; })(); - vtctldata.UpdateThrottlerConfigResponse = (function() { + vtctldata.GetVSchemaRequest = (function() { /** - * Properties of an UpdateThrottlerConfigResponse. + * Properties of a GetVSchemaRequest. * @memberof vtctldata - * @interface IUpdateThrottlerConfigResponse + * @interface IGetVSchemaRequest + * @property {string|null} [keyspace] GetVSchemaRequest keyspace */ /** - * Constructs a new UpdateThrottlerConfigResponse. + * Constructs a new GetVSchemaRequest. * @memberof vtctldata - * @classdesc Represents an UpdateThrottlerConfigResponse. - * @implements IUpdateThrottlerConfigResponse + * @classdesc Represents a GetVSchemaRequest. + * @implements IGetVSchemaRequest * @constructor - * @param {vtctldata.IUpdateThrottlerConfigResponse=} [properties] Properties to set + * @param {vtctldata.IGetVSchemaRequest=} [properties] Properties to set */ - function UpdateThrottlerConfigResponse(properties) { + function GetVSchemaRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -119642,63 +125389,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new UpdateThrottlerConfigResponse instance using the specified properties. + * GetVSchemaRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetVSchemaRequest + * @instance + */ + GetVSchemaRequest.prototype.keyspace = ""; + + /** + * Creates a new GetVSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static - * @param {vtctldata.IUpdateThrottlerConfigResponse=} [properties] Properties to set - * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse instance + * @param {vtctldata.IGetVSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest instance */ - UpdateThrottlerConfigResponse.create = function create(properties) { - return new UpdateThrottlerConfigResponse(properties); + GetVSchemaRequest.create = function create(properties) { + return new GetVSchemaRequest(properties); }; /** - * Encodes the specified UpdateThrottlerConfigResponse message. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. + * Encodes the specified GetVSchemaRequest message. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static - * @param {vtctldata.IUpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse message or plain object to encode + * @param {vtctldata.IGetVSchemaRequest} message GetVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateThrottlerConfigResponse.encode = function encode(message, writer) { + GetVSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); return writer; }; /** - * Encodes the specified UpdateThrottlerConfigResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateThrottlerConfigResponse.verify|verify} messages. + * Encodes the specified GetVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static - * @param {vtctldata.IUpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse message or plain object to encode + * @param {vtctldata.IGetVSchemaRequest} message GetVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateThrottlerConfigResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer. + * Decodes a GetVSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse + * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateThrottlerConfigResponse.decode = function decode(reader, length) { + GetVSchemaRequest.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.UpdateThrottlerConfigResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetVSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -119708,109 +125469,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateThrottlerConfigResponse message from the specified reader or buffer, length delimited. + * Decodes a GetVSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse + * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateThrottlerConfigResponse.decodeDelimited = function decodeDelimited(reader) { + GetVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateThrottlerConfigResponse message. + * Verifies a GetVSchemaRequest message. * @function verify - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateThrottlerConfigResponse.verify = function verify(message) { + GetVSchemaRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; return null; }; /** - * Creates an UpdateThrottlerConfigResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateThrottlerConfigResponse} UpdateThrottlerConfigResponse + * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest */ - UpdateThrottlerConfigResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateThrottlerConfigResponse) + GetVSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetVSchemaRequest) return object; - return new $root.vtctldata.UpdateThrottlerConfigResponse(); + let message = new $root.vtctldata.GetVSchemaRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + return message; }; /** - * Creates a plain object from an UpdateThrottlerConfigResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetVSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static - * @param {vtctldata.UpdateThrottlerConfigResponse} message UpdateThrottlerConfigResponse + * @param {vtctldata.GetVSchemaRequest} message GetVSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateThrottlerConfigResponse.toObject = function toObject() { - return {}; + GetVSchemaRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.keyspace = ""; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + return object; }; /** - * Converts this UpdateThrottlerConfigResponse to JSON. + * Converts this GetVSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @instance * @returns {Object.} JSON object */ - UpdateThrottlerConfigResponse.prototype.toJSON = function toJSON() { + GetVSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateThrottlerConfigResponse + * Gets the default type url for GetVSchemaRequest * @function getTypeUrl - * @memberof vtctldata.UpdateThrottlerConfigResponse + * @memberof vtctldata.GetVSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateThrottlerConfigResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateThrottlerConfigResponse"; + return typeUrlPrefix + "/vtctldata.GetVSchemaRequest"; }; - return UpdateThrottlerConfigResponse; + return GetVSchemaRequest; })(); - vtctldata.GetSrvVSchemaRequest = (function() { + vtctldata.GetVersionRequest = (function() { /** - * Properties of a GetSrvVSchemaRequest. + * Properties of a GetVersionRequest. * @memberof vtctldata - * @interface IGetSrvVSchemaRequest - * @property {string|null} [cell] GetSrvVSchemaRequest cell + * @interface IGetVersionRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] GetVersionRequest tablet_alias */ /** - * Constructs a new GetSrvVSchemaRequest. + * Constructs a new GetVersionRequest. * @memberof vtctldata - * @classdesc Represents a GetSrvVSchemaRequest. - * @implements IGetSrvVSchemaRequest + * @classdesc Represents a GetVersionRequest. + * @implements IGetVersionRequest * @constructor - * @param {vtctldata.IGetSrvVSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IGetVersionRequest=} [properties] Properties to set */ - function GetSrvVSchemaRequest(properties) { + function GetVersionRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -119818,75 +125592,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvVSchemaRequest cell. - * @member {string} cell - * @memberof vtctldata.GetSrvVSchemaRequest + * GetVersionRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.GetVersionRequest * @instance */ - GetSrvVSchemaRequest.prototype.cell = ""; + GetVersionRequest.prototype.tablet_alias = null; /** - * Creates a new GetSrvVSchemaRequest instance using the specified properties. + * Creates a new GetVersionRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static - * @param {vtctldata.IGetSrvVSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest instance + * @param {vtctldata.IGetVersionRequest=} [properties] Properties to set + * @returns {vtctldata.GetVersionRequest} GetVersionRequest instance */ - GetSrvVSchemaRequest.create = function create(properties) { - return new GetSrvVSchemaRequest(properties); + GetVersionRequest.create = function create(properties) { + return new GetVersionRequest(properties); }; /** - * Encodes the specified GetSrvVSchemaRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. + * Encodes the specified GetVersionRequest message. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static - * @param {vtctldata.IGetSrvVSchemaRequest} message GetSrvVSchemaRequest message or plain object to encode + * @param {vtctldata.IGetVersionRequest} message GetVersionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemaRequest.encode = function encode(message, writer) { + GetVersionRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.cell); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetSrvVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaRequest.verify|verify} messages. + * Encodes the specified GetVersionRequest message, length delimited. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static - * @param {vtctldata.IGetSrvVSchemaRequest} message GetSrvVSchemaRequest message or plain object to encode + * @param {vtctldata.IGetVersionRequest} message GetVersionRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetVersionRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer. + * Decodes a GetVersionRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest + * @returns {vtctldata.GetVersionRequest} GetVersionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemaRequest.decode = function decode(reader, length) { + GetVersionRequest.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.GetSrvVSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetVersionRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.cell = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -119898,122 +125672,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a GetVersionRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest + * @returns {vtctldata.GetVersionRequest} GetVersionRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + GetVersionRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvVSchemaRequest message. + * Verifies a GetVersionRequest message. * @function verify - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvVSchemaRequest.verify = function verify(message) { + GetVersionRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a GetSrvVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetVersionRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvVSchemaRequest} GetSrvVSchemaRequest + * @returns {vtctldata.GetVersionRequest} GetVersionRequest */ - GetSrvVSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvVSchemaRequest) + GetVersionRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetVersionRequest) return object; - let message = new $root.vtctldata.GetSrvVSchemaRequest(); - if (object.cell != null) - message.cell = String(object.cell); + let message = new $root.vtctldata.GetVersionRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.GetVersionRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a GetSrvVSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetVersionRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static - * @param {vtctldata.GetSrvVSchemaRequest} message GetSrvVSchemaRequest + * @param {vtctldata.GetVersionRequest} message GetVersionRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvVSchemaRequest.toObject = function toObject(message, options) { + GetVersionRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.cell = ""; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this GetSrvVSchemaRequest to JSON. + * Converts this GetVersionRequest to JSON. * @function toJSON - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @instance * @returns {Object.} JSON object */ - GetSrvVSchemaRequest.prototype.toJSON = function toJSON() { + GetVersionRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvVSchemaRequest + * Gets the default type url for GetVersionRequest * @function getTypeUrl - * @memberof vtctldata.GetSrvVSchemaRequest + * @memberof vtctldata.GetVersionRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetVersionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvVSchemaRequest"; + return typeUrlPrefix + "/vtctldata.GetVersionRequest"; }; - return GetSrvVSchemaRequest; + return GetVersionRequest; })(); - vtctldata.GetSrvVSchemaResponse = (function() { + vtctldata.GetVersionResponse = (function() { /** - * Properties of a GetSrvVSchemaResponse. + * Properties of a GetVersionResponse. * @memberof vtctldata - * @interface IGetSrvVSchemaResponse - * @property {vschema.ISrvVSchema|null} [srv_v_schema] GetSrvVSchemaResponse srv_v_schema + * @interface IGetVersionResponse + * @property {string|null} [version] GetVersionResponse version */ /** - * Constructs a new GetSrvVSchemaResponse. + * Constructs a new GetVersionResponse. * @memberof vtctldata - * @classdesc Represents a GetSrvVSchemaResponse. - * @implements IGetSrvVSchemaResponse + * @classdesc Represents a GetVersionResponse. + * @implements IGetVersionResponse * @constructor - * @param {vtctldata.IGetSrvVSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IGetVersionResponse=} [properties] Properties to set */ - function GetSrvVSchemaResponse(properties) { + function GetVersionResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -120021,75 +125800,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvVSchemaResponse srv_v_schema. - * @member {vschema.ISrvVSchema|null|undefined} srv_v_schema - * @memberof vtctldata.GetSrvVSchemaResponse + * GetVersionResponse version. + * @member {string} version + * @memberof vtctldata.GetVersionResponse * @instance */ - GetSrvVSchemaResponse.prototype.srv_v_schema = null; + GetVersionResponse.prototype.version = ""; /** - * Creates a new GetSrvVSchemaResponse instance using the specified properties. + * Creates a new GetVersionResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static - * @param {vtctldata.IGetSrvVSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse instance + * @param {vtctldata.IGetVersionResponse=} [properties] Properties to set + * @returns {vtctldata.GetVersionResponse} GetVersionResponse instance */ - GetSrvVSchemaResponse.create = function create(properties) { - return new GetSrvVSchemaResponse(properties); + GetVersionResponse.create = function create(properties) { + return new GetVersionResponse(properties); }; /** - * Encodes the specified GetSrvVSchemaResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. + * Encodes the specified GetVersionResponse message. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static - * @param {vtctldata.IGetSrvVSchemaResponse} message GetSrvVSchemaResponse message or plain object to encode + * @param {vtctldata.IGetVersionResponse} message GetVersionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemaResponse.encode = function encode(message, writer) { + GetVersionResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.srv_v_schema != null && Object.hasOwnProperty.call(message, "srv_v_schema")) - $root.vschema.SrvVSchema.encode(message.srv_v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.version != null && Object.hasOwnProperty.call(message, "version")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.version); return writer; }; /** - * Encodes the specified GetSrvVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemaResponse.verify|verify} messages. + * Encodes the specified GetVersionResponse message, length delimited. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static - * @param {vtctldata.IGetSrvVSchemaResponse} message GetSrvVSchemaResponse message or plain object to encode + * @param {vtctldata.IGetVersionResponse} message GetVersionResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetVersionResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer. + * Decodes a GetVersionResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse + * @returns {vtctldata.GetVersionResponse} GetVersionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemaResponse.decode = function decode(reader, length) { + GetVersionResponse.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.GetSrvVSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetVersionResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.srv_v_schema = $root.vschema.SrvVSchema.decode(reader, reader.uint32()); + message.version = reader.string(); break; } default: @@ -120101,128 +125880,122 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a GetVersionResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse + * @returns {vtctldata.GetVersionResponse} GetVersionResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + GetVersionResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvVSchemaResponse message. + * Verifies a GetVersionResponse message. * @function verify - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvVSchemaResponse.verify = function verify(message) { + GetVersionResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.srv_v_schema != null && message.hasOwnProperty("srv_v_schema")) { - let error = $root.vschema.SrvVSchema.verify(message.srv_v_schema); - if (error) - return "srv_v_schema." + error; - } + if (message.version != null && message.hasOwnProperty("version")) + if (!$util.isString(message.version)) + return "version: string expected"; return null; }; /** - * Creates a GetSrvVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetVersionResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvVSchemaResponse} GetSrvVSchemaResponse + * @returns {vtctldata.GetVersionResponse} GetVersionResponse */ - GetSrvVSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvVSchemaResponse) + GetVersionResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetVersionResponse) return object; - let message = new $root.vtctldata.GetSrvVSchemaResponse(); - if (object.srv_v_schema != null) { - if (typeof object.srv_v_schema !== "object") - throw TypeError(".vtctldata.GetSrvVSchemaResponse.srv_v_schema: object expected"); - message.srv_v_schema = $root.vschema.SrvVSchema.fromObject(object.srv_v_schema); - } + let message = new $root.vtctldata.GetVersionResponse(); + if (object.version != null) + message.version = String(object.version); return message; }; /** - * Creates a plain object from a GetSrvVSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetVersionResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static - * @param {vtctldata.GetSrvVSchemaResponse} message GetSrvVSchemaResponse + * @param {vtctldata.GetVersionResponse} message GetVersionResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvVSchemaResponse.toObject = function toObject(message, options) { + GetVersionResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.srv_v_schema = null; - if (message.srv_v_schema != null && message.hasOwnProperty("srv_v_schema")) - object.srv_v_schema = $root.vschema.SrvVSchema.toObject(message.srv_v_schema, options); + object.version = ""; + if (message.version != null && message.hasOwnProperty("version")) + object.version = message.version; return object; }; /** - * Converts this GetSrvVSchemaResponse to JSON. + * Converts this GetVersionResponse to JSON. * @function toJSON - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @instance * @returns {Object.} JSON object */ - GetSrvVSchemaResponse.prototype.toJSON = function toJSON() { + GetVersionResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvVSchemaResponse + * Gets the default type url for GetVersionResponse * @function getTypeUrl - * @memberof vtctldata.GetSrvVSchemaResponse + * @memberof vtctldata.GetVersionResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetVersionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvVSchemaResponse"; + return typeUrlPrefix + "/vtctldata.GetVersionResponse"; }; - return GetSrvVSchemaResponse; + return GetVersionResponse; })(); - vtctldata.GetSrvVSchemasRequest = (function() { + vtctldata.GetVSchemaResponse = (function() { /** - * Properties of a GetSrvVSchemasRequest. + * Properties of a GetVSchemaResponse. * @memberof vtctldata - * @interface IGetSrvVSchemasRequest - * @property {Array.|null} [cells] GetSrvVSchemasRequest cells + * @interface IGetVSchemaResponse + * @property {vschema.IKeyspace|null} [v_schema] GetVSchemaResponse v_schema */ /** - * Constructs a new GetSrvVSchemasRequest. + * Constructs a new GetVSchemaResponse. * @memberof vtctldata - * @classdesc Represents a GetSrvVSchemasRequest. - * @implements IGetSrvVSchemasRequest + * @classdesc Represents a GetVSchemaResponse. + * @implements IGetVSchemaResponse * @constructor - * @param {vtctldata.IGetSrvVSchemasRequest=} [properties] Properties to set + * @param {vtctldata.IGetVSchemaResponse=} [properties] Properties to set */ - function GetSrvVSchemasRequest(properties) { - this.cells = []; + function GetVSchemaResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -120230,78 +126003,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvVSchemasRequest cells. - * @member {Array.} cells - * @memberof vtctldata.GetSrvVSchemasRequest + * GetVSchemaResponse v_schema. + * @member {vschema.IKeyspace|null|undefined} v_schema + * @memberof vtctldata.GetVSchemaResponse * @instance */ - GetSrvVSchemasRequest.prototype.cells = $util.emptyArray; + GetVSchemaResponse.prototype.v_schema = null; /** - * Creates a new GetSrvVSchemasRequest instance using the specified properties. + * Creates a new GetVSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static - * @param {vtctldata.IGetSrvVSchemasRequest=} [properties] Properties to set - * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest instance + * @param {vtctldata.IGetVSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse instance */ - GetSrvVSchemasRequest.create = function create(properties) { - return new GetSrvVSchemasRequest(properties); + GetVSchemaResponse.create = function create(properties) { + return new GetVSchemaResponse(properties); }; /** - * Encodes the specified GetSrvVSchemasRequest message. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. + * Encodes the specified GetVSchemaResponse message. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static - * @param {vtctldata.IGetSrvVSchemasRequest} message GetSrvVSchemasRequest message or plain object to encode + * @param {vtctldata.IGetVSchemaResponse} message GetVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemasRequest.encode = function encode(message, writer) { + GetVSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) + $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetSrvVSchemasRequest message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasRequest.verify|verify} messages. + * Encodes the specified GetVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static - * @param {vtctldata.IGetSrvVSchemasRequest} message GetSrvVSchemasRequest message or plain object to encode + * @param {vtctldata.IGetVSchemaResponse} message GetVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemasRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer. + * Decodes a GetVSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest + * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemasRequest.decode = function decode(reader, length) { + GetVSchemaResponse.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.GetSrvVSchemasRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetVSchemaResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + case 1: { + message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); break; } default: @@ -120313,135 +126083,130 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvVSchemasRequest message from the specified reader or buffer, length delimited. + * Decodes a GetVSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest + * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemasRequest.decodeDelimited = function decodeDelimited(reader) { + GetVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvVSchemasRequest message. + * Verifies a GetVSchemaResponse message. * @function verify - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvVSchemasRequest.verify = function verify(message) { + GetVSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; + if (message.v_schema != null && message.hasOwnProperty("v_schema")) { + let error = $root.vschema.Keyspace.verify(message.v_schema); + if (error) + return "v_schema." + error; } return null; }; /** - * Creates a GetSrvVSchemasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvVSchemasRequest} GetSrvVSchemasRequest + * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse */ - GetSrvVSchemasRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvVSchemasRequest) + GetVSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetVSchemaResponse) return object; - let message = new $root.vtctldata.GetSrvVSchemasRequest(); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.GetSrvVSchemasRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); + let message = new $root.vtctldata.GetVSchemaResponse(); + if (object.v_schema != null) { + if (typeof object.v_schema !== "object") + throw TypeError(".vtctldata.GetVSchemaResponse.v_schema: object expected"); + message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); } return message; }; /** - * Creates a plain object from a GetSrvVSchemasRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetVSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static - * @param {vtctldata.GetSrvVSchemasRequest} message GetSrvVSchemasRequest + * @param {vtctldata.GetVSchemaResponse} message GetVSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvVSchemasRequest.toObject = function toObject(message, options) { + GetVSchemaResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } + if (options.defaults) + object.v_schema = null; + if (message.v_schema != null && message.hasOwnProperty("v_schema")) + object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); return object; }; /** - * Converts this GetSrvVSchemasRequest to JSON. + * Converts this GetVSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @instance * @returns {Object.} JSON object */ - GetSrvVSchemasRequest.prototype.toJSON = function toJSON() { + GetVSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvVSchemasRequest + * Gets the default type url for GetVSchemaResponse * @function getTypeUrl - * @memberof vtctldata.GetSrvVSchemasRequest + * @memberof vtctldata.GetVSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvVSchemasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvVSchemasRequest"; + return typeUrlPrefix + "/vtctldata.GetVSchemaResponse"; }; - return GetSrvVSchemasRequest; + return GetVSchemaResponse; })(); - vtctldata.GetSrvVSchemasResponse = (function() { + vtctldata.GetWorkflowsRequest = (function() { /** - * Properties of a GetSrvVSchemasResponse. + * Properties of a GetWorkflowsRequest. * @memberof vtctldata - * @interface IGetSrvVSchemasResponse - * @property {Object.|null} [srv_v_schemas] GetSrvVSchemasResponse srv_v_schemas + * @interface IGetWorkflowsRequest + * @property {string|null} [keyspace] GetWorkflowsRequest keyspace + * @property {boolean|null} [active_only] GetWorkflowsRequest active_only + * @property {boolean|null} [name_only] GetWorkflowsRequest name_only + * @property {string|null} [workflow] GetWorkflowsRequest workflow */ /** - * Constructs a new GetSrvVSchemasResponse. + * Constructs a new GetWorkflowsRequest. * @memberof vtctldata - * @classdesc Represents a GetSrvVSchemasResponse. - * @implements IGetSrvVSchemasResponse + * @classdesc Represents a GetWorkflowsRequest. + * @implements IGetWorkflowsRequest * @constructor - * @param {vtctldata.IGetSrvVSchemasResponse=} [properties] Properties to set + * @param {vtctldata.IGetWorkflowsRequest=} [properties] Properties to set */ - function GetSrvVSchemasResponse(properties) { - this.srv_v_schemas = {}; + function GetWorkflowsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -120449,97 +126214,117 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetSrvVSchemasResponse srv_v_schemas. - * @member {Object.} srv_v_schemas - * @memberof vtctldata.GetSrvVSchemasResponse + * GetWorkflowsRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.GetWorkflowsRequest * @instance */ - GetSrvVSchemasResponse.prototype.srv_v_schemas = $util.emptyObject; + GetWorkflowsRequest.prototype.keyspace = ""; /** - * Creates a new GetSrvVSchemasResponse instance using the specified properties. + * GetWorkflowsRequest active_only. + * @member {boolean} active_only + * @memberof vtctldata.GetWorkflowsRequest + * @instance + */ + GetWorkflowsRequest.prototype.active_only = false; + + /** + * GetWorkflowsRequest name_only. + * @member {boolean} name_only + * @memberof vtctldata.GetWorkflowsRequest + * @instance + */ + GetWorkflowsRequest.prototype.name_only = false; + + /** + * GetWorkflowsRequest workflow. + * @member {string} workflow + * @memberof vtctldata.GetWorkflowsRequest + * @instance + */ + GetWorkflowsRequest.prototype.workflow = ""; + + /** + * Creates a new GetWorkflowsRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static - * @param {vtctldata.IGetSrvVSchemasResponse=} [properties] Properties to set - * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse instance + * @param {vtctldata.IGetWorkflowsRequest=} [properties] Properties to set + * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest instance */ - GetSrvVSchemasResponse.create = function create(properties) { - return new GetSrvVSchemasResponse(properties); + GetWorkflowsRequest.create = function create(properties) { + return new GetWorkflowsRequest(properties); }; /** - * Encodes the specified GetSrvVSchemasResponse message. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. + * Encodes the specified GetWorkflowsRequest message. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static - * @param {vtctldata.IGetSrvVSchemasResponse} message GetSrvVSchemasResponse message or plain object to encode + * @param {vtctldata.IGetWorkflowsRequest} message GetWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemasResponse.encode = function encode(message, writer) { + GetWorkflowsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.srv_v_schemas != null && Object.hasOwnProperty.call(message, "srv_v_schemas")) - for (let keys = Object.keys(message.srv_v_schemas), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vschema.SrvVSchema.encode(message.srv_v_schemas[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.active_only != null && Object.hasOwnProperty.call(message, "active_only")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.active_only); + if (message.name_only != null && Object.hasOwnProperty.call(message, "name_only")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.name_only); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.workflow); return writer; }; /** - * Encodes the specified GetSrvVSchemasResponse message, length delimited. Does not implicitly {@link vtctldata.GetSrvVSchemasResponse.verify|verify} messages. + * Encodes the specified GetWorkflowsRequest message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static - * @param {vtctldata.IGetSrvVSchemasResponse} message GetSrvVSchemasResponse message or plain object to encode + * @param {vtctldata.IGetWorkflowsRequest} message GetWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetSrvVSchemasResponse.encodeDelimited = function encodeDelimited(message, writer) { + GetWorkflowsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer. + * Decodes a GetWorkflowsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse + * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemasResponse.decode = function decode(reader, length) { + GetWorkflowsRequest.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.GetSrvVSchemasResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetWorkflowsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.srv_v_schemas === $util.emptyObject) - message.srv_v_schemas = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vschema.SrvVSchema.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.srv_v_schemas[key] = value; + message.keyspace = reader.string(); + break; + } + case 2: { + message.active_only = reader.bool(); + break; + } + case 3: { + message.name_only = reader.bool(); + break; + } + case 4: { + message.workflow = reader.string(); break; } default: @@ -120551,141 +126336,148 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetSrvVSchemasResponse message from the specified reader or buffer, length delimited. + * Decodes a GetWorkflowsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse + * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetSrvVSchemasResponse.decodeDelimited = function decodeDelimited(reader) { + GetWorkflowsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetSrvVSchemasResponse message. + * Verifies a GetWorkflowsRequest message. * @function verify - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetSrvVSchemasResponse.verify = function verify(message) { + GetWorkflowsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.srv_v_schemas != null && message.hasOwnProperty("srv_v_schemas")) { - if (!$util.isObject(message.srv_v_schemas)) - return "srv_v_schemas: object expected"; - let key = Object.keys(message.srv_v_schemas); - for (let i = 0; i < key.length; ++i) { - let error = $root.vschema.SrvVSchema.verify(message.srv_v_schemas[key[i]]); - if (error) - return "srv_v_schemas." + error; - } - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.active_only != null && message.hasOwnProperty("active_only")) + if (typeof message.active_only !== "boolean") + return "active_only: boolean expected"; + if (message.name_only != null && message.hasOwnProperty("name_only")) + if (typeof message.name_only !== "boolean") + return "name_only: boolean expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; return null; }; /** - * Creates a GetSrvVSchemasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a GetWorkflowsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetSrvVSchemasResponse} GetSrvVSchemasResponse + * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest */ - GetSrvVSchemasResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetSrvVSchemasResponse) + GetWorkflowsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetWorkflowsRequest) return object; - let message = new $root.vtctldata.GetSrvVSchemasResponse(); - if (object.srv_v_schemas) { - if (typeof object.srv_v_schemas !== "object") - throw TypeError(".vtctldata.GetSrvVSchemasResponse.srv_v_schemas: object expected"); - message.srv_v_schemas = {}; - for (let keys = Object.keys(object.srv_v_schemas), i = 0; i < keys.length; ++i) { - if (typeof object.srv_v_schemas[keys[i]] !== "object") - throw TypeError(".vtctldata.GetSrvVSchemasResponse.srv_v_schemas: object expected"); - message.srv_v_schemas[keys[i]] = $root.vschema.SrvVSchema.fromObject(object.srv_v_schemas[keys[i]]); - } - } + let message = new $root.vtctldata.GetWorkflowsRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.active_only != null) + message.active_only = Boolean(object.active_only); + if (object.name_only != null) + message.name_only = Boolean(object.name_only); + if (object.workflow != null) + message.workflow = String(object.workflow); return message; }; /** - * Creates a plain object from a GetSrvVSchemasResponse message. Also converts values to other types if specified. + * Creates a plain object from a GetWorkflowsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static - * @param {vtctldata.GetSrvVSchemasResponse} message GetSrvVSchemasResponse + * @param {vtctldata.GetWorkflowsRequest} message GetWorkflowsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetSrvVSchemasResponse.toObject = function toObject(message, options) { + GetWorkflowsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) - object.srv_v_schemas = {}; - let keys2; - if (message.srv_v_schemas && (keys2 = Object.keys(message.srv_v_schemas)).length) { - object.srv_v_schemas = {}; - for (let j = 0; j < keys2.length; ++j) - object.srv_v_schemas[keys2[j]] = $root.vschema.SrvVSchema.toObject(message.srv_v_schemas[keys2[j]], options); + if (options.defaults) { + object.keyspace = ""; + object.active_only = false; + object.name_only = false; + object.workflow = ""; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.active_only != null && message.hasOwnProperty("active_only")) + object.active_only = message.active_only; + if (message.name_only != null && message.hasOwnProperty("name_only")) + object.name_only = message.name_only; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; return object; }; /** - * Converts this GetSrvVSchemasResponse to JSON. + * Converts this GetWorkflowsRequest to JSON. * @function toJSON - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @instance * @returns {Object.} JSON object */ - GetSrvVSchemasResponse.prototype.toJSON = function toJSON() { + GetWorkflowsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetSrvVSchemasResponse + * Gets the default type url for GetWorkflowsRequest * @function getTypeUrl - * @memberof vtctldata.GetSrvVSchemasResponse + * @memberof vtctldata.GetWorkflowsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetSrvVSchemasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetWorkflowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetSrvVSchemasResponse"; + return typeUrlPrefix + "/vtctldata.GetWorkflowsRequest"; }; - return GetSrvVSchemasResponse; + return GetWorkflowsRequest; })(); - vtctldata.GetTabletRequest = (function() { + vtctldata.GetWorkflowsResponse = (function() { /** - * Properties of a GetTabletRequest. + * Properties of a GetWorkflowsResponse. * @memberof vtctldata - * @interface IGetTabletRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] GetTabletRequest tablet_alias + * @interface IGetWorkflowsResponse + * @property {Array.|null} [workflows] GetWorkflowsResponse workflows */ /** - * Constructs a new GetTabletRequest. + * Constructs a new GetWorkflowsResponse. * @memberof vtctldata - * @classdesc Represents a GetTabletRequest. - * @implements IGetTabletRequest + * @classdesc Represents a GetWorkflowsResponse. + * @implements IGetWorkflowsResponse * @constructor - * @param {vtctldata.IGetTabletRequest=} [properties] Properties to set + * @param {vtctldata.IGetWorkflowsResponse=} [properties] Properties to set */ - function GetTabletRequest(properties) { + function GetWorkflowsResponse(properties) { + this.workflows = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -120693,75 +126485,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTabletRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.GetTabletRequest + * GetWorkflowsResponse workflows. + * @member {Array.} workflows + * @memberof vtctldata.GetWorkflowsResponse * @instance */ - GetTabletRequest.prototype.tablet_alias = null; + GetWorkflowsResponse.prototype.workflows = $util.emptyArray; /** - * Creates a new GetTabletRequest instance using the specified properties. + * Creates a new GetWorkflowsResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static - * @param {vtctldata.IGetTabletRequest=} [properties] Properties to set - * @returns {vtctldata.GetTabletRequest} GetTabletRequest instance + * @param {vtctldata.IGetWorkflowsResponse=} [properties] Properties to set + * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse instance */ - GetTabletRequest.create = function create(properties) { - return new GetTabletRequest(properties); + GetWorkflowsResponse.create = function create(properties) { + return new GetWorkflowsResponse(properties); }; /** - * Encodes the specified GetTabletRequest message. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. + * Encodes the specified GetWorkflowsResponse message. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static - * @param {vtctldata.IGetTabletRequest} message GetTabletRequest message or plain object to encode + * @param {vtctldata.IGetWorkflowsResponse} message GetWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletRequest.encode = function encode(message, writer) { + GetWorkflowsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.workflows != null && message.workflows.length) + for (let i = 0; i < message.workflows.length; ++i) + $root.vtctldata.Workflow.encode(message.workflows[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetTabletRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletRequest.verify|verify} messages. + * Encodes the specified GetWorkflowsResponse message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static - * @param {vtctldata.IGetTabletRequest} message GetTabletRequest message or plain object to encode + * @param {vtctldata.IGetWorkflowsResponse} message GetWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { + GetWorkflowsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTabletRequest message from the specified reader or buffer. + * Decodes a GetWorkflowsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTabletRequest} GetTabletRequest + * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletRequest.decode = function decode(reader, length) { + GetWorkflowsResponse.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.GetTabletRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.GetWorkflowsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + if (!(message.workflows && message.workflows.length)) + message.workflows = []; + message.workflows.push($root.vtctldata.Workflow.decode(reader, reader.uint32())); break; } default: @@ -120773,127 +126568,143 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a GetWorkflowsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTabletRequest} GetTabletRequest + * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletRequest.decodeDelimited = function decodeDelimited(reader) { + GetWorkflowsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetTabletRequest message. + * Verifies a GetWorkflowsResponse message. * @function verify - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetTabletRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; + GetWorkflowsResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflows != null && message.hasOwnProperty("workflows")) { + if (!Array.isArray(message.workflows)) + return "workflows: array expected"; + for (let i = 0; i < message.workflows.length; ++i) { + let error = $root.vtctldata.Workflow.verify(message.workflows[i]); + if (error) + return "workflows." + error; + } } return null; }; /** - * Creates a GetTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a GetWorkflowsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetTabletRequest} GetTabletRequest + * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse */ - GetTabletRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTabletRequest) + GetWorkflowsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetWorkflowsResponse) return object; - let message = new $root.vtctldata.GetTabletRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.GetTabletRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + let message = new $root.vtctldata.GetWorkflowsResponse(); + if (object.workflows) { + if (!Array.isArray(object.workflows)) + throw TypeError(".vtctldata.GetWorkflowsResponse.workflows: array expected"); + message.workflows = []; + for (let i = 0; i < object.workflows.length; ++i) { + if (typeof object.workflows[i] !== "object") + throw TypeError(".vtctldata.GetWorkflowsResponse.workflows: object expected"); + message.workflows[i] = $root.vtctldata.Workflow.fromObject(object.workflows[i]); + } } return message; }; /** - * Creates a plain object from a GetTabletRequest message. Also converts values to other types if specified. + * Creates a plain object from a GetWorkflowsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static - * @param {vtctldata.GetTabletRequest} message GetTabletRequest + * @param {vtctldata.GetWorkflowsResponse} message GetWorkflowsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTabletRequest.toObject = function toObject(message, options) { + GetWorkflowsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.arrays || options.defaults) + object.workflows = []; + if (message.workflows && message.workflows.length) { + object.workflows = []; + for (let j = 0; j < message.workflows.length; ++j) + object.workflows[j] = $root.vtctldata.Workflow.toObject(message.workflows[j], options); + } return object; }; /** - * Converts this GetTabletRequest to JSON. + * Converts this GetWorkflowsResponse to JSON. * @function toJSON - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @instance * @returns {Object.} JSON object */ - GetTabletRequest.prototype.toJSON = function toJSON() { + GetWorkflowsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTabletRequest + * Gets the default type url for GetWorkflowsResponse * @function getTypeUrl - * @memberof vtctldata.GetTabletRequest + * @memberof vtctldata.GetWorkflowsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + GetWorkflowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTabletRequest"; + return typeUrlPrefix + "/vtctldata.GetWorkflowsResponse"; }; - return GetTabletRequest; + return GetWorkflowsResponse; })(); - vtctldata.GetTabletResponse = (function() { + vtctldata.InitShardPrimaryRequest = (function() { /** - * Properties of a GetTabletResponse. + * Properties of an InitShardPrimaryRequest. * @memberof vtctldata - * @interface IGetTabletResponse - * @property {topodata.ITablet|null} [tablet] GetTabletResponse tablet + * @interface IInitShardPrimaryRequest + * @property {string|null} [keyspace] InitShardPrimaryRequest keyspace + * @property {string|null} [shard] InitShardPrimaryRequest shard + * @property {topodata.ITabletAlias|null} [primary_elect_tablet_alias] InitShardPrimaryRequest primary_elect_tablet_alias + * @property {boolean|null} [force] InitShardPrimaryRequest force + * @property {vttime.IDuration|null} [wait_replicas_timeout] InitShardPrimaryRequest wait_replicas_timeout */ /** - * Constructs a new GetTabletResponse. + * Constructs a new InitShardPrimaryRequest. * @memberof vtctldata - * @classdesc Represents a GetTabletResponse. - * @implements IGetTabletResponse + * @classdesc Represents an InitShardPrimaryRequest. + * @implements IInitShardPrimaryRequest * @constructor - * @param {vtctldata.IGetTabletResponse=} [properties] Properties to set + * @param {vtctldata.IInitShardPrimaryRequest=} [properties] Properties to set */ - function GetTabletResponse(properties) { + function InitShardPrimaryRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -120901,75 +126712,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTabletResponse tablet. - * @member {topodata.ITablet|null|undefined} tablet - * @memberof vtctldata.GetTabletResponse + * InitShardPrimaryRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.InitShardPrimaryRequest * @instance */ - GetTabletResponse.prototype.tablet = null; + InitShardPrimaryRequest.prototype.keyspace = ""; /** - * Creates a new GetTabletResponse instance using the specified properties. + * InitShardPrimaryRequest shard. + * @member {string} shard + * @memberof vtctldata.InitShardPrimaryRequest + * @instance + */ + InitShardPrimaryRequest.prototype.shard = ""; + + /** + * InitShardPrimaryRequest primary_elect_tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} primary_elect_tablet_alias + * @memberof vtctldata.InitShardPrimaryRequest + * @instance + */ + InitShardPrimaryRequest.prototype.primary_elect_tablet_alias = null; + + /** + * InitShardPrimaryRequest force. + * @member {boolean} force + * @memberof vtctldata.InitShardPrimaryRequest + * @instance + */ + InitShardPrimaryRequest.prototype.force = false; + + /** + * InitShardPrimaryRequest wait_replicas_timeout. + * @member {vttime.IDuration|null|undefined} wait_replicas_timeout + * @memberof vtctldata.InitShardPrimaryRequest + * @instance + */ + InitShardPrimaryRequest.prototype.wait_replicas_timeout = null; + + /** + * Creates a new InitShardPrimaryRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static - * @param {vtctldata.IGetTabletResponse=} [properties] Properties to set - * @returns {vtctldata.GetTabletResponse} GetTabletResponse instance + * @param {vtctldata.IInitShardPrimaryRequest=} [properties] Properties to set + * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest instance */ - GetTabletResponse.create = function create(properties) { - return new GetTabletResponse(properties); + InitShardPrimaryRequest.create = function create(properties) { + return new InitShardPrimaryRequest(properties); }; /** - * Encodes the specified GetTabletResponse message. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. + * Encodes the specified InitShardPrimaryRequest message. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static - * @param {vtctldata.IGetTabletResponse} message GetTabletResponse message or plain object to encode + * @param {vtctldata.IInitShardPrimaryRequest} message InitShardPrimaryRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletResponse.encode = function encode(message, writer) { + InitShardPrimaryRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) - $root.topodata.Tablet.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.primary_elect_tablet_alias != null && Object.hasOwnProperty.call(message, "primary_elect_tablet_alias")) + $root.topodata.TabletAlias.encode(message.primary_elect_tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); + if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) + $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetTabletResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletResponse.verify|verify} messages. + * Encodes the specified InitShardPrimaryRequest message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static - * @param {vtctldata.IGetTabletResponse} message GetTabletResponse message or plain object to encode + * @param {vtctldata.IInitShardPrimaryRequest} message InitShardPrimaryRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { + InitShardPrimaryRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTabletResponse message from the specified reader or buffer. + * Decodes an InitShardPrimaryRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTabletResponse} GetTabletResponse + * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletResponse.decode = function decode(reader, length) { + InitShardPrimaryRequest.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.GetTabletResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.InitShardPrimaryRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet = $root.topodata.Tablet.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.primary_elect_tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + message.force = reader.bool(); + break; + } + case 5: { + message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); break; } default: @@ -120981,134 +126848,166 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTabletResponse message from the specified reader or buffer, length delimited. + * Decodes an InitShardPrimaryRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTabletResponse} GetTabletResponse + * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletResponse.decodeDelimited = function decodeDelimited(reader) { + InitShardPrimaryRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetTabletResponse message. + * Verifies an InitShardPrimaryRequest message. * @function verify - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetTabletResponse.verify = function verify(message) { + InitShardPrimaryRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet != null && message.hasOwnProperty("tablet")) { - let error = $root.topodata.Tablet.verify(message.tablet); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.primary_elect_tablet_alias != null && message.hasOwnProperty("primary_elect_tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.primary_elect_tablet_alias); if (error) - return "tablet." + error; + return "primary_elect_tablet_alias." + error; + } + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { + let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); + if (error) + return "wait_replicas_timeout." + error; } return null; }; /** - * Creates a GetTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates an InitShardPrimaryRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetTabletResponse} GetTabletResponse + * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest */ - GetTabletResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTabletResponse) + InitShardPrimaryRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.InitShardPrimaryRequest) return object; - let message = new $root.vtctldata.GetTabletResponse(); - if (object.tablet != null) { - if (typeof object.tablet !== "object") - throw TypeError(".vtctldata.GetTabletResponse.tablet: object expected"); - message.tablet = $root.topodata.Tablet.fromObject(object.tablet); + let message = new $root.vtctldata.InitShardPrimaryRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.primary_elect_tablet_alias != null) { + if (typeof object.primary_elect_tablet_alias !== "object") + throw TypeError(".vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias: object expected"); + message.primary_elect_tablet_alias = $root.topodata.TabletAlias.fromObject(object.primary_elect_tablet_alias); + } + if (object.force != null) + message.force = Boolean(object.force); + if (object.wait_replicas_timeout != null) { + if (typeof object.wait_replicas_timeout !== "object") + throw TypeError(".vtctldata.InitShardPrimaryRequest.wait_replicas_timeout: object expected"); + message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); } return message; }; /** - * Creates a plain object from a GetTabletResponse message. Also converts values to other types if specified. + * Creates a plain object from an InitShardPrimaryRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static - * @param {vtctldata.GetTabletResponse} message GetTabletResponse + * @param {vtctldata.InitShardPrimaryRequest} message InitShardPrimaryRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTabletResponse.toObject = function toObject(message, options) { + InitShardPrimaryRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet = null; - if (message.tablet != null && message.hasOwnProperty("tablet")) - object.tablet = $root.topodata.Tablet.toObject(message.tablet, options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.primary_elect_tablet_alias = null; + object.force = false; + object.wait_replicas_timeout = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.primary_elect_tablet_alias != null && message.hasOwnProperty("primary_elect_tablet_alias")) + object.primary_elect_tablet_alias = $root.topodata.TabletAlias.toObject(message.primary_elect_tablet_alias, options); + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) + object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); return object; }; /** - * Converts this GetTabletResponse to JSON. + * Converts this InitShardPrimaryRequest to JSON. * @function toJSON - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @instance * @returns {Object.} JSON object */ - GetTabletResponse.prototype.toJSON = function toJSON() { + InitShardPrimaryRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTabletResponse + * Gets the default type url for InitShardPrimaryRequest * @function getTypeUrl - * @memberof vtctldata.GetTabletResponse + * @memberof vtctldata.InitShardPrimaryRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + InitShardPrimaryRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTabletResponse"; + return typeUrlPrefix + "/vtctldata.InitShardPrimaryRequest"; }; - return GetTabletResponse; + return InitShardPrimaryRequest; })(); - vtctldata.GetTabletsRequest = (function() { + vtctldata.InitShardPrimaryResponse = (function() { /** - * Properties of a GetTabletsRequest. + * Properties of an InitShardPrimaryResponse. * @memberof vtctldata - * @interface IGetTabletsRequest - * @property {string|null} [keyspace] GetTabletsRequest keyspace - * @property {string|null} [shard] GetTabletsRequest shard - * @property {Array.|null} [cells] GetTabletsRequest cells - * @property {boolean|null} [strict] GetTabletsRequest strict - * @property {Array.|null} [tablet_aliases] GetTabletsRequest tablet_aliases - * @property {topodata.TabletType|null} [tablet_type] GetTabletsRequest tablet_type + * @interface IInitShardPrimaryResponse + * @property {Array.|null} [events] InitShardPrimaryResponse events */ /** - * Constructs a new GetTabletsRequest. + * Constructs a new InitShardPrimaryResponse. * @memberof vtctldata - * @classdesc Represents a GetTabletsRequest. - * @implements IGetTabletsRequest + * @classdesc Represents an InitShardPrimaryResponse. + * @implements IInitShardPrimaryResponse * @constructor - * @param {vtctldata.IGetTabletsRequest=} [properties] Properties to set + * @param {vtctldata.IInitShardPrimaryResponse=} [properties] Properties to set */ - function GetTabletsRequest(properties) { - this.cells = []; - this.tablet_aliases = []; + function InitShardPrimaryResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -121116,151 +127015,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTabletsRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.GetTabletsRequest - * @instance - */ - GetTabletsRequest.prototype.keyspace = ""; - - /** - * GetTabletsRequest shard. - * @member {string} shard - * @memberof vtctldata.GetTabletsRequest - * @instance - */ - GetTabletsRequest.prototype.shard = ""; - - /** - * GetTabletsRequest cells. - * @member {Array.} cells - * @memberof vtctldata.GetTabletsRequest - * @instance - */ - GetTabletsRequest.prototype.cells = $util.emptyArray; - - /** - * GetTabletsRequest strict. - * @member {boolean} strict - * @memberof vtctldata.GetTabletsRequest - * @instance - */ - GetTabletsRequest.prototype.strict = false; - - /** - * GetTabletsRequest tablet_aliases. - * @member {Array.} tablet_aliases - * @memberof vtctldata.GetTabletsRequest - * @instance - */ - GetTabletsRequest.prototype.tablet_aliases = $util.emptyArray; - - /** - * GetTabletsRequest tablet_type. - * @member {topodata.TabletType} tablet_type - * @memberof vtctldata.GetTabletsRequest + * InitShardPrimaryResponse events. + * @member {Array.} events + * @memberof vtctldata.InitShardPrimaryResponse * @instance */ - GetTabletsRequest.prototype.tablet_type = 0; + InitShardPrimaryResponse.prototype.events = $util.emptyArray; /** - * Creates a new GetTabletsRequest instance using the specified properties. + * Creates a new InitShardPrimaryResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static - * @param {vtctldata.IGetTabletsRequest=} [properties] Properties to set - * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest instance + * @param {vtctldata.IInitShardPrimaryResponse=} [properties] Properties to set + * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse instance */ - GetTabletsRequest.create = function create(properties) { - return new GetTabletsRequest(properties); + InitShardPrimaryResponse.create = function create(properties) { + return new InitShardPrimaryResponse(properties); }; /** - * Encodes the specified GetTabletsRequest message. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. + * Encodes the specified InitShardPrimaryResponse message. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static - * @param {vtctldata.IGetTabletsRequest} message GetTabletsRequest message or plain object to encode + * @param {vtctldata.IInitShardPrimaryResponse} message InitShardPrimaryResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletsRequest.encode = function encode(message, writer) { + InitShardPrimaryResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); - if (message.strict != null && Object.hasOwnProperty.call(message, "strict")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.strict); - if (message.tablet_aliases != null && message.tablet_aliases.length) - for (let i = 0; i < message.tablet_aliases.length; ++i) - $root.topodata.TabletAlias.encode(message.tablet_aliases[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); - if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) - writer.uint32(/* id 6, wireType 0 =*/48).int32(message.tablet_type); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetTabletsRequest message, length delimited. Does not implicitly {@link vtctldata.GetTabletsRequest.verify|verify} messages. + * Encodes the specified InitShardPrimaryResponse message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static - * @param {vtctldata.IGetTabletsRequest} message GetTabletsRequest message or plain object to encode + * @param {vtctldata.IInitShardPrimaryResponse} message InitShardPrimaryResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletsRequest.encodeDelimited = function encodeDelimited(message, writer) { + InitShardPrimaryResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTabletsRequest message from the specified reader or buffer. + * Decodes an InitShardPrimaryResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest + * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletsRequest.decode = function decode(reader, length) { + InitShardPrimaryResponse.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.GetTabletsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.InitShardPrimaryResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); - break; - } - case 4: { - message.strict = reader.bool(); - break; - } - case 5: { - if (!(message.tablet_aliases && message.tablet_aliases.length)) - message.tablet_aliases = []; - message.tablet_aliases.push($root.topodata.TabletAlias.decode(reader, reader.uint32())); - break; - } - case 6: { - message.tablet_type = reader.int32(); + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.logutil.Event.decode(reader, reader.uint32())); break; } default: @@ -121272,259 +127098,160 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTabletsRequest message from the specified reader or buffer, length delimited. + * Decodes an InitShardPrimaryResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest + * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletsRequest.decodeDelimited = function decodeDelimited(reader) { + InitShardPrimaryResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a GetTabletsRequest message. - * @function verify - * @memberof vtctldata.GetTabletsRequest - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - GetTabletsRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } - if (message.strict != null && message.hasOwnProperty("strict")) - if (typeof message.strict !== "boolean") - return "strict: boolean expected"; - if (message.tablet_aliases != null && message.hasOwnProperty("tablet_aliases")) { - if (!Array.isArray(message.tablet_aliases)) - return "tablet_aliases: array expected"; - for (let i = 0; i < message.tablet_aliases.length; ++i) { - let error = $root.topodata.TabletAlias.verify(message.tablet_aliases[i]); + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an InitShardPrimaryResponse message. + * @function verify + * @memberof vtctldata.InitShardPrimaryResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + InitShardPrimaryResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.logutil.Event.verify(message.events[i]); if (error) - return "tablet_aliases." + error; + return "events." + error; } } - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - switch (message.tablet_type) { - default: - return "tablet_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } return null; }; /** - * Creates a GetTabletsRequest message from a plain object. Also converts values to their respective internal types. + * Creates an InitShardPrimaryResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetTabletsRequest} GetTabletsRequest + * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse */ - GetTabletsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTabletsRequest) + InitShardPrimaryResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.InitShardPrimaryResponse) return object; - let message = new $root.vtctldata.GetTabletsRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.GetTabletsRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } - if (object.strict != null) - message.strict = Boolean(object.strict); - if (object.tablet_aliases) { - if (!Array.isArray(object.tablet_aliases)) - throw TypeError(".vtctldata.GetTabletsRequest.tablet_aliases: array expected"); - message.tablet_aliases = []; - for (let i = 0; i < object.tablet_aliases.length; ++i) { - if (typeof object.tablet_aliases[i] !== "object") - throw TypeError(".vtctldata.GetTabletsRequest.tablet_aliases: object expected"); - message.tablet_aliases[i] = $root.topodata.TabletAlias.fromObject(object.tablet_aliases[i]); - } - } - switch (object.tablet_type) { - default: - if (typeof object.tablet_type === "number") { - message.tablet_type = object.tablet_type; - break; + let message = new $root.vtctldata.InitShardPrimaryResponse(); + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".vtctldata.InitShardPrimaryResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".vtctldata.InitShardPrimaryResponse.events: object expected"); + message.events[i] = $root.logutil.Event.fromObject(object.events[i]); } - break; - case "UNKNOWN": - case 0: - message.tablet_type = 0; - break; - case "PRIMARY": - case 1: - message.tablet_type = 1; - break; - case "MASTER": - case 1: - message.tablet_type = 1; - break; - case "REPLICA": - case 2: - message.tablet_type = 2; - break; - case "RDONLY": - case 3: - message.tablet_type = 3; - break; - case "BATCH": - case 3: - message.tablet_type = 3; - break; - case "SPARE": - case 4: - message.tablet_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_type = 5; - break; - case "BACKUP": - case 6: - message.tablet_type = 6; - break; - case "RESTORE": - case 7: - message.tablet_type = 7; - break; - case "DRAINED": - case 8: - message.tablet_type = 8; - break; } return message; }; /** - * Creates a plain object from a GetTabletsRequest message. Also converts values to other types if specified. + * Creates a plain object from an InitShardPrimaryResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static - * @param {vtctldata.GetTabletsRequest} message GetTabletsRequest + * @param {vtctldata.InitShardPrimaryResponse} message InitShardPrimaryResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTabletsRequest.toObject = function toObject(message, options) { + InitShardPrimaryResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.cells = []; - object.tablet_aliases = []; - } - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.strict = false; - object.tablet_type = options.enums === String ? "UNKNOWN" : 0; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } - if (message.strict != null && message.hasOwnProperty("strict")) - object.strict = message.strict; - if (message.tablet_aliases && message.tablet_aliases.length) { - object.tablet_aliases = []; - for (let j = 0; j < message.tablet_aliases.length; ++j) - object.tablet_aliases[j] = $root.topodata.TabletAlias.toObject(message.tablet_aliases[j], options); + if (options.arrays || options.defaults) + object.events = []; + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.logutil.Event.toObject(message.events[j], options); } - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; return object; }; /** - * Converts this GetTabletsRequest to JSON. + * Converts this InitShardPrimaryResponse to JSON. * @function toJSON - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @instance * @returns {Object.} JSON object */ - GetTabletsRequest.prototype.toJSON = function toJSON() { + InitShardPrimaryResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTabletsRequest + * Gets the default type url for InitShardPrimaryResponse * @function getTypeUrl - * @memberof vtctldata.GetTabletsRequest + * @memberof vtctldata.InitShardPrimaryResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTabletsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + InitShardPrimaryResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTabletsRequest"; + return typeUrlPrefix + "/vtctldata.InitShardPrimaryResponse"; }; - return GetTabletsRequest; + return InitShardPrimaryResponse; })(); - vtctldata.GetTabletsResponse = (function() { + vtctldata.MoveTablesCreateRequest = (function() { /** - * Properties of a GetTabletsResponse. + * Properties of a MoveTablesCreateRequest. * @memberof vtctldata - * @interface IGetTabletsResponse - * @property {Array.|null} [tablets] GetTabletsResponse tablets - */ - - /** - * Constructs a new GetTabletsResponse. + * @interface IMoveTablesCreateRequest + * @property {string|null} [workflow] MoveTablesCreateRequest workflow + * @property {string|null} [source_keyspace] MoveTablesCreateRequest source_keyspace + * @property {string|null} [target_keyspace] MoveTablesCreateRequest target_keyspace + * @property {Array.|null} [cells] MoveTablesCreateRequest cells + * @property {Array.|null} [tablet_types] MoveTablesCreateRequest tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] MoveTablesCreateRequest tablet_selection_preference + * @property {Array.|null} [source_shards] MoveTablesCreateRequest source_shards + * @property {boolean|null} [all_tables] MoveTablesCreateRequest all_tables + * @property {Array.|null} [include_tables] MoveTablesCreateRequest include_tables + * @property {Array.|null} [exclude_tables] MoveTablesCreateRequest exclude_tables + * @property {string|null} [external_cluster_name] MoveTablesCreateRequest external_cluster_name + * @property {string|null} [source_time_zone] MoveTablesCreateRequest source_time_zone + * @property {string|null} [on_ddl] MoveTablesCreateRequest on_ddl + * @property {boolean|null} [stop_after_copy] MoveTablesCreateRequest stop_after_copy + * @property {boolean|null} [drop_foreign_keys] MoveTablesCreateRequest drop_foreign_keys + * @property {boolean|null} [defer_secondary_keys] MoveTablesCreateRequest defer_secondary_keys + * @property {boolean|null} [auto_start] MoveTablesCreateRequest auto_start + */ + + /** + * Constructs a new MoveTablesCreateRequest. * @memberof vtctldata - * @classdesc Represents a GetTabletsResponse. - * @implements IGetTabletsResponse + * @classdesc Represents a MoveTablesCreateRequest. + * @implements IMoveTablesCreateRequest * @constructor - * @param {vtctldata.IGetTabletsResponse=} [properties] Properties to set + * @param {vtctldata.IMoveTablesCreateRequest=} [properties] Properties to set */ - function GetTabletsResponse(properties) { - this.tablets = []; + function MoveTablesCreateRequest(properties) { + this.cells = []; + this.tablet_types = []; + this.source_shards = []; + this.include_tables = []; + this.exclude_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -121532,78 +127259,322 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTabletsResponse tablets. - * @member {Array.} tablets - * @memberof vtctldata.GetTabletsResponse + * MoveTablesCreateRequest workflow. + * @member {string} workflow + * @memberof vtctldata.MoveTablesCreateRequest * @instance */ - GetTabletsResponse.prototype.tablets = $util.emptyArray; + MoveTablesCreateRequest.prototype.workflow = ""; /** - * Creates a new GetTabletsResponse instance using the specified properties. + * MoveTablesCreateRequest source_keyspace. + * @member {string} source_keyspace + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.source_keyspace = ""; + + /** + * MoveTablesCreateRequest target_keyspace. + * @member {string} target_keyspace + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.target_keyspace = ""; + + /** + * MoveTablesCreateRequest cells. + * @member {Array.} cells + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.cells = $util.emptyArray; + + /** + * MoveTablesCreateRequest tablet_types. + * @member {Array.} tablet_types + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.tablet_types = $util.emptyArray; + + /** + * MoveTablesCreateRequest tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.tablet_selection_preference = 0; + + /** + * MoveTablesCreateRequest source_shards. + * @member {Array.} source_shards + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.source_shards = $util.emptyArray; + + /** + * MoveTablesCreateRequest all_tables. + * @member {boolean} all_tables + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.all_tables = false; + + /** + * MoveTablesCreateRequest include_tables. + * @member {Array.} include_tables + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.include_tables = $util.emptyArray; + + /** + * MoveTablesCreateRequest exclude_tables. + * @member {Array.} exclude_tables + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.exclude_tables = $util.emptyArray; + + /** + * MoveTablesCreateRequest external_cluster_name. + * @member {string} external_cluster_name + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.external_cluster_name = ""; + + /** + * MoveTablesCreateRequest source_time_zone. + * @member {string} source_time_zone + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.source_time_zone = ""; + + /** + * MoveTablesCreateRequest on_ddl. + * @member {string} on_ddl + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.on_ddl = ""; + + /** + * MoveTablesCreateRequest stop_after_copy. + * @member {boolean} stop_after_copy + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.stop_after_copy = false; + + /** + * MoveTablesCreateRequest drop_foreign_keys. + * @member {boolean} drop_foreign_keys + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.drop_foreign_keys = false; + + /** + * MoveTablesCreateRequest defer_secondary_keys. + * @member {boolean} defer_secondary_keys + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.defer_secondary_keys = false; + + /** + * MoveTablesCreateRequest auto_start. + * @member {boolean} auto_start + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.auto_start = false; + + /** + * Creates a new MoveTablesCreateRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static - * @param {vtctldata.IGetTabletsResponse=} [properties] Properties to set - * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse instance + * @param {vtctldata.IMoveTablesCreateRequest=} [properties] Properties to set + * @returns {vtctldata.MoveTablesCreateRequest} MoveTablesCreateRequest instance */ - GetTabletsResponse.create = function create(properties) { - return new GetTabletsResponse(properties); + MoveTablesCreateRequest.create = function create(properties) { + return new MoveTablesCreateRequest(properties); }; /** - * Encodes the specified GetTabletsResponse message. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. + * Encodes the specified MoveTablesCreateRequest message. Does not implicitly {@link vtctldata.MoveTablesCreateRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static - * @param {vtctldata.IGetTabletsResponse} message GetTabletsResponse message or plain object to encode + * @param {vtctldata.IMoveTablesCreateRequest} message MoveTablesCreateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletsResponse.encode = function encode(message, writer) { + MoveTablesCreateRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablets != null && message.tablets.length) - for (let i = 0; i < message.tablets.length; ++i) - $root.topodata.Tablet.encode(message.tablets[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_keyspace); + if (message.target_keyspace != null && Object.hasOwnProperty.call(message, "target_keyspace")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_keyspace); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.cells[i]); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 5, wireType 2 =*/42).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.tablet_selection_preference); + if (message.source_shards != null && message.source_shards.length) + for (let i = 0; i < message.source_shards.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.source_shards[i]); + if (message.all_tables != null && Object.hasOwnProperty.call(message, "all_tables")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.all_tables); + if (message.include_tables != null && message.include_tables.length) + for (let i = 0; i < message.include_tables.length; ++i) + writer.uint32(/* id 9, wireType 2 =*/74).string(message.include_tables[i]); + if (message.exclude_tables != null && message.exclude_tables.length) + for (let i = 0; i < message.exclude_tables.length; ++i) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.exclude_tables[i]); + if (message.external_cluster_name != null && Object.hasOwnProperty.call(message, "external_cluster_name")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.external_cluster_name); + if (message.source_time_zone != null && Object.hasOwnProperty.call(message, "source_time_zone")) + writer.uint32(/* id 12, wireType 2 =*/98).string(message.source_time_zone); + if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) + writer.uint32(/* id 13, wireType 2 =*/106).string(message.on_ddl); + if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) + writer.uint32(/* id 14, wireType 0 =*/112).bool(message.stop_after_copy); + if (message.drop_foreign_keys != null && Object.hasOwnProperty.call(message, "drop_foreign_keys")) + writer.uint32(/* id 15, wireType 0 =*/120).bool(message.drop_foreign_keys); + if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) + writer.uint32(/* id 16, wireType 0 =*/128).bool(message.defer_secondary_keys); + if (message.auto_start != null && Object.hasOwnProperty.call(message, "auto_start")) + writer.uint32(/* id 17, wireType 0 =*/136).bool(message.auto_start); return writer; }; /** - * Encodes the specified GetTabletsResponse message, length delimited. Does not implicitly {@link vtctldata.GetTabletsResponse.verify|verify} messages. + * Encodes the specified MoveTablesCreateRequest message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static - * @param {vtctldata.IGetTabletsResponse} message GetTabletsResponse message or plain object to encode + * @param {vtctldata.IMoveTablesCreateRequest} message MoveTablesCreateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTabletsResponse.encodeDelimited = function encodeDelimited(message, writer) { + MoveTablesCreateRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTabletsResponse message from the specified reader or buffer. + * Decodes a MoveTablesCreateRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse + * @returns {vtctldata.MoveTablesCreateRequest} MoveTablesCreateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletsResponse.decode = function decode(reader, length) { + MoveTablesCreateRequest.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.GetTabletsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.MoveTablesCreateRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.tablets && message.tablets.length)) - message.tablets = []; - message.tablets.push($root.topodata.Tablet.decode(reader, reader.uint32())); + message.workflow = reader.string(); + break; + } + case 2: { + message.source_keyspace = reader.string(); + break; + } + case 3: { + message.target_keyspace = reader.string(); + break; + } + case 4: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 5: { + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); + break; + } + case 6: { + message.tablet_selection_preference = reader.int32(); + break; + } + case 7: { + if (!(message.source_shards && message.source_shards.length)) + message.source_shards = []; + message.source_shards.push(reader.string()); + break; + } + case 8: { + message.all_tables = reader.bool(); + break; + } + case 9: { + if (!(message.include_tables && message.include_tables.length)) + message.include_tables = []; + message.include_tables.push(reader.string()); + break; + } + case 10: { + if (!(message.exclude_tables && message.exclude_tables.length)) + message.exclude_tables = []; + message.exclude_tables.push(reader.string()); + break; + } + case 11: { + message.external_cluster_name = reader.string(); + break; + } + case 12: { + message.source_time_zone = reader.string(); + break; + } + case 13: { + message.on_ddl = reader.string(); + break; + } + case 14: { + message.stop_after_copy = reader.bool(); + break; + } + case 15: { + message.drop_foreign_keys = reader.bool(); + break; + } + case 16: { + message.defer_secondary_keys = reader.bool(); + break; + } + case 17: { + message.auto_start = reader.bool(); break; } default: @@ -121615,139 +127586,403 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTabletsResponse message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCreateRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse + * @returns {vtctldata.MoveTablesCreateRequest} MoveTablesCreateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTabletsResponse.decodeDelimited = function decodeDelimited(reader) { + MoveTablesCreateRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetTabletsResponse message. + * Verifies a MoveTablesCreateRequest message. * @function verify - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetTabletsResponse.verify = function verify(message) { + MoveTablesCreateRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablets != null && message.hasOwnProperty("tablets")) { - if (!Array.isArray(message.tablets)) - return "tablets: array expected"; - for (let i = 0; i < message.tablets.length; ++i) { - let error = $root.topodata.Tablet.verify(message.tablets[i]); - if (error) - return "tablets." + error; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + if (!$util.isString(message.source_keyspace)) + return "source_keyspace: string expected"; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + if (!$util.isString(message.target_keyspace)) + return "target_keyspace: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; } + if (message.source_shards != null && message.hasOwnProperty("source_shards")) { + if (!Array.isArray(message.source_shards)) + return "source_shards: array expected"; + for (let i = 0; i < message.source_shards.length; ++i) + if (!$util.isString(message.source_shards[i])) + return "source_shards: string[] expected"; + } + if (message.all_tables != null && message.hasOwnProperty("all_tables")) + if (typeof message.all_tables !== "boolean") + return "all_tables: boolean expected"; + if (message.include_tables != null && message.hasOwnProperty("include_tables")) { + if (!Array.isArray(message.include_tables)) + return "include_tables: array expected"; + for (let i = 0; i < message.include_tables.length; ++i) + if (!$util.isString(message.include_tables[i])) + return "include_tables: string[] expected"; + } + if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { + if (!Array.isArray(message.exclude_tables)) + return "exclude_tables: array expected"; + for (let i = 0; i < message.exclude_tables.length; ++i) + if (!$util.isString(message.exclude_tables[i])) + return "exclude_tables: string[] expected"; + } + if (message.external_cluster_name != null && message.hasOwnProperty("external_cluster_name")) + if (!$util.isString(message.external_cluster_name)) + return "external_cluster_name: string expected"; + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + if (!$util.isString(message.source_time_zone)) + return "source_time_zone: string expected"; + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + if (!$util.isString(message.on_ddl)) + return "on_ddl: string expected"; + 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.drop_foreign_keys != null && message.hasOwnProperty("drop_foreign_keys")) + if (typeof message.drop_foreign_keys !== "boolean") + return "drop_foreign_keys: boolean expected"; + 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.auto_start != null && message.hasOwnProperty("auto_start")) + if (typeof message.auto_start !== "boolean") + return "auto_start: boolean expected"; + return null; + }; + + /** + * Creates a MoveTablesCreateRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.MoveTablesCreateRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.MoveTablesCreateRequest} MoveTablesCreateRequest + */ + MoveTablesCreateRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MoveTablesCreateRequest) + return object; + let message = new $root.vtctldata.MoveTablesCreateRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.source_keyspace != null) + message.source_keyspace = String(object.source_keyspace); + if (object.target_keyspace != null) + message.target_keyspace = String(object.target_keyspace); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.MoveTablesCreateRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".vtctldata.MoveTablesCreateRequest.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } } - return null; - }; - - /** - * Creates a GetTabletsResponse message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.GetTabletsResponse - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.GetTabletsResponse} GetTabletsResponse - */ - GetTabletsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTabletsResponse) - return object; - let message = new $root.vtctldata.GetTabletsResponse(); - if (object.tablets) { - if (!Array.isArray(object.tablets)) - throw TypeError(".vtctldata.GetTabletsResponse.tablets: array expected"); - message.tablets = []; - for (let i = 0; i < object.tablets.length; ++i) { - if (typeof object.tablets[i] !== "object") - throw TypeError(".vtctldata.GetTabletsResponse.tablets: object expected"); - message.tablets[i] = $root.topodata.Tablet.fromObject(object.tablets[i]); + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; + break; } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + if (object.source_shards) { + if (!Array.isArray(object.source_shards)) + throw TypeError(".vtctldata.MoveTablesCreateRequest.source_shards: array expected"); + message.source_shards = []; + for (let i = 0; i < object.source_shards.length; ++i) + message.source_shards[i] = String(object.source_shards[i]); + } + if (object.all_tables != null) + message.all_tables = Boolean(object.all_tables); + if (object.include_tables) { + if (!Array.isArray(object.include_tables)) + throw TypeError(".vtctldata.MoveTablesCreateRequest.include_tables: array expected"); + message.include_tables = []; + for (let i = 0; i < object.include_tables.length; ++i) + message.include_tables[i] = String(object.include_tables[i]); + } + if (object.exclude_tables) { + if (!Array.isArray(object.exclude_tables)) + throw TypeError(".vtctldata.MoveTablesCreateRequest.exclude_tables: array expected"); + message.exclude_tables = []; + for (let i = 0; i < object.exclude_tables.length; ++i) + message.exclude_tables[i] = String(object.exclude_tables[i]); } + if (object.external_cluster_name != null) + message.external_cluster_name = String(object.external_cluster_name); + if (object.source_time_zone != null) + message.source_time_zone = String(object.source_time_zone); + if (object.on_ddl != null) + message.on_ddl = String(object.on_ddl); + if (object.stop_after_copy != null) + message.stop_after_copy = Boolean(object.stop_after_copy); + if (object.drop_foreign_keys != null) + message.drop_foreign_keys = Boolean(object.drop_foreign_keys); + if (object.defer_secondary_keys != null) + message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + if (object.auto_start != null) + message.auto_start = Boolean(object.auto_start); return message; }; /** - * Creates a plain object from a GetTabletsResponse message. Also converts values to other types if specified. + * Creates a plain object from a MoveTablesCreateRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static - * @param {vtctldata.GetTabletsResponse} message GetTabletsResponse + * @param {vtctldata.MoveTablesCreateRequest} message MoveTablesCreateRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTabletsResponse.toObject = function toObject(message, options) { + MoveTablesCreateRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tablets = []; - if (message.tablets && message.tablets.length) { - object.tablets = []; - for (let j = 0; j < message.tablets.length; ++j) - object.tablets[j] = $root.topodata.Tablet.toObject(message.tablets[j], options); + if (options.arrays || options.defaults) { + object.cells = []; + object.tablet_types = []; + object.source_shards = []; + object.include_tables = []; + object.exclude_tables = []; + } + if (options.defaults) { + object.workflow = ""; + object.source_keyspace = ""; + object.target_keyspace = ""; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.all_tables = false; + object.external_cluster_name = ""; + object.source_time_zone = ""; + object.on_ddl = ""; + object.stop_after_copy = false; + object.drop_foreign_keys = false; + object.defer_secondary_keys = false; + object.auto_start = false; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + object.source_keyspace = message.source_keyspace; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + object.target_keyspace = message.target_keyspace; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.source_shards && message.source_shards.length) { + object.source_shards = []; + for (let j = 0; j < message.source_shards.length; ++j) + object.source_shards[j] = message.source_shards[j]; + } + if (message.all_tables != null && message.hasOwnProperty("all_tables")) + object.all_tables = message.all_tables; + if (message.include_tables && message.include_tables.length) { + object.include_tables = []; + for (let j = 0; j < message.include_tables.length; ++j) + object.include_tables[j] = message.include_tables[j]; + } + if (message.exclude_tables && message.exclude_tables.length) { + object.exclude_tables = []; + for (let j = 0; j < message.exclude_tables.length; ++j) + object.exclude_tables[j] = message.exclude_tables[j]; } + if (message.external_cluster_name != null && message.hasOwnProperty("external_cluster_name")) + object.external_cluster_name = message.external_cluster_name; + if (message.source_time_zone != null && message.hasOwnProperty("source_time_zone")) + object.source_time_zone = message.source_time_zone; + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + object.on_ddl = message.on_ddl; + if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) + object.stop_after_copy = message.stop_after_copy; + if (message.drop_foreign_keys != null && message.hasOwnProperty("drop_foreign_keys")) + object.drop_foreign_keys = message.drop_foreign_keys; + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + object.defer_secondary_keys = message.defer_secondary_keys; + if (message.auto_start != null && message.hasOwnProperty("auto_start")) + object.auto_start = message.auto_start; return object; }; /** - * Converts this GetTabletsResponse to JSON. + * Converts this MoveTablesCreateRequest to JSON. * @function toJSON - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @instance * @returns {Object.} JSON object */ - GetTabletsResponse.prototype.toJSON = function toJSON() { + MoveTablesCreateRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTabletsResponse + * Gets the default type url for MoveTablesCreateRequest * @function getTypeUrl - * @memberof vtctldata.GetTabletsResponse + * @memberof vtctldata.MoveTablesCreateRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTabletsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MoveTablesCreateRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTabletsResponse"; + return typeUrlPrefix + "/vtctldata.MoveTablesCreateRequest"; }; - return GetTabletsResponse; + return MoveTablesCreateRequest; })(); - vtctldata.GetTopologyPathRequest = (function() { + vtctldata.MoveTablesCreateResponse = (function() { /** - * Properties of a GetTopologyPathRequest. + * Properties of a MoveTablesCreateResponse. * @memberof vtctldata - * @interface IGetTopologyPathRequest - * @property {string|null} [path] GetTopologyPathRequest path + * @interface IMoveTablesCreateResponse + * @property {string|null} [summary] MoveTablesCreateResponse summary + * @property {Array.|null} [details] MoveTablesCreateResponse details */ /** - * Constructs a new GetTopologyPathRequest. + * Constructs a new MoveTablesCreateResponse. * @memberof vtctldata - * @classdesc Represents a GetTopologyPathRequest. - * @implements IGetTopologyPathRequest + * @classdesc Represents a MoveTablesCreateResponse. + * @implements IMoveTablesCreateResponse * @constructor - * @param {vtctldata.IGetTopologyPathRequest=} [properties] Properties to set + * @param {vtctldata.IMoveTablesCreateResponse=} [properties] Properties to set */ - function GetTopologyPathRequest(properties) { + function MoveTablesCreateResponse(properties) { + this.details = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -121755,75 +127990,92 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTopologyPathRequest path. - * @member {string} path - * @memberof vtctldata.GetTopologyPathRequest + * MoveTablesCreateResponse summary. + * @member {string} summary + * @memberof vtctldata.MoveTablesCreateResponse * @instance */ - GetTopologyPathRequest.prototype.path = ""; + MoveTablesCreateResponse.prototype.summary = ""; /** - * Creates a new GetTopologyPathRequest instance using the specified properties. + * MoveTablesCreateResponse details. + * @member {Array.} details + * @memberof vtctldata.MoveTablesCreateResponse + * @instance + */ + MoveTablesCreateResponse.prototype.details = $util.emptyArray; + + /** + * Creates a new MoveTablesCreateResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static - * @param {vtctldata.IGetTopologyPathRequest=} [properties] Properties to set - * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest instance + * @param {vtctldata.IMoveTablesCreateResponse=} [properties] Properties to set + * @returns {vtctldata.MoveTablesCreateResponse} MoveTablesCreateResponse instance */ - GetTopologyPathRequest.create = function create(properties) { - return new GetTopologyPathRequest(properties); + MoveTablesCreateResponse.create = function create(properties) { + return new MoveTablesCreateResponse(properties); }; /** - * Encodes the specified GetTopologyPathRequest message. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. + * Encodes the specified MoveTablesCreateResponse message. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static - * @param {vtctldata.IGetTopologyPathRequest} message GetTopologyPathRequest message or plain object to encode + * @param {vtctldata.IMoveTablesCreateResponse} message MoveTablesCreateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTopologyPathRequest.encode = function encode(message, writer) { + MoveTablesCreateResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.path != null && Object.hasOwnProperty.call(message, "path")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.path); + if (message.summary != null && Object.hasOwnProperty.call(message, "summary")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.summary); + if (message.details != null && message.details.length) + for (let i = 0; i < message.details.length; ++i) + $root.vtctldata.MoveTablesCreateResponse.TabletInfo.encode(message.details[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetTopologyPathRequest message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathRequest.verify|verify} messages. + * Encodes the specified MoveTablesCreateResponse message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static - * @param {vtctldata.IGetTopologyPathRequest} message GetTopologyPathRequest message or plain object to encode + * @param {vtctldata.IMoveTablesCreateResponse} message MoveTablesCreateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTopologyPathRequest.encodeDelimited = function encodeDelimited(message, writer) { + MoveTablesCreateResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTopologyPathRequest message from the specified reader or buffer. + * Decodes a MoveTablesCreateResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest + * @returns {vtctldata.MoveTablesCreateResponse} MoveTablesCreateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTopologyPathRequest.decode = function decode(reader, length) { + MoveTablesCreateResponse.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.GetTopologyPathRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.MoveTablesCreateResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.path = reader.string(); + message.summary = reader.string(); + break; + } + case 2: { + if (!(message.details && message.details.length)) + message.details = []; + message.details.push($root.vtctldata.MoveTablesCreateResponse.TabletInfo.decode(reader, reader.uint32())); break; } default: @@ -121835,122 +128087,385 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTopologyPathRequest message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCreateResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest + * @returns {vtctldata.MoveTablesCreateResponse} MoveTablesCreateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTopologyPathRequest.decodeDelimited = function decodeDelimited(reader) { + MoveTablesCreateResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetTopologyPathRequest message. + * Verifies a MoveTablesCreateResponse message. * @function verify - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetTopologyPathRequest.verify = function verify(message) { + MoveTablesCreateResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.path != null && message.hasOwnProperty("path")) - if (!$util.isString(message.path)) - return "path: string expected"; + if (message.summary != null && message.hasOwnProperty("summary")) + if (!$util.isString(message.summary)) + return "summary: string expected"; + if (message.details != null && message.hasOwnProperty("details")) { + if (!Array.isArray(message.details)) + return "details: array expected"; + for (let i = 0; i < message.details.length; ++i) { + let error = $root.vtctldata.MoveTablesCreateResponse.TabletInfo.verify(message.details[i]); + if (error) + return "details." + error; + } + } return null; }; /** - * Creates a GetTopologyPathRequest message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCreateResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetTopologyPathRequest} GetTopologyPathRequest + * @returns {vtctldata.MoveTablesCreateResponse} MoveTablesCreateResponse */ - GetTopologyPathRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTopologyPathRequest) + MoveTablesCreateResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MoveTablesCreateResponse) return object; - let message = new $root.vtctldata.GetTopologyPathRequest(); - if (object.path != null) - message.path = String(object.path); + let message = new $root.vtctldata.MoveTablesCreateResponse(); + if (object.summary != null) + message.summary = String(object.summary); + if (object.details) { + if (!Array.isArray(object.details)) + throw TypeError(".vtctldata.MoveTablesCreateResponse.details: array expected"); + message.details = []; + for (let i = 0; i < object.details.length; ++i) { + if (typeof object.details[i] !== "object") + throw TypeError(".vtctldata.MoveTablesCreateResponse.details: object expected"); + message.details[i] = $root.vtctldata.MoveTablesCreateResponse.TabletInfo.fromObject(object.details[i]); + } + } return message; }; /** - * Creates a plain object from a GetTopologyPathRequest message. Also converts values to other types if specified. + * Creates a plain object from a MoveTablesCreateResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static - * @param {vtctldata.GetTopologyPathRequest} message GetTopologyPathRequest + * @param {vtctldata.MoveTablesCreateResponse} message MoveTablesCreateResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTopologyPathRequest.toObject = function toObject(message, options) { + MoveTablesCreateResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.details = []; if (options.defaults) - object.path = ""; - if (message.path != null && message.hasOwnProperty("path")) - object.path = message.path; + object.summary = ""; + if (message.summary != null && message.hasOwnProperty("summary")) + object.summary = message.summary; + if (message.details && message.details.length) { + object.details = []; + for (let j = 0; j < message.details.length; ++j) + object.details[j] = $root.vtctldata.MoveTablesCreateResponse.TabletInfo.toObject(message.details[j], options); + } return object; }; /** - * Converts this GetTopologyPathRequest to JSON. + * Converts this MoveTablesCreateResponse to JSON. * @function toJSON - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @instance * @returns {Object.} JSON object */ - GetTopologyPathRequest.prototype.toJSON = function toJSON() { + MoveTablesCreateResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTopologyPathRequest + * Gets the default type url for MoveTablesCreateResponse * @function getTypeUrl - * @memberof vtctldata.GetTopologyPathRequest + * @memberof vtctldata.MoveTablesCreateResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTopologyPathRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MoveTablesCreateResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTopologyPathRequest"; + return typeUrlPrefix + "/vtctldata.MoveTablesCreateResponse"; }; - return GetTopologyPathRequest; + MoveTablesCreateResponse.TabletInfo = (function() { + + /** + * Properties of a TabletInfo. + * @memberof vtctldata.MoveTablesCreateResponse + * @interface ITabletInfo + * @property {topodata.ITabletAlias|null} [tablet] TabletInfo tablet + * @property {boolean|null} [created] TabletInfo created + */ + + /** + * Constructs a new TabletInfo. + * @memberof vtctldata.MoveTablesCreateResponse + * @classdesc Represents a TabletInfo. + * @implements ITabletInfo + * @constructor + * @param {vtctldata.MoveTablesCreateResponse.ITabletInfo=} [properties] Properties to set + */ + function TabletInfo(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]]; + } + + /** + * TabletInfo tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @instance + */ + TabletInfo.prototype.tablet = null; + + /** + * TabletInfo created. + * @member {boolean} created + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @instance + */ + TabletInfo.prototype.created = false; + + /** + * Creates a new TabletInfo instance using the specified properties. + * @function create + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {vtctldata.MoveTablesCreateResponse.ITabletInfo=} [properties] Properties to set + * @returns {vtctldata.MoveTablesCreateResponse.TabletInfo} TabletInfo instance + */ + TabletInfo.create = function create(properties) { + return new TabletInfo(properties); + }; + + /** + * Encodes the specified TabletInfo message. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.TabletInfo.verify|verify} messages. + * @function encode + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {vtctldata.MoveTablesCreateResponse.ITabletInfo} message TabletInfo message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TabletInfo.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.created != null && Object.hasOwnProperty.call(message, "created")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.created); + return writer; + }; + + /** + * Encodes the specified TabletInfo message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCreateResponse.TabletInfo.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {vtctldata.MoveTablesCreateResponse.ITabletInfo} message TabletInfo message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TabletInfo.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a TabletInfo message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.MoveTablesCreateResponse.TabletInfo} TabletInfo + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TabletInfo.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.MoveTablesCreateResponse.TabletInfo(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.created = reader.bool(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a TabletInfo message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.MoveTablesCreateResponse.TabletInfo} TabletInfo + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TabletInfo.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a TabletInfo message. + * @function verify + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + TabletInfo.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } + if (message.created != null && message.hasOwnProperty("created")) + if (typeof message.created !== "boolean") + return "created: boolean expected"; + return null; + }; + + /** + * Creates a TabletInfo message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.MoveTablesCreateResponse.TabletInfo} TabletInfo + */ + TabletInfo.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MoveTablesCreateResponse.TabletInfo) + return object; + let message = new $root.vtctldata.MoveTablesCreateResponse.TabletInfo(); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.MoveTablesCreateResponse.TabletInfo.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } + if (object.created != null) + message.created = Boolean(object.created); + return message; + }; + + /** + * Creates a plain object from a TabletInfo message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {vtctldata.MoveTablesCreateResponse.TabletInfo} message TabletInfo + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + TabletInfo.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet = null; + object.created = false; + } + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (message.created != null && message.hasOwnProperty("created")) + object.created = message.created; + return object; + }; + + /** + * Converts this TabletInfo to JSON. + * @function toJSON + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @instance + * @returns {Object.} JSON object + */ + TabletInfo.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for TabletInfo + * @function getTypeUrl + * @memberof vtctldata.MoveTablesCreateResponse.TabletInfo + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + TabletInfo.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.MoveTablesCreateResponse.TabletInfo"; + }; + + return TabletInfo; + })(); + + return MoveTablesCreateResponse; })(); - vtctldata.GetTopologyPathResponse = (function() { + vtctldata.MoveTablesCompleteRequest = (function() { /** - * Properties of a GetTopologyPathResponse. + * Properties of a MoveTablesCompleteRequest. * @memberof vtctldata - * @interface IGetTopologyPathResponse - * @property {vtctldata.ITopologyCell|null} [cell] GetTopologyPathResponse cell + * @interface IMoveTablesCompleteRequest + * @property {string|null} [workflow] MoveTablesCompleteRequest workflow + * @property {string|null} [target_keyspace] MoveTablesCompleteRequest target_keyspace + * @property {boolean|null} [keep_data] MoveTablesCompleteRequest keep_data + * @property {boolean|null} [keep_routing_rules] MoveTablesCompleteRequest keep_routing_rules + * @property {boolean|null} [rename_tables] MoveTablesCompleteRequest rename_tables + * @property {boolean|null} [dry_run] MoveTablesCompleteRequest dry_run */ /** - * Constructs a new GetTopologyPathResponse. + * Constructs a new MoveTablesCompleteRequest. * @memberof vtctldata - * @classdesc Represents a GetTopologyPathResponse. - * @implements IGetTopologyPathResponse + * @classdesc Represents a MoveTablesCompleteRequest. + * @implements IMoveTablesCompleteRequest * @constructor - * @param {vtctldata.IGetTopologyPathResponse=} [properties] Properties to set + * @param {vtctldata.IMoveTablesCompleteRequest=} [properties] Properties to set */ - function GetTopologyPathResponse(properties) { + function MoveTablesCompleteRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -121958,75 +128473,145 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetTopologyPathResponse cell. - * @member {vtctldata.ITopologyCell|null|undefined} cell - * @memberof vtctldata.GetTopologyPathResponse + * MoveTablesCompleteRequest workflow. + * @member {string} workflow + * @memberof vtctldata.MoveTablesCompleteRequest * @instance */ - GetTopologyPathResponse.prototype.cell = null; + MoveTablesCompleteRequest.prototype.workflow = ""; /** - * Creates a new GetTopologyPathResponse instance using the specified properties. + * MoveTablesCompleteRequest target_keyspace. + * @member {string} target_keyspace + * @memberof vtctldata.MoveTablesCompleteRequest + * @instance + */ + MoveTablesCompleteRequest.prototype.target_keyspace = ""; + + /** + * MoveTablesCompleteRequest keep_data. + * @member {boolean} keep_data + * @memberof vtctldata.MoveTablesCompleteRequest + * @instance + */ + MoveTablesCompleteRequest.prototype.keep_data = false; + + /** + * MoveTablesCompleteRequest keep_routing_rules. + * @member {boolean} keep_routing_rules + * @memberof vtctldata.MoveTablesCompleteRequest + * @instance + */ + MoveTablesCompleteRequest.prototype.keep_routing_rules = false; + + /** + * MoveTablesCompleteRequest rename_tables. + * @member {boolean} rename_tables + * @memberof vtctldata.MoveTablesCompleteRequest + * @instance + */ + MoveTablesCompleteRequest.prototype.rename_tables = false; + + /** + * MoveTablesCompleteRequest dry_run. + * @member {boolean} dry_run + * @memberof vtctldata.MoveTablesCompleteRequest + * @instance + */ + MoveTablesCompleteRequest.prototype.dry_run = false; + + /** + * Creates a new MoveTablesCompleteRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static - * @param {vtctldata.IGetTopologyPathResponse=} [properties] Properties to set - * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse instance + * @param {vtctldata.IMoveTablesCompleteRequest=} [properties] Properties to set + * @returns {vtctldata.MoveTablesCompleteRequest} MoveTablesCompleteRequest instance */ - GetTopologyPathResponse.create = function create(properties) { - return new GetTopologyPathResponse(properties); + MoveTablesCompleteRequest.create = function create(properties) { + return new MoveTablesCompleteRequest(properties); }; /** - * Encodes the specified GetTopologyPathResponse message. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. + * Encodes the specified MoveTablesCompleteRequest message. Does not implicitly {@link vtctldata.MoveTablesCompleteRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static - * @param {vtctldata.IGetTopologyPathResponse} message GetTopologyPathResponse message or plain object to encode + * @param {vtctldata.IMoveTablesCompleteRequest} message MoveTablesCompleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTopologyPathResponse.encode = function encode(message, writer) { + MoveTablesCompleteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - $root.vtctldata.TopologyCell.encode(message.cell, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.target_keyspace != null && Object.hasOwnProperty.call(message, "target_keyspace")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_keyspace); + if (message.keep_data != null && Object.hasOwnProperty.call(message, "keep_data")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.keep_data); + if (message.keep_routing_rules != null && Object.hasOwnProperty.call(message, "keep_routing_rules")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.keep_routing_rules); + if (message.rename_tables != null && Object.hasOwnProperty.call(message, "rename_tables")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.rename_tables); + if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) + writer.uint32(/* id 7, wireType 0 =*/56).bool(message.dry_run); return writer; }; /** - * Encodes the specified GetTopologyPathResponse message, length delimited. Does not implicitly {@link vtctldata.GetTopologyPathResponse.verify|verify} messages. + * Encodes the specified MoveTablesCompleteRequest message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCompleteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static - * @param {vtctldata.IGetTopologyPathResponse} message GetTopologyPathResponse message or plain object to encode + * @param {vtctldata.IMoveTablesCompleteRequest} message MoveTablesCompleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetTopologyPathResponse.encodeDelimited = function encodeDelimited(message, writer) { + MoveTablesCompleteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetTopologyPathResponse message from the specified reader or buffer. + * Decodes a MoveTablesCompleteRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse + * @returns {vtctldata.MoveTablesCompleteRequest} MoveTablesCompleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTopologyPathResponse.decode = function decode(reader, length) { + MoveTablesCompleteRequest.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.GetTopologyPathResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.MoveTablesCompleteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.cell = $root.vtctldata.TopologyCell.decode(reader, reader.uint32()); + message.workflow = reader.string(); + break; + } + case 3: { + message.target_keyspace = reader.string(); + break; + } + case 4: { + message.keep_data = reader.bool(); + break; + } + case 5: { + message.keep_routing_rules = reader.bool(); + break; + } + case 6: { + message.rename_tables = reader.bool(); + break; + } + case 7: { + message.dry_run = reader.bool(); break; } default: @@ -122038,131 +128623,165 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetTopologyPathResponse message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCompleteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse + * @returns {vtctldata.MoveTablesCompleteRequest} MoveTablesCompleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetTopologyPathResponse.decodeDelimited = function decodeDelimited(reader) { + MoveTablesCompleteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetTopologyPathResponse message. + * Verifies a MoveTablesCompleteRequest message. * @function verify - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetTopologyPathResponse.verify = function verify(message) { + MoveTablesCompleteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cell != null && message.hasOwnProperty("cell")) { - let error = $root.vtctldata.TopologyCell.verify(message.cell); - if (error) - return "cell." + error; - } + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + if (!$util.isString(message.target_keyspace)) + return "target_keyspace: string expected"; + if (message.keep_data != null && message.hasOwnProperty("keep_data")) + if (typeof message.keep_data !== "boolean") + return "keep_data: boolean expected"; + if (message.keep_routing_rules != null && message.hasOwnProperty("keep_routing_rules")) + if (typeof message.keep_routing_rules !== "boolean") + return "keep_routing_rules: boolean expected"; + if (message.rename_tables != null && message.hasOwnProperty("rename_tables")) + if (typeof message.rename_tables !== "boolean") + return "rename_tables: boolean expected"; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + if (typeof message.dry_run !== "boolean") + return "dry_run: boolean expected"; return null; }; /** - * Creates a GetTopologyPathResponse message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCompleteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetTopologyPathResponse} GetTopologyPathResponse + * @returns {vtctldata.MoveTablesCompleteRequest} MoveTablesCompleteRequest */ - GetTopologyPathResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetTopologyPathResponse) + MoveTablesCompleteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MoveTablesCompleteRequest) return object; - let message = new $root.vtctldata.GetTopologyPathResponse(); - if (object.cell != null) { - if (typeof object.cell !== "object") - throw TypeError(".vtctldata.GetTopologyPathResponse.cell: object expected"); - message.cell = $root.vtctldata.TopologyCell.fromObject(object.cell); - } + let message = new $root.vtctldata.MoveTablesCompleteRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.target_keyspace != null) + message.target_keyspace = String(object.target_keyspace); + if (object.keep_data != null) + message.keep_data = Boolean(object.keep_data); + if (object.keep_routing_rules != null) + message.keep_routing_rules = Boolean(object.keep_routing_rules); + if (object.rename_tables != null) + message.rename_tables = Boolean(object.rename_tables); + if (object.dry_run != null) + message.dry_run = Boolean(object.dry_run); return message; }; /** - * Creates a plain object from a GetTopologyPathResponse message. Also converts values to other types if specified. + * Creates a plain object from a MoveTablesCompleteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static - * @param {vtctldata.GetTopologyPathResponse} message GetTopologyPathResponse + * @param {vtctldata.MoveTablesCompleteRequest} message MoveTablesCompleteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetTopologyPathResponse.toObject = function toObject(message, options) { + MoveTablesCompleteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.cell = null; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = $root.vtctldata.TopologyCell.toObject(message.cell, options); + if (options.defaults) { + object.workflow = ""; + object.target_keyspace = ""; + object.keep_data = false; + object.keep_routing_rules = false; + object.rename_tables = false; + object.dry_run = false; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.target_keyspace != null && message.hasOwnProperty("target_keyspace")) + object.target_keyspace = message.target_keyspace; + if (message.keep_data != null && message.hasOwnProperty("keep_data")) + object.keep_data = message.keep_data; + if (message.keep_routing_rules != null && message.hasOwnProperty("keep_routing_rules")) + object.keep_routing_rules = message.keep_routing_rules; + if (message.rename_tables != null && message.hasOwnProperty("rename_tables")) + object.rename_tables = message.rename_tables; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + object.dry_run = message.dry_run; return object; }; /** - * Converts this GetTopologyPathResponse to JSON. + * Converts this MoveTablesCompleteRequest to JSON. * @function toJSON - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @instance * @returns {Object.} JSON object */ - GetTopologyPathResponse.prototype.toJSON = function toJSON() { + MoveTablesCompleteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetTopologyPathResponse + * Gets the default type url for MoveTablesCompleteRequest * @function getTypeUrl - * @memberof vtctldata.GetTopologyPathResponse + * @memberof vtctldata.MoveTablesCompleteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetTopologyPathResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MoveTablesCompleteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetTopologyPathResponse"; + return typeUrlPrefix + "/vtctldata.MoveTablesCompleteRequest"; }; - return GetTopologyPathResponse; + return MoveTablesCompleteRequest; })(); - vtctldata.TopologyCell = (function() { + vtctldata.MoveTablesCompleteResponse = (function() { /** - * Properties of a TopologyCell. + * Properties of a MoveTablesCompleteResponse. * @memberof vtctldata - * @interface ITopologyCell - * @property {string|null} [name] TopologyCell name - * @property {string|null} [path] TopologyCell path - * @property {string|null} [data] TopologyCell data - * @property {Array.|null} [children] TopologyCell children + * @interface IMoveTablesCompleteResponse + * @property {string|null} [summary] MoveTablesCompleteResponse summary + * @property {Array.|null} [dry_run_results] MoveTablesCompleteResponse dry_run_results */ /** - * Constructs a new TopologyCell. + * Constructs a new MoveTablesCompleteResponse. * @memberof vtctldata - * @classdesc Represents a TopologyCell. - * @implements ITopologyCell + * @classdesc Represents a MoveTablesCompleteResponse. + * @implements IMoveTablesCompleteResponse * @constructor - * @param {vtctldata.ITopologyCell=} [properties] Properties to set + * @param {vtctldata.IMoveTablesCompleteResponse=} [properties] Properties to set */ - function TopologyCell(properties) { - this.children = []; + function MoveTablesCompleteResponse(properties) { + this.dry_run_results = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -122170,120 +128789,92 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * TopologyCell name. - * @member {string} name - * @memberof vtctldata.TopologyCell - * @instance - */ - TopologyCell.prototype.name = ""; - - /** - * TopologyCell path. - * @member {string} path - * @memberof vtctldata.TopologyCell - * @instance - */ - TopologyCell.prototype.path = ""; - - /** - * TopologyCell data. - * @member {string} data - * @memberof vtctldata.TopologyCell + * MoveTablesCompleteResponse summary. + * @member {string} summary + * @memberof vtctldata.MoveTablesCompleteResponse * @instance */ - TopologyCell.prototype.data = ""; + MoveTablesCompleteResponse.prototype.summary = ""; /** - * TopologyCell children. - * @member {Array.} children - * @memberof vtctldata.TopologyCell + * MoveTablesCompleteResponse dry_run_results. + * @member {Array.} dry_run_results + * @memberof vtctldata.MoveTablesCompleteResponse * @instance */ - TopologyCell.prototype.children = $util.emptyArray; + MoveTablesCompleteResponse.prototype.dry_run_results = $util.emptyArray; /** - * Creates a new TopologyCell instance using the specified properties. + * Creates a new MoveTablesCompleteResponse instance using the specified properties. * @function create - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static - * @param {vtctldata.ITopologyCell=} [properties] Properties to set - * @returns {vtctldata.TopologyCell} TopologyCell instance + * @param {vtctldata.IMoveTablesCompleteResponse=} [properties] Properties to set + * @returns {vtctldata.MoveTablesCompleteResponse} MoveTablesCompleteResponse instance */ - TopologyCell.create = function create(properties) { - return new TopologyCell(properties); + MoveTablesCompleteResponse.create = function create(properties) { + return new MoveTablesCompleteResponse(properties); }; /** - * Encodes the specified TopologyCell message. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. + * Encodes the specified MoveTablesCompleteResponse message. Does not implicitly {@link vtctldata.MoveTablesCompleteResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static - * @param {vtctldata.ITopologyCell} message TopologyCell message or plain object to encode + * @param {vtctldata.IMoveTablesCompleteResponse} message MoveTablesCompleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TopologyCell.encode = function encode(message, writer) { + MoveTablesCompleteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.path != null && Object.hasOwnProperty.call(message, "path")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.path); - if (message.data != null && Object.hasOwnProperty.call(message, "data")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.data); - if (message.children != null && message.children.length) - for (let i = 0; i < message.children.length; ++i) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.children[i]); + if (message.summary != null && Object.hasOwnProperty.call(message, "summary")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.summary); + if (message.dry_run_results != null && message.dry_run_results.length) + for (let i = 0; i < message.dry_run_results.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.dry_run_results[i]); return writer; }; /** - * Encodes the specified TopologyCell message, length delimited. Does not implicitly {@link vtctldata.TopologyCell.verify|verify} messages. + * Encodes the specified MoveTablesCompleteResponse message, length delimited. Does not implicitly {@link vtctldata.MoveTablesCompleteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static - * @param {vtctldata.ITopologyCell} message TopologyCell message or plain object to encode + * @param {vtctldata.IMoveTablesCompleteResponse} message MoveTablesCompleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TopologyCell.encodeDelimited = function encodeDelimited(message, writer) { + MoveTablesCompleteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TopologyCell message from the specified reader or buffer. + * Decodes a MoveTablesCompleteResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.TopologyCell} TopologyCell + * @returns {vtctldata.MoveTablesCompleteResponse} MoveTablesCompleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TopologyCell.decode = function decode(reader, length) { + MoveTablesCompleteResponse.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.TopologyCell(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.MoveTablesCompleteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.summary = reader.string(); break; } case 2: { - message.path = reader.string(); - break; - } - case 3: { - message.data = reader.string(); - break; - } - case 4: { - if (!(message.children && message.children.length)) - message.children = []; - message.children.push(reader.string()); + if (!(message.dry_run_results && message.dry_run_results.length)) + message.dry_run_results = []; + message.dry_run_results.push(reader.string()); break; } default: @@ -122295,160 +128886,143 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a TopologyCell message from the specified reader or buffer, length delimited. + * Decodes a MoveTablesCompleteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.TopologyCell} TopologyCell + * @returns {vtctldata.MoveTablesCompleteResponse} MoveTablesCompleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TopologyCell.decodeDelimited = function decodeDelimited(reader) { + MoveTablesCompleteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TopologyCell message. + * Verifies a MoveTablesCompleteResponse message. * @function verify - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TopologyCell.verify = function verify(message) { + MoveTablesCompleteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.path != null && message.hasOwnProperty("path")) - if (!$util.isString(message.path)) - return "path: string expected"; - if (message.data != null && message.hasOwnProperty("data")) - if (!$util.isString(message.data)) - return "data: string expected"; - if (message.children != null && message.hasOwnProperty("children")) { - if (!Array.isArray(message.children)) - return "children: array expected"; - for (let i = 0; i < message.children.length; ++i) - if (!$util.isString(message.children[i])) - return "children: string[] expected"; + if (message.summary != null && message.hasOwnProperty("summary")) + if (!$util.isString(message.summary)) + return "summary: string expected"; + if (message.dry_run_results != null && message.hasOwnProperty("dry_run_results")) { + if (!Array.isArray(message.dry_run_results)) + return "dry_run_results: array expected"; + for (let i = 0; i < message.dry_run_results.length; ++i) + if (!$util.isString(message.dry_run_results[i])) + return "dry_run_results: string[] expected"; } return null; }; /** - * Creates a TopologyCell message from a plain object. Also converts values to their respective internal types. + * Creates a MoveTablesCompleteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.TopologyCell} TopologyCell + * @returns {vtctldata.MoveTablesCompleteResponse} MoveTablesCompleteResponse */ - TopologyCell.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.TopologyCell) + MoveTablesCompleteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.MoveTablesCompleteResponse) return object; - let message = new $root.vtctldata.TopologyCell(); - if (object.name != null) - message.name = String(object.name); - if (object.path != null) - message.path = String(object.path); - if (object.data != null) - message.data = String(object.data); - if (object.children) { - if (!Array.isArray(object.children)) - throw TypeError(".vtctldata.TopologyCell.children: array expected"); - message.children = []; - for (let i = 0; i < object.children.length; ++i) - message.children[i] = String(object.children[i]); + let message = new $root.vtctldata.MoveTablesCompleteResponse(); + if (object.summary != null) + message.summary = String(object.summary); + if (object.dry_run_results) { + if (!Array.isArray(object.dry_run_results)) + throw TypeError(".vtctldata.MoveTablesCompleteResponse.dry_run_results: array expected"); + message.dry_run_results = []; + for (let i = 0; i < object.dry_run_results.length; ++i) + message.dry_run_results[i] = String(object.dry_run_results[i]); } return message; }; /** - * Creates a plain object from a TopologyCell message. Also converts values to other types if specified. + * Creates a plain object from a MoveTablesCompleteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static - * @param {vtctldata.TopologyCell} message TopologyCell + * @param {vtctldata.MoveTablesCompleteResponse} message MoveTablesCompleteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TopologyCell.toObject = function toObject(message, options) { + MoveTablesCompleteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.children = []; - if (options.defaults) { - object.name = ""; - object.path = ""; - object.data = ""; - } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.path != null && message.hasOwnProperty("path")) - object.path = message.path; - if (message.data != null && message.hasOwnProperty("data")) - object.data = message.data; - if (message.children && message.children.length) { - object.children = []; - for (let j = 0; j < message.children.length; ++j) - object.children[j] = message.children[j]; + object.dry_run_results = []; + if (options.defaults) + object.summary = ""; + if (message.summary != null && message.hasOwnProperty("summary")) + object.summary = message.summary; + if (message.dry_run_results && message.dry_run_results.length) { + object.dry_run_results = []; + for (let j = 0; j < message.dry_run_results.length; ++j) + object.dry_run_results[j] = message.dry_run_results[j]; } return object; }; /** - * Converts this TopologyCell to JSON. + * Converts this MoveTablesCompleteResponse to JSON. * @function toJSON - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @instance * @returns {Object.} JSON object */ - TopologyCell.prototype.toJSON = function toJSON() { + MoveTablesCompleteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TopologyCell + * Gets the default type url for MoveTablesCompleteResponse * @function getTypeUrl - * @memberof vtctldata.TopologyCell + * @memberof vtctldata.MoveTablesCompleteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TopologyCell.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + MoveTablesCompleteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.TopologyCell"; + return typeUrlPrefix + "/vtctldata.MoveTablesCompleteResponse"; }; - return TopologyCell; + return MoveTablesCompleteResponse; })(); - vtctldata.GetVSchemaRequest = (function() { + vtctldata.PingTabletRequest = (function() { /** - * Properties of a GetVSchemaRequest. + * Properties of a PingTabletRequest. * @memberof vtctldata - * @interface IGetVSchemaRequest - * @property {string|null} [keyspace] GetVSchemaRequest keyspace + * @interface IPingTabletRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] PingTabletRequest tablet_alias */ /** - * Constructs a new GetVSchemaRequest. + * Constructs a new PingTabletRequest. * @memberof vtctldata - * @classdesc Represents a GetVSchemaRequest. - * @implements IGetVSchemaRequest + * @classdesc Represents a PingTabletRequest. + * @implements IPingTabletRequest * @constructor - * @param {vtctldata.IGetVSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IPingTabletRequest=} [properties] Properties to set */ - function GetVSchemaRequest(properties) { + function PingTabletRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -122456,75 +129030,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetVSchemaRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.GetVSchemaRequest + * PingTabletRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.PingTabletRequest * @instance */ - GetVSchemaRequest.prototype.keyspace = ""; + PingTabletRequest.prototype.tablet_alias = null; /** - * Creates a new GetVSchemaRequest instance using the specified properties. + * Creates a new PingTabletRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static - * @param {vtctldata.IGetVSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest instance + * @param {vtctldata.IPingTabletRequest=} [properties] Properties to set + * @returns {vtctldata.PingTabletRequest} PingTabletRequest instance */ - GetVSchemaRequest.create = function create(properties) { - return new GetVSchemaRequest(properties); + PingTabletRequest.create = function create(properties) { + return new PingTabletRequest(properties); }; /** - * Encodes the specified GetVSchemaRequest message. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. + * Encodes the specified PingTabletRequest message. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static - * @param {vtctldata.IGetVSchemaRequest} message GetVSchemaRequest message or plain object to encode + * @param {vtctldata.IPingTabletRequest} message PingTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVSchemaRequest.encode = function encode(message, writer) { + PingTabletRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaRequest.verify|verify} messages. + * Encodes the specified PingTabletRequest message, length delimited. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static - * @param {vtctldata.IGetVSchemaRequest} message GetVSchemaRequest message or plain object to encode + * @param {vtctldata.IPingTabletRequest} message PingTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + PingTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetVSchemaRequest message from the specified reader or buffer. + * Decodes a PingTabletRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest + * @returns {vtctldata.PingTabletRequest} PingTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVSchemaRequest.decode = function decode(reader, length) { + PingTabletRequest.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.GetVSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.PingTabletRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -122536,122 +129110,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a PingTabletRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest + * @returns {vtctldata.PingTabletRequest} PingTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + PingTabletRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetVSchemaRequest message. + * Verifies a PingTabletRequest message. * @function verify - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetVSchemaRequest.verify = function verify(message) { + PingTabletRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a GetVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a PingTabletRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetVSchemaRequest} GetVSchemaRequest + * @returns {vtctldata.PingTabletRequest} PingTabletRequest */ - GetVSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetVSchemaRequest) + PingTabletRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.PingTabletRequest) return object; - let message = new $root.vtctldata.GetVSchemaRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); + let message = new $root.vtctldata.PingTabletRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.PingTabletRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a GetVSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a PingTabletRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static - * @param {vtctldata.GetVSchemaRequest} message GetVSchemaRequest + * @param {vtctldata.PingTabletRequest} message PingTabletRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetVSchemaRequest.toObject = function toObject(message, options) { + PingTabletRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.keyspace = ""; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this GetVSchemaRequest to JSON. + * Converts this PingTabletRequest to JSON. * @function toJSON - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @instance * @returns {Object.} JSON object */ - GetVSchemaRequest.prototype.toJSON = function toJSON() { + PingTabletRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetVSchemaRequest + * Gets the default type url for PingTabletRequest * @function getTypeUrl - * @memberof vtctldata.GetVSchemaRequest + * @memberof vtctldata.PingTabletRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PingTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetVSchemaRequest"; + return typeUrlPrefix + "/vtctldata.PingTabletRequest"; }; - return GetVSchemaRequest; + return PingTabletRequest; })(); - vtctldata.GetVersionRequest = (function() { + vtctldata.PingTabletResponse = (function() { /** - * Properties of a GetVersionRequest. + * Properties of a PingTabletResponse. * @memberof vtctldata - * @interface IGetVersionRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] GetVersionRequest tablet_alias + * @interface IPingTabletResponse */ /** - * Constructs a new GetVersionRequest. + * Constructs a new PingTabletResponse. * @memberof vtctldata - * @classdesc Represents a GetVersionRequest. - * @implements IGetVersionRequest + * @classdesc Represents a PingTabletResponse. + * @implements IPingTabletResponse * @constructor - * @param {vtctldata.IGetVersionRequest=} [properties] Properties to set + * @param {vtctldata.IPingTabletResponse=} [properties] Properties to set */ - function GetVersionRequest(properties) { + function PingTabletResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -122659,77 +129237,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetVersionRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.GetVersionRequest - * @instance - */ - GetVersionRequest.prototype.tablet_alias = null; - - /** - * Creates a new GetVersionRequest instance using the specified properties. + * Creates a new PingTabletResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static - * @param {vtctldata.IGetVersionRequest=} [properties] Properties to set - * @returns {vtctldata.GetVersionRequest} GetVersionRequest instance + * @param {vtctldata.IPingTabletResponse=} [properties] Properties to set + * @returns {vtctldata.PingTabletResponse} PingTabletResponse instance */ - GetVersionRequest.create = function create(properties) { - return new GetVersionRequest(properties); + PingTabletResponse.create = function create(properties) { + return new PingTabletResponse(properties); }; /** - * Encodes the specified GetVersionRequest message. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. + * Encodes the specified PingTabletResponse message. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static - * @param {vtctldata.IGetVersionRequest} message GetVersionRequest message or plain object to encode + * @param {vtctldata.IPingTabletResponse} message PingTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVersionRequest.encode = function encode(message, writer) { + PingTabletResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetVersionRequest message, length delimited. Does not implicitly {@link vtctldata.GetVersionRequest.verify|verify} messages. + * Encodes the specified PingTabletResponse message, length delimited. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static - * @param {vtctldata.IGetVersionRequest} message GetVersionRequest message or plain object to encode + * @param {vtctldata.IPingTabletResponse} message PingTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVersionRequest.encodeDelimited = function encodeDelimited(message, writer) { + PingTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetVersionRequest message from the specified reader or buffer. + * Decodes a PingTabletResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetVersionRequest} GetVersionRequest + * @returns {vtctldata.PingTabletResponse} PingTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVersionRequest.decode = function decode(reader, length) { + PingTabletResponse.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.GetVersionRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.PingTabletResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -122739,127 +129303,113 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetVersionRequest message from the specified reader or buffer, length delimited. + * Decodes a PingTabletResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetVersionRequest} GetVersionRequest + * @returns {vtctldata.PingTabletResponse} PingTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVersionRequest.decodeDelimited = function decodeDelimited(reader) { + PingTabletResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetVersionRequest message. + * Verifies a PingTabletResponse message. * @function verify - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetVersionRequest.verify = function verify(message) { + PingTabletResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } return null; }; /** - * Creates a GetVersionRequest message from a plain object. Also converts values to their respective internal types. + * Creates a PingTabletResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetVersionRequest} GetVersionRequest + * @returns {vtctldata.PingTabletResponse} PingTabletResponse */ - GetVersionRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetVersionRequest) + PingTabletResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.PingTabletResponse) return object; - let message = new $root.vtctldata.GetVersionRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.GetVersionRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - return message; + return new $root.vtctldata.PingTabletResponse(); }; /** - * Creates a plain object from a GetVersionRequest message. Also converts values to other types if specified. + * Creates a plain object from a PingTabletResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static - * @param {vtctldata.GetVersionRequest} message GetVersionRequest + * @param {vtctldata.PingTabletResponse} message PingTabletResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetVersionRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - return object; + PingTabletResponse.toObject = function toObject() { + return {}; }; /** - * Converts this GetVersionRequest to JSON. + * Converts this PingTabletResponse to JSON. * @function toJSON - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @instance * @returns {Object.} JSON object */ - GetVersionRequest.prototype.toJSON = function toJSON() { + PingTabletResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetVersionRequest + * Gets the default type url for PingTabletResponse * @function getTypeUrl - * @memberof vtctldata.GetVersionRequest + * @memberof vtctldata.PingTabletResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetVersionRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PingTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetVersionRequest"; + return typeUrlPrefix + "/vtctldata.PingTabletResponse"; }; - return GetVersionRequest; + return PingTabletResponse; })(); - vtctldata.GetVersionResponse = (function() { + vtctldata.PlannedReparentShardRequest = (function() { /** - * Properties of a GetVersionResponse. + * Properties of a PlannedReparentShardRequest. * @memberof vtctldata - * @interface IGetVersionResponse - * @property {string|null} [version] GetVersionResponse version + * @interface IPlannedReparentShardRequest + * @property {string|null} [keyspace] PlannedReparentShardRequest keyspace + * @property {string|null} [shard] PlannedReparentShardRequest shard + * @property {topodata.ITabletAlias|null} [new_primary] PlannedReparentShardRequest new_primary + * @property {topodata.ITabletAlias|null} [avoid_primary] PlannedReparentShardRequest avoid_primary + * @property {vttime.IDuration|null} [wait_replicas_timeout] PlannedReparentShardRequest wait_replicas_timeout */ /** - * Constructs a new GetVersionResponse. + * Constructs a new PlannedReparentShardRequest. * @memberof vtctldata - * @classdesc Represents a GetVersionResponse. - * @implements IGetVersionResponse + * @classdesc Represents a PlannedReparentShardRequest. + * @implements IPlannedReparentShardRequest * @constructor - * @param {vtctldata.IGetVersionResponse=} [properties] Properties to set + * @param {vtctldata.IPlannedReparentShardRequest=} [properties] Properties to set */ - function GetVersionResponse(properties) { + function PlannedReparentShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -122867,75 +129417,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetVersionResponse version. - * @member {string} version - * @memberof vtctldata.GetVersionResponse + * PlannedReparentShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.PlannedReparentShardRequest * @instance */ - GetVersionResponse.prototype.version = ""; + PlannedReparentShardRequest.prototype.keyspace = ""; /** - * Creates a new GetVersionResponse instance using the specified properties. + * PlannedReparentShardRequest shard. + * @member {string} shard + * @memberof vtctldata.PlannedReparentShardRequest + * @instance + */ + PlannedReparentShardRequest.prototype.shard = ""; + + /** + * PlannedReparentShardRequest new_primary. + * @member {topodata.ITabletAlias|null|undefined} new_primary + * @memberof vtctldata.PlannedReparentShardRequest + * @instance + */ + PlannedReparentShardRequest.prototype.new_primary = null; + + /** + * PlannedReparentShardRequest avoid_primary. + * @member {topodata.ITabletAlias|null|undefined} avoid_primary + * @memberof vtctldata.PlannedReparentShardRequest + * @instance + */ + PlannedReparentShardRequest.prototype.avoid_primary = null; + + /** + * PlannedReparentShardRequest wait_replicas_timeout. + * @member {vttime.IDuration|null|undefined} wait_replicas_timeout + * @memberof vtctldata.PlannedReparentShardRequest + * @instance + */ + PlannedReparentShardRequest.prototype.wait_replicas_timeout = null; + + /** + * Creates a new PlannedReparentShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static - * @param {vtctldata.IGetVersionResponse=} [properties] Properties to set - * @returns {vtctldata.GetVersionResponse} GetVersionResponse instance + * @param {vtctldata.IPlannedReparentShardRequest=} [properties] Properties to set + * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest instance */ - GetVersionResponse.create = function create(properties) { - return new GetVersionResponse(properties); + PlannedReparentShardRequest.create = function create(properties) { + return new PlannedReparentShardRequest(properties); }; /** - * Encodes the specified GetVersionResponse message. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. + * Encodes the specified PlannedReparentShardRequest message. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static - * @param {vtctldata.IGetVersionResponse} message GetVersionResponse message or plain object to encode + * @param {vtctldata.IPlannedReparentShardRequest} message PlannedReparentShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVersionResponse.encode = function encode(message, writer) { + PlannedReparentShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.version != null && Object.hasOwnProperty.call(message, "version")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.version); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) + $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.avoid_primary != null && Object.hasOwnProperty.call(message, "avoid_primary")) + $root.topodata.TabletAlias.encode(message.avoid_primary, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) + $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetVersionResponse message, length delimited. Does not implicitly {@link vtctldata.GetVersionResponse.verify|verify} messages. + * Encodes the specified PlannedReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static - * @param {vtctldata.IGetVersionResponse} message GetVersionResponse message or plain object to encode + * @param {vtctldata.IPlannedReparentShardRequest} message PlannedReparentShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVersionResponse.encodeDelimited = function encodeDelimited(message, writer) { + PlannedReparentShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetVersionResponse message from the specified reader or buffer. + * Decodes a PlannedReparentShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetVersionResponse} GetVersionResponse + * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVersionResponse.decode = function decode(reader, length) { + PlannedReparentShardRequest.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.GetVersionResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.PlannedReparentShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.version = reader.string(); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + message.avoid_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 5: { + message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); break; } default: @@ -122947,122 +129553,174 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetVersionResponse message from the specified reader or buffer, length delimited. + * Decodes a PlannedReparentShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetVersionResponse} GetVersionResponse + * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVersionResponse.decodeDelimited = function decodeDelimited(reader) { + PlannedReparentShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetVersionResponse message. + * Verifies a PlannedReparentShardRequest message. * @function verify - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetVersionResponse.verify = function verify(message) { + PlannedReparentShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.version != null && message.hasOwnProperty("version")) - if (!$util.isString(message.version)) - return "version: string expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) { + let error = $root.topodata.TabletAlias.verify(message.new_primary); + if (error) + return "new_primary." + error; + } + if (message.avoid_primary != null && message.hasOwnProperty("avoid_primary")) { + let error = $root.topodata.TabletAlias.verify(message.avoid_primary); + if (error) + return "avoid_primary." + error; + } + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { + let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); + if (error) + return "wait_replicas_timeout." + error; + } return null; }; /** - * Creates a GetVersionResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PlannedReparentShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetVersionResponse} GetVersionResponse + * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest */ - GetVersionResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetVersionResponse) + PlannedReparentShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.PlannedReparentShardRequest) return object; - let message = new $root.vtctldata.GetVersionResponse(); - if (object.version != null) - message.version = String(object.version); + let message = new $root.vtctldata.PlannedReparentShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.new_primary != null) { + if (typeof object.new_primary !== "object") + throw TypeError(".vtctldata.PlannedReparentShardRequest.new_primary: object expected"); + message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); + } + if (object.avoid_primary != null) { + if (typeof object.avoid_primary !== "object") + throw TypeError(".vtctldata.PlannedReparentShardRequest.avoid_primary: object expected"); + message.avoid_primary = $root.topodata.TabletAlias.fromObject(object.avoid_primary); + } + if (object.wait_replicas_timeout != null) { + if (typeof object.wait_replicas_timeout !== "object") + throw TypeError(".vtctldata.PlannedReparentShardRequest.wait_replicas_timeout: object expected"); + message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + } return message; }; /** - * Creates a plain object from a GetVersionResponse message. Also converts values to other types if specified. + * Creates a plain object from a PlannedReparentShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static - * @param {vtctldata.GetVersionResponse} message GetVersionResponse + * @param {vtctldata.PlannedReparentShardRequest} message PlannedReparentShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetVersionResponse.toObject = function toObject(message, options) { + PlannedReparentShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.version = ""; - if (message.version != null && message.hasOwnProperty("version")) - object.version = message.version; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.new_primary = null; + object.avoid_primary = null; + object.wait_replicas_timeout = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) + object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); + if (message.avoid_primary != null && message.hasOwnProperty("avoid_primary")) + object.avoid_primary = $root.topodata.TabletAlias.toObject(message.avoid_primary, options); + if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) + object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); return object; }; /** - * Converts this GetVersionResponse to JSON. + * Converts this PlannedReparentShardRequest to JSON. * @function toJSON - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @instance * @returns {Object.} JSON object */ - GetVersionResponse.prototype.toJSON = function toJSON() { + PlannedReparentShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetVersionResponse + * Gets the default type url for PlannedReparentShardRequest * @function getTypeUrl - * @memberof vtctldata.GetVersionResponse + * @memberof vtctldata.PlannedReparentShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetVersionResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PlannedReparentShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetVersionResponse"; + return typeUrlPrefix + "/vtctldata.PlannedReparentShardRequest"; }; - return GetVersionResponse; + return PlannedReparentShardRequest; })(); - vtctldata.GetVSchemaResponse = (function() { + vtctldata.PlannedReparentShardResponse = (function() { /** - * Properties of a GetVSchemaResponse. + * Properties of a PlannedReparentShardResponse. * @memberof vtctldata - * @interface IGetVSchemaResponse - * @property {vschema.IKeyspace|null} [v_schema] GetVSchemaResponse v_schema + * @interface IPlannedReparentShardResponse + * @property {string|null} [keyspace] PlannedReparentShardResponse keyspace + * @property {string|null} [shard] PlannedReparentShardResponse shard + * @property {topodata.ITabletAlias|null} [promoted_primary] PlannedReparentShardResponse promoted_primary + * @property {Array.|null} [events] PlannedReparentShardResponse events */ /** - * Constructs a new GetVSchemaResponse. + * Constructs a new PlannedReparentShardResponse. * @memberof vtctldata - * @classdesc Represents a GetVSchemaResponse. - * @implements IGetVSchemaResponse + * @classdesc Represents a PlannedReparentShardResponse. + * @implements IPlannedReparentShardResponse * @constructor - * @param {vtctldata.IGetVSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IPlannedReparentShardResponse=} [properties] Properties to set */ - function GetVSchemaResponse(properties) { + function PlannedReparentShardResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -123070,75 +129728,120 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetVSchemaResponse v_schema. - * @member {vschema.IKeyspace|null|undefined} v_schema - * @memberof vtctldata.GetVSchemaResponse + * PlannedReparentShardResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.PlannedReparentShardResponse * @instance */ - GetVSchemaResponse.prototype.v_schema = null; + PlannedReparentShardResponse.prototype.keyspace = ""; /** - * Creates a new GetVSchemaResponse instance using the specified properties. + * PlannedReparentShardResponse shard. + * @member {string} shard + * @memberof vtctldata.PlannedReparentShardResponse + * @instance + */ + PlannedReparentShardResponse.prototype.shard = ""; + + /** + * PlannedReparentShardResponse promoted_primary. + * @member {topodata.ITabletAlias|null|undefined} promoted_primary + * @memberof vtctldata.PlannedReparentShardResponse + * @instance + */ + PlannedReparentShardResponse.prototype.promoted_primary = null; + + /** + * PlannedReparentShardResponse events. + * @member {Array.} events + * @memberof vtctldata.PlannedReparentShardResponse + * @instance + */ + PlannedReparentShardResponse.prototype.events = $util.emptyArray; + + /** + * Creates a new PlannedReparentShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static - * @param {vtctldata.IGetVSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse instance + * @param {vtctldata.IPlannedReparentShardResponse=} [properties] Properties to set + * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse instance */ - GetVSchemaResponse.create = function create(properties) { - return new GetVSchemaResponse(properties); + PlannedReparentShardResponse.create = function create(properties) { + return new PlannedReparentShardResponse(properties); }; /** - * Encodes the specified GetVSchemaResponse message. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. + * Encodes the specified PlannedReparentShardResponse message. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static - * @param {vtctldata.IGetVSchemaResponse} message GetVSchemaResponse message or plain object to encode + * @param {vtctldata.IPlannedReparentShardResponse} message PlannedReparentShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVSchemaResponse.encode = function encode(message, writer) { + PlannedReparentShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.v_schema != null && Object.hasOwnProperty.call(message, "v_schema")) - $root.vschema.Keyspace.encode(message.v_schema, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.promoted_primary != null && Object.hasOwnProperty.call(message, "promoted_primary")) + $root.topodata.TabletAlias.encode(message.promoted_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.GetVSchemaResponse.verify|verify} messages. + * Encodes the specified PlannedReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static - * @param {vtctldata.IGetVSchemaResponse} message GetVSchemaResponse message or plain object to encode + * @param {vtctldata.IPlannedReparentShardResponse} message PlannedReparentShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + PlannedReparentShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetVSchemaResponse message from the specified reader or buffer. + * Decodes a PlannedReparentShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse + * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVSchemaResponse.decode = function decode(reader, length) { + PlannedReparentShardResponse.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.GetVSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.PlannedReparentShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.v_schema = $root.vschema.Keyspace.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.promoted_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.logutil.Event.decode(reader, reader.uint32())); break; } default: @@ -123150,129 +129853,173 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a PlannedReparentShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse + * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + PlannedReparentShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetVSchemaResponse message. + * Verifies a PlannedReparentShardResponse message. * @function verify - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetVSchemaResponse.verify = function verify(message) { + PlannedReparentShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.v_schema != null && message.hasOwnProperty("v_schema")) { - let error = $root.vschema.Keyspace.verify(message.v_schema); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) { + let error = $root.topodata.TabletAlias.verify(message.promoted_primary); if (error) - return "v_schema." + error; + return "promoted_primary." + error; + } + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.logutil.Event.verify(message.events[i]); + if (error) + return "events." + error; + } } return null; }; /** - * Creates a GetVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a PlannedReparentShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetVSchemaResponse} GetVSchemaResponse + * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse */ - GetVSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetVSchemaResponse) + PlannedReparentShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.PlannedReparentShardResponse) return object; - let message = new $root.vtctldata.GetVSchemaResponse(); - if (object.v_schema != null) { - if (typeof object.v_schema !== "object") - throw TypeError(".vtctldata.GetVSchemaResponse.v_schema: object expected"); - message.v_schema = $root.vschema.Keyspace.fromObject(object.v_schema); + let message = new $root.vtctldata.PlannedReparentShardResponse(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.promoted_primary != null) { + if (typeof object.promoted_primary !== "object") + throw TypeError(".vtctldata.PlannedReparentShardResponse.promoted_primary: object expected"); + message.promoted_primary = $root.topodata.TabletAlias.fromObject(object.promoted_primary); + } + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".vtctldata.PlannedReparentShardResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".vtctldata.PlannedReparentShardResponse.events: object expected"); + message.events[i] = $root.logutil.Event.fromObject(object.events[i]); + } } return message; }; /** - * Creates a plain object from a GetVSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a PlannedReparentShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static - * @param {vtctldata.GetVSchemaResponse} message GetVSchemaResponse + * @param {vtctldata.PlannedReparentShardResponse} message PlannedReparentShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetVSchemaResponse.toObject = function toObject(message, options) { + PlannedReparentShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.v_schema = null; - if (message.v_schema != null && message.hasOwnProperty("v_schema")) - object.v_schema = $root.vschema.Keyspace.toObject(message.v_schema, options); + if (options.arrays || options.defaults) + object.events = []; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.promoted_primary = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) + object.promoted_primary = $root.topodata.TabletAlias.toObject(message.promoted_primary, options); + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + } return object; }; /** - * Converts this GetVSchemaResponse to JSON. + * Converts this PlannedReparentShardResponse to JSON. * @function toJSON - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @instance * @returns {Object.} JSON object */ - GetVSchemaResponse.prototype.toJSON = function toJSON() { + PlannedReparentShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetVSchemaResponse + * Gets the default type url for PlannedReparentShardResponse * @function getTypeUrl - * @memberof vtctldata.GetVSchemaResponse + * @memberof vtctldata.PlannedReparentShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + PlannedReparentShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetVSchemaResponse"; + return typeUrlPrefix + "/vtctldata.PlannedReparentShardResponse"; }; - return GetVSchemaResponse; + return PlannedReparentShardResponse; })(); - vtctldata.GetWorkflowsRequest = (function() { + vtctldata.RebuildKeyspaceGraphRequest = (function() { /** - * Properties of a GetWorkflowsRequest. + * Properties of a RebuildKeyspaceGraphRequest. * @memberof vtctldata - * @interface IGetWorkflowsRequest - * @property {string|null} [keyspace] GetWorkflowsRequest keyspace - * @property {boolean|null} [active_only] GetWorkflowsRequest active_only - * @property {boolean|null} [name_only] GetWorkflowsRequest name_only + * @interface IRebuildKeyspaceGraphRequest + * @property {string|null} [keyspace] RebuildKeyspaceGraphRequest keyspace + * @property {Array.|null} [cells] RebuildKeyspaceGraphRequest cells + * @property {boolean|null} [allow_partial] RebuildKeyspaceGraphRequest allow_partial */ /** - * Constructs a new GetWorkflowsRequest. + * Constructs a new RebuildKeyspaceGraphRequest. * @memberof vtctldata - * @classdesc Represents a GetWorkflowsRequest. - * @implements IGetWorkflowsRequest + * @classdesc Represents a RebuildKeyspaceGraphRequest. + * @implements IRebuildKeyspaceGraphRequest * @constructor - * @param {vtctldata.IGetWorkflowsRequest=} [properties] Properties to set + * @param {vtctldata.IRebuildKeyspaceGraphRequest=} [properties] Properties to set */ - function GetWorkflowsRequest(properties) { + function RebuildKeyspaceGraphRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -123280,90 +130027,91 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetWorkflowsRequest keyspace. + * RebuildKeyspaceGraphRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @instance */ - GetWorkflowsRequest.prototype.keyspace = ""; + RebuildKeyspaceGraphRequest.prototype.keyspace = ""; /** - * GetWorkflowsRequest active_only. - * @member {boolean} active_only - * @memberof vtctldata.GetWorkflowsRequest + * RebuildKeyspaceGraphRequest cells. + * @member {Array.} cells + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @instance */ - GetWorkflowsRequest.prototype.active_only = false; + RebuildKeyspaceGraphRequest.prototype.cells = $util.emptyArray; /** - * GetWorkflowsRequest name_only. - * @member {boolean} name_only - * @memberof vtctldata.GetWorkflowsRequest + * RebuildKeyspaceGraphRequest allow_partial. + * @member {boolean} allow_partial + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @instance */ - GetWorkflowsRequest.prototype.name_only = false; + RebuildKeyspaceGraphRequest.prototype.allow_partial = false; /** - * Creates a new GetWorkflowsRequest instance using the specified properties. + * Creates a new RebuildKeyspaceGraphRequest instance using the specified properties. * @function create - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static - * @param {vtctldata.IGetWorkflowsRequest=} [properties] Properties to set - * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest instance + * @param {vtctldata.IRebuildKeyspaceGraphRequest=} [properties] Properties to set + * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest instance */ - GetWorkflowsRequest.create = function create(properties) { - return new GetWorkflowsRequest(properties); + RebuildKeyspaceGraphRequest.create = function create(properties) { + return new RebuildKeyspaceGraphRequest(properties); }; /** - * Encodes the specified GetWorkflowsRequest message. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. + * Encodes the specified RebuildKeyspaceGraphRequest message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static - * @param {vtctldata.IGetWorkflowsRequest} message GetWorkflowsRequest message or plain object to encode + * @param {vtctldata.IRebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetWorkflowsRequest.encode = function encode(message, writer) { + RebuildKeyspaceGraphRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.active_only != null && Object.hasOwnProperty.call(message, "active_only")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.active_only); - if (message.name_only != null && Object.hasOwnProperty.call(message, "name_only")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.name_only); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.allow_partial != null && Object.hasOwnProperty.call(message, "allow_partial")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_partial); return writer; }; /** - * Encodes the specified GetWorkflowsRequest message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsRequest.verify|verify} messages. + * Encodes the specified RebuildKeyspaceGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static - * @param {vtctldata.IGetWorkflowsRequest} message GetWorkflowsRequest message or plain object to encode + * @param {vtctldata.IRebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetWorkflowsRequest.encodeDelimited = function encodeDelimited(message, writer) { + RebuildKeyspaceGraphRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetWorkflowsRequest message from the specified reader or buffer. + * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest + * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetWorkflowsRequest.decode = function decode(reader, length) { + RebuildKeyspaceGraphRequest.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.GetWorkflowsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RebuildKeyspaceGraphRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -123372,11 +130120,13 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.active_only = reader.bool(); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } case 3: { - message.name_only = reader.bool(); + message.allow_partial = reader.bool(); break; } default: @@ -123388,140 +130138,151 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetWorkflowsRequest message from the specified reader or buffer, length delimited. + * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest + * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetWorkflowsRequest.decodeDelimited = function decodeDelimited(reader) { + RebuildKeyspaceGraphRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetWorkflowsRequest message. + * Verifies a RebuildKeyspaceGraphRequest message. * @function verify - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetWorkflowsRequest.verify = function verify(message) { + RebuildKeyspaceGraphRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.active_only != null && message.hasOwnProperty("active_only")) - if (typeof message.active_only !== "boolean") - return "active_only: boolean expected"; - if (message.name_only != null && message.hasOwnProperty("name_only")) - if (typeof message.name_only !== "boolean") - return "name_only: boolean expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.allow_partial != null && message.hasOwnProperty("allow_partial")) + if (typeof message.allow_partial !== "boolean") + return "allow_partial: boolean expected"; return null; }; /** - * Creates a GetWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildKeyspaceGraphRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetWorkflowsRequest} GetWorkflowsRequest + * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest */ - GetWorkflowsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetWorkflowsRequest) + RebuildKeyspaceGraphRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RebuildKeyspaceGraphRequest) return object; - let message = new $root.vtctldata.GetWorkflowsRequest(); + let message = new $root.vtctldata.RebuildKeyspaceGraphRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.active_only != null) - message.active_only = Boolean(object.active_only); - if (object.name_only != null) - message.name_only = Boolean(object.name_only); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.RebuildKeyspaceGraphRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.allow_partial != null) + message.allow_partial = Boolean(object.allow_partial); return message; }; /** - * Creates a plain object from a GetWorkflowsRequest message. Also converts values to other types if specified. + * Creates a plain object from a RebuildKeyspaceGraphRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static - * @param {vtctldata.GetWorkflowsRequest} message GetWorkflowsRequest + * @param {vtctldata.RebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetWorkflowsRequest.toObject = function toObject(message, options) { + RebuildKeyspaceGraphRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.cells = []; if (options.defaults) { object.keyspace = ""; - object.active_only = false; - object.name_only = false; + object.allow_partial = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.active_only != null && message.hasOwnProperty("active_only")) - object.active_only = message.active_only; - if (message.name_only != null && message.hasOwnProperty("name_only")) - object.name_only = message.name_only; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.allow_partial != null && message.hasOwnProperty("allow_partial")) + object.allow_partial = message.allow_partial; return object; }; /** - * Converts this GetWorkflowsRequest to JSON. + * Converts this RebuildKeyspaceGraphRequest to JSON. * @function toJSON - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @instance * @returns {Object.} JSON object */ - GetWorkflowsRequest.prototype.toJSON = function toJSON() { + RebuildKeyspaceGraphRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetWorkflowsRequest + * Gets the default type url for RebuildKeyspaceGraphRequest * @function getTypeUrl - * @memberof vtctldata.GetWorkflowsRequest + * @memberof vtctldata.RebuildKeyspaceGraphRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetWorkflowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RebuildKeyspaceGraphRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetWorkflowsRequest"; + return typeUrlPrefix + "/vtctldata.RebuildKeyspaceGraphRequest"; }; - return GetWorkflowsRequest; + return RebuildKeyspaceGraphRequest; })(); - vtctldata.GetWorkflowsResponse = (function() { + vtctldata.RebuildKeyspaceGraphResponse = (function() { /** - * Properties of a GetWorkflowsResponse. + * Properties of a RebuildKeyspaceGraphResponse. * @memberof vtctldata - * @interface IGetWorkflowsResponse - * @property {Array.|null} [workflows] GetWorkflowsResponse workflows + * @interface IRebuildKeyspaceGraphResponse */ /** - * Constructs a new GetWorkflowsResponse. + * Constructs a new RebuildKeyspaceGraphResponse. * @memberof vtctldata - * @classdesc Represents a GetWorkflowsResponse. - * @implements IGetWorkflowsResponse + * @classdesc Represents a RebuildKeyspaceGraphResponse. + * @implements IRebuildKeyspaceGraphResponse * @constructor - * @param {vtctldata.IGetWorkflowsResponse=} [properties] Properties to set + * @param {vtctldata.IRebuildKeyspaceGraphResponse=} [properties] Properties to set */ - function GetWorkflowsResponse(properties) { - this.workflows = []; + function RebuildKeyspaceGraphResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -123529,80 +130290,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * GetWorkflowsResponse workflows. - * @member {Array.} workflows - * @memberof vtctldata.GetWorkflowsResponse - * @instance - */ - GetWorkflowsResponse.prototype.workflows = $util.emptyArray; - - /** - * Creates a new GetWorkflowsResponse instance using the specified properties. + * Creates a new RebuildKeyspaceGraphResponse instance using the specified properties. * @function create - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static - * @param {vtctldata.IGetWorkflowsResponse=} [properties] Properties to set - * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse instance + * @param {vtctldata.IRebuildKeyspaceGraphResponse=} [properties] Properties to set + * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse instance */ - GetWorkflowsResponse.create = function create(properties) { - return new GetWorkflowsResponse(properties); + RebuildKeyspaceGraphResponse.create = function create(properties) { + return new RebuildKeyspaceGraphResponse(properties); }; /** - * Encodes the specified GetWorkflowsResponse message. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. + * Encodes the specified RebuildKeyspaceGraphResponse message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static - * @param {vtctldata.IGetWorkflowsResponse} message GetWorkflowsResponse message or plain object to encode + * @param {vtctldata.IRebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetWorkflowsResponse.encode = function encode(message, writer) { + RebuildKeyspaceGraphResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.workflows != null && message.workflows.length) - for (let i = 0; i < message.workflows.length; ++i) - $root.vtctldata.Workflow.encode(message.workflows[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified GetWorkflowsResponse message, length delimited. Does not implicitly {@link vtctldata.GetWorkflowsResponse.verify|verify} messages. + * Encodes the specified RebuildKeyspaceGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static - * @param {vtctldata.IGetWorkflowsResponse} message GetWorkflowsResponse message or plain object to encode + * @param {vtctldata.IRebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - GetWorkflowsResponse.encodeDelimited = function encodeDelimited(message, writer) { + RebuildKeyspaceGraphResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a GetWorkflowsResponse message from the specified reader or buffer. + * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse + * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetWorkflowsResponse.decode = function decode(reader, length) { + RebuildKeyspaceGraphResponse.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.GetWorkflowsResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RebuildKeyspaceGraphResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - if (!(message.workflows && message.workflows.length)) - message.workflows = []; - message.workflows.push($root.vtctldata.Workflow.decode(reader, reader.uint32())); - break; - } default: reader.skipType(tag & 7); break; @@ -123612,143 +130356,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a GetWorkflowsResponse message from the specified reader or buffer, length delimited. + * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse + * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - GetWorkflowsResponse.decodeDelimited = function decodeDelimited(reader) { + RebuildKeyspaceGraphResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a GetWorkflowsResponse message. + * Verifies a RebuildKeyspaceGraphResponse message. * @function verify - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - GetWorkflowsResponse.verify = function verify(message) { + RebuildKeyspaceGraphResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.workflows != null && message.hasOwnProperty("workflows")) { - if (!Array.isArray(message.workflows)) - return "workflows: array expected"; - for (let i = 0; i < message.workflows.length; ++i) { - let error = $root.vtctldata.Workflow.verify(message.workflows[i]); - if (error) - return "workflows." + error; - } - } return null; }; /** - * Creates a GetWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildKeyspaceGraphResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.GetWorkflowsResponse} GetWorkflowsResponse + * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse */ - GetWorkflowsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.GetWorkflowsResponse) + RebuildKeyspaceGraphResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RebuildKeyspaceGraphResponse) return object; - let message = new $root.vtctldata.GetWorkflowsResponse(); - if (object.workflows) { - if (!Array.isArray(object.workflows)) - throw TypeError(".vtctldata.GetWorkflowsResponse.workflows: array expected"); - message.workflows = []; - for (let i = 0; i < object.workflows.length; ++i) { - if (typeof object.workflows[i] !== "object") - throw TypeError(".vtctldata.GetWorkflowsResponse.workflows: object expected"); - message.workflows[i] = $root.vtctldata.Workflow.fromObject(object.workflows[i]); - } - } - return message; + return new $root.vtctldata.RebuildKeyspaceGraphResponse(); }; /** - * Creates a plain object from a GetWorkflowsResponse message. Also converts values to other types if specified. + * Creates a plain object from a RebuildKeyspaceGraphResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static - * @param {vtctldata.GetWorkflowsResponse} message GetWorkflowsResponse + * @param {vtctldata.RebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - GetWorkflowsResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.workflows = []; - if (message.workflows && message.workflows.length) { - object.workflows = []; - for (let j = 0; j < message.workflows.length; ++j) - object.workflows[j] = $root.vtctldata.Workflow.toObject(message.workflows[j], options); - } - return object; + RebuildKeyspaceGraphResponse.toObject = function toObject() { + return {}; }; /** - * Converts this GetWorkflowsResponse to JSON. + * Converts this RebuildKeyspaceGraphResponse to JSON. * @function toJSON - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @instance * @returns {Object.} JSON object */ - GetWorkflowsResponse.prototype.toJSON = function toJSON() { + RebuildKeyspaceGraphResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for GetWorkflowsResponse + * Gets the default type url for RebuildKeyspaceGraphResponse * @function getTypeUrl - * @memberof vtctldata.GetWorkflowsResponse + * @memberof vtctldata.RebuildKeyspaceGraphResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - GetWorkflowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RebuildKeyspaceGraphResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.GetWorkflowsResponse"; + return typeUrlPrefix + "/vtctldata.RebuildKeyspaceGraphResponse"; }; - return GetWorkflowsResponse; + return RebuildKeyspaceGraphResponse; })(); - vtctldata.InitShardPrimaryRequest = (function() { + vtctldata.RebuildVSchemaGraphRequest = (function() { /** - * Properties of an InitShardPrimaryRequest. + * Properties of a RebuildVSchemaGraphRequest. * @memberof vtctldata - * @interface IInitShardPrimaryRequest - * @property {string|null} [keyspace] InitShardPrimaryRequest keyspace - * @property {string|null} [shard] InitShardPrimaryRequest shard - * @property {topodata.ITabletAlias|null} [primary_elect_tablet_alias] InitShardPrimaryRequest primary_elect_tablet_alias - * @property {boolean|null} [force] InitShardPrimaryRequest force - * @property {vttime.IDuration|null} [wait_replicas_timeout] InitShardPrimaryRequest wait_replicas_timeout + * @interface IRebuildVSchemaGraphRequest + * @property {Array.|null} [cells] RebuildVSchemaGraphRequest cells */ /** - * Constructs a new InitShardPrimaryRequest. + * Constructs a new RebuildVSchemaGraphRequest. * @memberof vtctldata - * @classdesc Represents an InitShardPrimaryRequest. - * @implements IInitShardPrimaryRequest + * @classdesc Represents a RebuildVSchemaGraphRequest. + * @implements IRebuildVSchemaGraphRequest * @constructor - * @param {vtctldata.IInitShardPrimaryRequest=} [properties] Properties to set + * @param {vtctldata.IRebuildVSchemaGraphRequest=} [properties] Properties to set */ - function InitShardPrimaryRequest(properties) { + function RebuildVSchemaGraphRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -123756,131 +130467,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * InitShardPrimaryRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.InitShardPrimaryRequest - * @instance - */ - InitShardPrimaryRequest.prototype.keyspace = ""; - - /** - * InitShardPrimaryRequest shard. - * @member {string} shard - * @memberof vtctldata.InitShardPrimaryRequest - * @instance - */ - InitShardPrimaryRequest.prototype.shard = ""; - - /** - * InitShardPrimaryRequest primary_elect_tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} primary_elect_tablet_alias - * @memberof vtctldata.InitShardPrimaryRequest - * @instance - */ - InitShardPrimaryRequest.prototype.primary_elect_tablet_alias = null; - - /** - * InitShardPrimaryRequest force. - * @member {boolean} force - * @memberof vtctldata.InitShardPrimaryRequest - * @instance - */ - InitShardPrimaryRequest.prototype.force = false; - - /** - * InitShardPrimaryRequest wait_replicas_timeout. - * @member {vttime.IDuration|null|undefined} wait_replicas_timeout - * @memberof vtctldata.InitShardPrimaryRequest + * RebuildVSchemaGraphRequest cells. + * @member {Array.} cells + * @memberof vtctldata.RebuildVSchemaGraphRequest * @instance */ - InitShardPrimaryRequest.prototype.wait_replicas_timeout = null; + RebuildVSchemaGraphRequest.prototype.cells = $util.emptyArray; /** - * Creates a new InitShardPrimaryRequest instance using the specified properties. + * Creates a new RebuildVSchemaGraphRequest instance using the specified properties. * @function create - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static - * @param {vtctldata.IInitShardPrimaryRequest=} [properties] Properties to set - * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest instance + * @param {vtctldata.IRebuildVSchemaGraphRequest=} [properties] Properties to set + * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest instance */ - InitShardPrimaryRequest.create = function create(properties) { - return new InitShardPrimaryRequest(properties); + RebuildVSchemaGraphRequest.create = function create(properties) { + return new RebuildVSchemaGraphRequest(properties); }; /** - * Encodes the specified InitShardPrimaryRequest message. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. + * Encodes the specified RebuildVSchemaGraphRequest message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static - * @param {vtctldata.IInitShardPrimaryRequest} message InitShardPrimaryRequest message or plain object to encode + * @param {vtctldata.IRebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - InitShardPrimaryRequest.encode = function encode(message, writer) { + RebuildVSchemaGraphRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.primary_elect_tablet_alias != null && Object.hasOwnProperty.call(message, "primary_elect_tablet_alias")) - $root.topodata.TabletAlias.encode(message.primary_elect_tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); - if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) - $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.cells[i]); return writer; }; /** - * Encodes the specified InitShardPrimaryRequest message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryRequest.verify|verify} messages. + * Encodes the specified RebuildVSchemaGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static - * @param {vtctldata.IInitShardPrimaryRequest} message InitShardPrimaryRequest message or plain object to encode + * @param {vtctldata.IRebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - InitShardPrimaryRequest.encodeDelimited = function encodeDelimited(message, writer) { + RebuildVSchemaGraphRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an InitShardPrimaryRequest message from the specified reader or buffer. + * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest + * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - InitShardPrimaryRequest.decode = function decode(reader, length) { + RebuildVSchemaGraphRequest.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.InitShardPrimaryRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RebuildVSchemaGraphRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.primary_elect_tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - message.force = reader.bool(); - break; - } - case 5: { - message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } default: @@ -123892,166 +130550,133 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an InitShardPrimaryRequest message from the specified reader or buffer, length delimited. + * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest + * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - InitShardPrimaryRequest.decodeDelimited = function decodeDelimited(reader) { + RebuildVSchemaGraphRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an InitShardPrimaryRequest message. + * Verifies a RebuildVSchemaGraphRequest message. * @function verify - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - InitShardPrimaryRequest.verify = function verify(message) { + RebuildVSchemaGraphRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.primary_elect_tablet_alias != null && message.hasOwnProperty("primary_elect_tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.primary_elect_tablet_alias); - if (error) - return "primary_elect_tablet_alias." + error; - } - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { - let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); - if (error) - return "wait_replicas_timeout." + error; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; } return null; }; /** - * Creates an InitShardPrimaryRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildVSchemaGraphRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.InitShardPrimaryRequest} InitShardPrimaryRequest + * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest */ - InitShardPrimaryRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.InitShardPrimaryRequest) + RebuildVSchemaGraphRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RebuildVSchemaGraphRequest) return object; - let message = new $root.vtctldata.InitShardPrimaryRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.primary_elect_tablet_alias != null) { - if (typeof object.primary_elect_tablet_alias !== "object") - throw TypeError(".vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias: object expected"); - message.primary_elect_tablet_alias = $root.topodata.TabletAlias.fromObject(object.primary_elect_tablet_alias); - } - if (object.force != null) - message.force = Boolean(object.force); - if (object.wait_replicas_timeout != null) { - if (typeof object.wait_replicas_timeout !== "object") - throw TypeError(".vtctldata.InitShardPrimaryRequest.wait_replicas_timeout: object expected"); - message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + let message = new $root.vtctldata.RebuildVSchemaGraphRequest(); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.RebuildVSchemaGraphRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); } return message; }; /** - * Creates a plain object from an InitShardPrimaryRequest message. Also converts values to other types if specified. + * Creates a plain object from a RebuildVSchemaGraphRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static - * @param {vtctldata.InitShardPrimaryRequest} message InitShardPrimaryRequest + * @param {vtctldata.RebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - InitShardPrimaryRequest.toObject = function toObject(message, options) { + RebuildVSchemaGraphRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.primary_elect_tablet_alias = null; - object.force = false; - object.wait_replicas_timeout = null; + if (options.arrays || options.defaults) + object.cells = []; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.primary_elect_tablet_alias != null && message.hasOwnProperty("primary_elect_tablet_alias")) - object.primary_elect_tablet_alias = $root.topodata.TabletAlias.toObject(message.primary_elect_tablet_alias, options); - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) - object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); return object; }; /** - * Converts this InitShardPrimaryRequest to JSON. + * Converts this RebuildVSchemaGraphRequest to JSON. * @function toJSON - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @instance * @returns {Object.} JSON object */ - InitShardPrimaryRequest.prototype.toJSON = function toJSON() { + RebuildVSchemaGraphRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for InitShardPrimaryRequest + * Gets the default type url for RebuildVSchemaGraphRequest * @function getTypeUrl - * @memberof vtctldata.InitShardPrimaryRequest + * @memberof vtctldata.RebuildVSchemaGraphRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - InitShardPrimaryRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RebuildVSchemaGraphRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.InitShardPrimaryRequest"; + return typeUrlPrefix + "/vtctldata.RebuildVSchemaGraphRequest"; }; - return InitShardPrimaryRequest; + return RebuildVSchemaGraphRequest; })(); - vtctldata.InitShardPrimaryResponse = (function() { + vtctldata.RebuildVSchemaGraphResponse = (function() { /** - * Properties of an InitShardPrimaryResponse. + * Properties of a RebuildVSchemaGraphResponse. * @memberof vtctldata - * @interface IInitShardPrimaryResponse - * @property {Array.|null} [events] InitShardPrimaryResponse events + * @interface IRebuildVSchemaGraphResponse */ /** - * Constructs a new InitShardPrimaryResponse. + * Constructs a new RebuildVSchemaGraphResponse. * @memberof vtctldata - * @classdesc Represents an InitShardPrimaryResponse. - * @implements IInitShardPrimaryResponse + * @classdesc Represents a RebuildVSchemaGraphResponse. + * @implements IRebuildVSchemaGraphResponse * @constructor - * @param {vtctldata.IInitShardPrimaryResponse=} [properties] Properties to set + * @param {vtctldata.IRebuildVSchemaGraphResponse=} [properties] Properties to set */ - function InitShardPrimaryResponse(properties) { - this.events = []; + function RebuildVSchemaGraphResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -124059,80 +130684,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * InitShardPrimaryResponse events. - * @member {Array.} events - * @memberof vtctldata.InitShardPrimaryResponse - * @instance - */ - InitShardPrimaryResponse.prototype.events = $util.emptyArray; - - /** - * Creates a new InitShardPrimaryResponse instance using the specified properties. + * Creates a new RebuildVSchemaGraphResponse instance using the specified properties. * @function create - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static - * @param {vtctldata.IInitShardPrimaryResponse=} [properties] Properties to set - * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse instance + * @param {vtctldata.IRebuildVSchemaGraphResponse=} [properties] Properties to set + * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse instance */ - InitShardPrimaryResponse.create = function create(properties) { - return new InitShardPrimaryResponse(properties); + RebuildVSchemaGraphResponse.create = function create(properties) { + return new RebuildVSchemaGraphResponse(properties); }; /** - * Encodes the specified InitShardPrimaryResponse message. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. + * Encodes the specified RebuildVSchemaGraphResponse message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static - * @param {vtctldata.IInitShardPrimaryResponse} message InitShardPrimaryResponse message or plain object to encode + * @param {vtctldata.IRebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - InitShardPrimaryResponse.encode = function encode(message, writer) { + RebuildVSchemaGraphResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified InitShardPrimaryResponse message, length delimited. Does not implicitly {@link vtctldata.InitShardPrimaryResponse.verify|verify} messages. + * Encodes the specified RebuildVSchemaGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static - * @param {vtctldata.IInitShardPrimaryResponse} message InitShardPrimaryResponse message or plain object to encode + * @param {vtctldata.IRebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - InitShardPrimaryResponse.encodeDelimited = function encodeDelimited(message, writer) { + RebuildVSchemaGraphResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an InitShardPrimaryResponse message from the specified reader or buffer. + * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse + * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - InitShardPrimaryResponse.decode = function decode(reader, length) { + RebuildVSchemaGraphResponse.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.InitShardPrimaryResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RebuildVSchemaGraphResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.logutil.Event.decode(reader, reader.uint32())); - break; - } default: reader.skipType(tag & 7); break; @@ -124142,139 +130750,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an InitShardPrimaryResponse message from the specified reader or buffer, length delimited. + * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse + * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - InitShardPrimaryResponse.decodeDelimited = function decodeDelimited(reader) { + RebuildVSchemaGraphResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an InitShardPrimaryResponse message. + * Verifies a RebuildVSchemaGraphResponse message. * @function verify - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - InitShardPrimaryResponse.verify = function verify(message) { + RebuildVSchemaGraphResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.logutil.Event.verify(message.events[i]); - if (error) - return "events." + error; - } - } return null; }; /** - * Creates an InitShardPrimaryResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RebuildVSchemaGraphResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.InitShardPrimaryResponse} InitShardPrimaryResponse + * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse */ - InitShardPrimaryResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.InitShardPrimaryResponse) + RebuildVSchemaGraphResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RebuildVSchemaGraphResponse) return object; - let message = new $root.vtctldata.InitShardPrimaryResponse(); - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".vtctldata.InitShardPrimaryResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".vtctldata.InitShardPrimaryResponse.events: object expected"); - message.events[i] = $root.logutil.Event.fromObject(object.events[i]); - } - } - return message; + return new $root.vtctldata.RebuildVSchemaGraphResponse(); }; /** - * Creates a plain object from an InitShardPrimaryResponse message. Also converts values to other types if specified. + * Creates a plain object from a RebuildVSchemaGraphResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static - * @param {vtctldata.InitShardPrimaryResponse} message InitShardPrimaryResponse + * @param {vtctldata.RebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - InitShardPrimaryResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.events = []; - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.logutil.Event.toObject(message.events[j], options); - } - return object; + RebuildVSchemaGraphResponse.toObject = function toObject() { + return {}; }; /** - * Converts this InitShardPrimaryResponse to JSON. + * Converts this RebuildVSchemaGraphResponse to JSON. * @function toJSON - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @instance * @returns {Object.} JSON object */ - InitShardPrimaryResponse.prototype.toJSON = function toJSON() { + RebuildVSchemaGraphResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for InitShardPrimaryResponse + * Gets the default type url for RebuildVSchemaGraphResponse * @function getTypeUrl - * @memberof vtctldata.InitShardPrimaryResponse + * @memberof vtctldata.RebuildVSchemaGraphResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - InitShardPrimaryResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RebuildVSchemaGraphResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.InitShardPrimaryResponse"; + return typeUrlPrefix + "/vtctldata.RebuildVSchemaGraphResponse"; }; - return InitShardPrimaryResponse; + return RebuildVSchemaGraphResponse; })(); - vtctldata.PingTabletRequest = (function() { + vtctldata.RefreshStateRequest = (function() { /** - * Properties of a PingTabletRequest. + * Properties of a RefreshStateRequest. * @memberof vtctldata - * @interface IPingTabletRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] PingTabletRequest tablet_alias + * @interface IRefreshStateRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] RefreshStateRequest tablet_alias */ /** - * Constructs a new PingTabletRequest. + * Constructs a new RefreshStateRequest. * @memberof vtctldata - * @classdesc Represents a PingTabletRequest. - * @implements IPingTabletRequest + * @classdesc Represents a RefreshStateRequest. + * @implements IRefreshStateRequest * @constructor - * @param {vtctldata.IPingTabletRequest=} [properties] Properties to set + * @param {vtctldata.IRefreshStateRequest=} [properties] Properties to set */ - function PingTabletRequest(properties) { + function RefreshStateRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -124282,35 +130860,35 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * PingTabletRequest tablet_alias. + * RefreshStateRequest tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @instance */ - PingTabletRequest.prototype.tablet_alias = null; + RefreshStateRequest.prototype.tablet_alias = null; /** - * Creates a new PingTabletRequest instance using the specified properties. + * Creates a new RefreshStateRequest instance using the specified properties. * @function create - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static - * @param {vtctldata.IPingTabletRequest=} [properties] Properties to set - * @returns {vtctldata.PingTabletRequest} PingTabletRequest instance + * @param {vtctldata.IRefreshStateRequest=} [properties] Properties to set + * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest instance */ - PingTabletRequest.create = function create(properties) { - return new PingTabletRequest(properties); + RefreshStateRequest.create = function create(properties) { + return new RefreshStateRequest(properties); }; /** - * Encodes the specified PingTabletRequest message. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. + * Encodes the specified RefreshStateRequest message. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static - * @param {vtctldata.IPingTabletRequest} message PingTabletRequest message or plain object to encode + * @param {vtctldata.IRefreshStateRequest} message RefreshStateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PingTabletRequest.encode = function encode(message, writer) { + RefreshStateRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) @@ -124319,33 +130897,33 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Encodes the specified PingTabletRequest message, length delimited. Does not implicitly {@link vtctldata.PingTabletRequest.verify|verify} messages. + * Encodes the specified RefreshStateRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static - * @param {vtctldata.IPingTabletRequest} message PingTabletRequest message or plain object to encode + * @param {vtctldata.IRefreshStateRequest} message RefreshStateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PingTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { + RefreshStateRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PingTabletRequest message from the specified reader or buffer. + * Decodes a RefreshStateRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.PingTabletRequest} PingTabletRequest + * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PingTabletRequest.decode = function decode(reader, length) { + RefreshStateRequest.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.PingTabletRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RefreshStateRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -124362,30 +130940,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a PingTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.PingTabletRequest} PingTabletRequest + * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PingTabletRequest.decodeDelimited = function decodeDelimited(reader) { + RefreshStateRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PingTabletRequest message. + * Verifies a RefreshStateRequest message. * @function verify - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PingTabletRequest.verify = function verify(message) { + RefreshStateRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { @@ -124397,35 +130975,35 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Creates a PingTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.PingTabletRequest} PingTabletRequest + * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest */ - PingTabletRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.PingTabletRequest) + RefreshStateRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RefreshStateRequest) return object; - let message = new $root.vtctldata.PingTabletRequest(); + let message = new $root.vtctldata.RefreshStateRequest(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.PingTabletRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.RefreshStateRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } return message; }; /** - * Creates a plain object from a PingTabletRequest message. Also converts values to other types if specified. + * Creates a plain object from a RefreshStateRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static - * @param {vtctldata.PingTabletRequest} message PingTabletRequest + * @param {vtctldata.RefreshStateRequest} message RefreshStateRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PingTabletRequest.toObject = function toObject(message, options) { + RefreshStateRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -124437,51 +131015,51 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Converts this PingTabletRequest to JSON. + * Converts this RefreshStateRequest to JSON. * @function toJSON - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @instance * @returns {Object.} JSON object */ - PingTabletRequest.prototype.toJSON = function toJSON() { + RefreshStateRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PingTabletRequest + * Gets the default type url for RefreshStateRequest * @function getTypeUrl - * @memberof vtctldata.PingTabletRequest + * @memberof vtctldata.RefreshStateRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PingTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RefreshStateRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.PingTabletRequest"; + return typeUrlPrefix + "/vtctldata.RefreshStateRequest"; }; - return PingTabletRequest; + return RefreshStateRequest; })(); - vtctldata.PingTabletResponse = (function() { + vtctldata.RefreshStateResponse = (function() { /** - * Properties of a PingTabletResponse. + * Properties of a RefreshStateResponse. * @memberof vtctldata - * @interface IPingTabletResponse + * @interface IRefreshStateResponse */ /** - * Constructs a new PingTabletResponse. + * Constructs a new RefreshStateResponse. * @memberof vtctldata - * @classdesc Represents a PingTabletResponse. - * @implements IPingTabletResponse + * @classdesc Represents a RefreshStateResponse. + * @implements IRefreshStateResponse * @constructor - * @param {vtctldata.IPingTabletResponse=} [properties] Properties to set + * @param {vtctldata.IRefreshStateResponse=} [properties] Properties to set */ - function PingTabletResponse(properties) { + function RefreshStateResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -124489,60 +131067,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new PingTabletResponse instance using the specified properties. + * Creates a new RefreshStateResponse instance using the specified properties. * @function create - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static - * @param {vtctldata.IPingTabletResponse=} [properties] Properties to set - * @returns {vtctldata.PingTabletResponse} PingTabletResponse instance + * @param {vtctldata.IRefreshStateResponse=} [properties] Properties to set + * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse instance */ - PingTabletResponse.create = function create(properties) { - return new PingTabletResponse(properties); + RefreshStateResponse.create = function create(properties) { + return new RefreshStateResponse(properties); }; /** - * Encodes the specified PingTabletResponse message. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. + * Encodes the specified RefreshStateResponse message. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static - * @param {vtctldata.IPingTabletResponse} message PingTabletResponse message or plain object to encode + * @param {vtctldata.IRefreshStateResponse} message RefreshStateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PingTabletResponse.encode = function encode(message, writer) { + RefreshStateResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified PingTabletResponse message, length delimited. Does not implicitly {@link vtctldata.PingTabletResponse.verify|verify} messages. + * Encodes the specified RefreshStateResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static - * @param {vtctldata.IPingTabletResponse} message PingTabletResponse message or plain object to encode + * @param {vtctldata.IRefreshStateResponse} message RefreshStateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PingTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { + RefreshStateResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PingTabletResponse message from the specified reader or buffer. + * Decodes a RefreshStateResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.PingTabletResponse} PingTabletResponse + * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PingTabletResponse.decode = function decode(reader, length) { + RefreshStateResponse.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.PingTabletResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RefreshStateResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -124555,113 +131133,112 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a PingTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.PingTabletResponse} PingTabletResponse + * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PingTabletResponse.decodeDelimited = function decodeDelimited(reader) { + RefreshStateResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PingTabletResponse message. + * Verifies a RefreshStateResponse message. * @function verify - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PingTabletResponse.verify = function verify(message) { + RefreshStateResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a PingTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.PingTabletResponse} PingTabletResponse + * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse */ - PingTabletResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.PingTabletResponse) + RefreshStateResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RefreshStateResponse) return object; - return new $root.vtctldata.PingTabletResponse(); + return new $root.vtctldata.RefreshStateResponse(); }; /** - * Creates a plain object from a PingTabletResponse message. Also converts values to other types if specified. + * Creates a plain object from a RefreshStateResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static - * @param {vtctldata.PingTabletResponse} message PingTabletResponse + * @param {vtctldata.RefreshStateResponse} message RefreshStateResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PingTabletResponse.toObject = function toObject() { + RefreshStateResponse.toObject = function toObject() { return {}; }; /** - * Converts this PingTabletResponse to JSON. + * Converts this RefreshStateResponse to JSON. * @function toJSON - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @instance * @returns {Object.} JSON object */ - PingTabletResponse.prototype.toJSON = function toJSON() { + RefreshStateResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PingTabletResponse + * Gets the default type url for RefreshStateResponse * @function getTypeUrl - * @memberof vtctldata.PingTabletResponse + * @memberof vtctldata.RefreshStateResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PingTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RefreshStateResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.PingTabletResponse"; + return typeUrlPrefix + "/vtctldata.RefreshStateResponse"; }; - return PingTabletResponse; + return RefreshStateResponse; })(); - vtctldata.PlannedReparentShardRequest = (function() { + vtctldata.RefreshStateByShardRequest = (function() { /** - * Properties of a PlannedReparentShardRequest. + * Properties of a RefreshStateByShardRequest. * @memberof vtctldata - * @interface IPlannedReparentShardRequest - * @property {string|null} [keyspace] PlannedReparentShardRequest keyspace - * @property {string|null} [shard] PlannedReparentShardRequest shard - * @property {topodata.ITabletAlias|null} [new_primary] PlannedReparentShardRequest new_primary - * @property {topodata.ITabletAlias|null} [avoid_primary] PlannedReparentShardRequest avoid_primary - * @property {vttime.IDuration|null} [wait_replicas_timeout] PlannedReparentShardRequest wait_replicas_timeout + * @interface IRefreshStateByShardRequest + * @property {string|null} [keyspace] RefreshStateByShardRequest keyspace + * @property {string|null} [shard] RefreshStateByShardRequest shard + * @property {Array.|null} [cells] RefreshStateByShardRequest cells */ /** - * Constructs a new PlannedReparentShardRequest. + * Constructs a new RefreshStateByShardRequest. * @memberof vtctldata - * @classdesc Represents a PlannedReparentShardRequest. - * @implements IPlannedReparentShardRequest + * @classdesc Represents a RefreshStateByShardRequest. + * @implements IRefreshStateByShardRequest * @constructor - * @param {vtctldata.IPlannedReparentShardRequest=} [properties] Properties to set + * @param {vtctldata.IRefreshStateByShardRequest=} [properties] Properties to set */ - function PlannedReparentShardRequest(properties) { + function RefreshStateByShardRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -124669,110 +131246,91 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * PlannedReparentShardRequest keyspace. + * RefreshStateByShardRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @instance */ - PlannedReparentShardRequest.prototype.keyspace = ""; + RefreshStateByShardRequest.prototype.keyspace = ""; /** - * PlannedReparentShardRequest shard. + * RefreshStateByShardRequest shard. * @member {string} shard - * @memberof vtctldata.PlannedReparentShardRequest - * @instance - */ - PlannedReparentShardRequest.prototype.shard = ""; - - /** - * PlannedReparentShardRequest new_primary. - * @member {topodata.ITabletAlias|null|undefined} new_primary - * @memberof vtctldata.PlannedReparentShardRequest - * @instance - */ - PlannedReparentShardRequest.prototype.new_primary = null; - - /** - * PlannedReparentShardRequest avoid_primary. - * @member {topodata.ITabletAlias|null|undefined} avoid_primary - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @instance */ - PlannedReparentShardRequest.prototype.avoid_primary = null; + RefreshStateByShardRequest.prototype.shard = ""; /** - * PlannedReparentShardRequest wait_replicas_timeout. - * @member {vttime.IDuration|null|undefined} wait_replicas_timeout - * @memberof vtctldata.PlannedReparentShardRequest + * RefreshStateByShardRequest cells. + * @member {Array.} cells + * @memberof vtctldata.RefreshStateByShardRequest * @instance */ - PlannedReparentShardRequest.prototype.wait_replicas_timeout = null; + RefreshStateByShardRequest.prototype.cells = $util.emptyArray; /** - * Creates a new PlannedReparentShardRequest instance using the specified properties. + * Creates a new RefreshStateByShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static - * @param {vtctldata.IPlannedReparentShardRequest=} [properties] Properties to set - * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest instance + * @param {vtctldata.IRefreshStateByShardRequest=} [properties] Properties to set + * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest instance */ - PlannedReparentShardRequest.create = function create(properties) { - return new PlannedReparentShardRequest(properties); + RefreshStateByShardRequest.create = function create(properties) { + return new RefreshStateByShardRequest(properties); }; /** - * Encodes the specified PlannedReparentShardRequest message. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. + * Encodes the specified RefreshStateByShardRequest message. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static - * @param {vtctldata.IPlannedReparentShardRequest} message PlannedReparentShardRequest message or plain object to encode + * @param {vtctldata.IRefreshStateByShardRequest} message RefreshStateByShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PlannedReparentShardRequest.encode = function encode(message, writer) { + RefreshStateByShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) - $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.avoid_primary != null && Object.hasOwnProperty.call(message, "avoid_primary")) - $root.topodata.TabletAlias.encode(message.avoid_primary, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); - if (message.wait_replicas_timeout != null && Object.hasOwnProperty.call(message, "wait_replicas_timeout")) - $root.vttime.Duration.encode(message.wait_replicas_timeout, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); return writer; }; /** - * Encodes the specified PlannedReparentShardRequest message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardRequest.verify|verify} messages. + * Encodes the specified RefreshStateByShardRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static - * @param {vtctldata.IPlannedReparentShardRequest} message PlannedReparentShardRequest message or plain object to encode + * @param {vtctldata.IRefreshStateByShardRequest} message RefreshStateByShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PlannedReparentShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + RefreshStateByShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PlannedReparentShardRequest message from the specified reader or buffer. + * Decodes a RefreshStateByShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest + * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PlannedReparentShardRequest.decode = function decode(reader, length) { + RefreshStateByShardRequest.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.PlannedReparentShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RefreshStateByShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -124785,15 +131343,9 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 3: { - message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - message.avoid_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 5: { - message.wait_replicas_timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } default: @@ -124805,30 +131357,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a PlannedReparentShardRequest message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateByShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest + * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PlannedReparentShardRequest.decodeDelimited = function decodeDelimited(reader) { + RefreshStateByShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PlannedReparentShardRequest message. + * Verifies a RefreshStateByShardRequest message. * @function verify - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PlannedReparentShardRequest.verify = function verify(message) { + RefreshStateByShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -124837,142 +131389,121 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) { - let error = $root.topodata.TabletAlias.verify(message.new_primary); - if (error) - return "new_primary." + error; - } - if (message.avoid_primary != null && message.hasOwnProperty("avoid_primary")) { - let error = $root.topodata.TabletAlias.verify(message.avoid_primary); - if (error) - return "avoid_primary." + error; - } - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) { - let error = $root.vttime.Duration.verify(message.wait_replicas_timeout); - if (error) - return "wait_replicas_timeout." + error; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; } return null; }; /** - * Creates a PlannedReparentShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateByShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.PlannedReparentShardRequest} PlannedReparentShardRequest + * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest */ - PlannedReparentShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.PlannedReparentShardRequest) + RefreshStateByShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RefreshStateByShardRequest) return object; - let message = new $root.vtctldata.PlannedReparentShardRequest(); + let message = new $root.vtctldata.RefreshStateByShardRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - if (object.new_primary != null) { - if (typeof object.new_primary !== "object") - throw TypeError(".vtctldata.PlannedReparentShardRequest.new_primary: object expected"); - message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); - } - if (object.avoid_primary != null) { - if (typeof object.avoid_primary !== "object") - throw TypeError(".vtctldata.PlannedReparentShardRequest.avoid_primary: object expected"); - message.avoid_primary = $root.topodata.TabletAlias.fromObject(object.avoid_primary); - } - if (object.wait_replicas_timeout != null) { - if (typeof object.wait_replicas_timeout !== "object") - throw TypeError(".vtctldata.PlannedReparentShardRequest.wait_replicas_timeout: object expected"); - message.wait_replicas_timeout = $root.vttime.Duration.fromObject(object.wait_replicas_timeout); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.RefreshStateByShardRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); } return message; }; /** - * Creates a plain object from a PlannedReparentShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a RefreshStateByShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static - * @param {vtctldata.PlannedReparentShardRequest} message PlannedReparentShardRequest + * @param {vtctldata.RefreshStateByShardRequest} message RefreshStateByShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PlannedReparentShardRequest.toObject = function toObject(message, options) { + RefreshStateByShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.cells = []; if (options.defaults) { object.keyspace = ""; object.shard = ""; - object.new_primary = null; - object.avoid_primary = null; - object.wait_replicas_timeout = null; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) - object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); - if (message.avoid_primary != null && message.hasOwnProperty("avoid_primary")) - object.avoid_primary = $root.topodata.TabletAlias.toObject(message.avoid_primary, options); - if (message.wait_replicas_timeout != null && message.hasOwnProperty("wait_replicas_timeout")) - object.wait_replicas_timeout = $root.vttime.Duration.toObject(message.wait_replicas_timeout, options); + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } return object; }; /** - * Converts this PlannedReparentShardRequest to JSON. + * Converts this RefreshStateByShardRequest to JSON. * @function toJSON - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @instance * @returns {Object.} JSON object */ - PlannedReparentShardRequest.prototype.toJSON = function toJSON() { + RefreshStateByShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PlannedReparentShardRequest + * Gets the default type url for RefreshStateByShardRequest * @function getTypeUrl - * @memberof vtctldata.PlannedReparentShardRequest + * @memberof vtctldata.RefreshStateByShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PlannedReparentShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RefreshStateByShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.PlannedReparentShardRequest"; + return typeUrlPrefix + "/vtctldata.RefreshStateByShardRequest"; }; - return PlannedReparentShardRequest; + return RefreshStateByShardRequest; })(); - vtctldata.PlannedReparentShardResponse = (function() { + vtctldata.RefreshStateByShardResponse = (function() { /** - * Properties of a PlannedReparentShardResponse. + * Properties of a RefreshStateByShardResponse. * @memberof vtctldata - * @interface IPlannedReparentShardResponse - * @property {string|null} [keyspace] PlannedReparentShardResponse keyspace - * @property {string|null} [shard] PlannedReparentShardResponse shard - * @property {topodata.ITabletAlias|null} [promoted_primary] PlannedReparentShardResponse promoted_primary - * @property {Array.|null} [events] PlannedReparentShardResponse events + * @interface IRefreshStateByShardResponse + * @property {boolean|null} [is_partial_refresh] RefreshStateByShardResponse is_partial_refresh + * @property {string|null} [partial_refresh_details] RefreshStateByShardResponse partial_refresh_details */ /** - * Constructs a new PlannedReparentShardResponse. + * Constructs a new RefreshStateByShardResponse. * @memberof vtctldata - * @classdesc Represents a PlannedReparentShardResponse. - * @implements IPlannedReparentShardResponse + * @classdesc Represents a RefreshStateByShardResponse. + * @implements IRefreshStateByShardResponse * @constructor - * @param {vtctldata.IPlannedReparentShardResponse=} [properties] Properties to set + * @param {vtctldata.IRefreshStateByShardResponse=} [properties] Properties to set */ - function PlannedReparentShardResponse(properties) { - this.events = []; + function RefreshStateByShardResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -124980,120 +131511,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * PlannedReparentShardResponse keyspace. - * @member {string} keyspace - * @memberof vtctldata.PlannedReparentShardResponse - * @instance - */ - PlannedReparentShardResponse.prototype.keyspace = ""; - - /** - * PlannedReparentShardResponse shard. - * @member {string} shard - * @memberof vtctldata.PlannedReparentShardResponse - * @instance - */ - PlannedReparentShardResponse.prototype.shard = ""; - - /** - * PlannedReparentShardResponse promoted_primary. - * @member {topodata.ITabletAlias|null|undefined} promoted_primary - * @memberof vtctldata.PlannedReparentShardResponse + * RefreshStateByShardResponse is_partial_refresh. + * @member {boolean} is_partial_refresh + * @memberof vtctldata.RefreshStateByShardResponse * @instance */ - PlannedReparentShardResponse.prototype.promoted_primary = null; + RefreshStateByShardResponse.prototype.is_partial_refresh = false; /** - * PlannedReparentShardResponse events. - * @member {Array.} events - * @memberof vtctldata.PlannedReparentShardResponse + * RefreshStateByShardResponse partial_refresh_details. + * @member {string} partial_refresh_details + * @memberof vtctldata.RefreshStateByShardResponse * @instance */ - PlannedReparentShardResponse.prototype.events = $util.emptyArray; + RefreshStateByShardResponse.prototype.partial_refresh_details = ""; /** - * Creates a new PlannedReparentShardResponse instance using the specified properties. + * Creates a new RefreshStateByShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static - * @param {vtctldata.IPlannedReparentShardResponse=} [properties] Properties to set - * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse instance + * @param {vtctldata.IRefreshStateByShardResponse=} [properties] Properties to set + * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse instance */ - PlannedReparentShardResponse.create = function create(properties) { - return new PlannedReparentShardResponse(properties); + RefreshStateByShardResponse.create = function create(properties) { + return new RefreshStateByShardResponse(properties); }; /** - * Encodes the specified PlannedReparentShardResponse message. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. + * Encodes the specified RefreshStateByShardResponse message. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static - * @param {vtctldata.IPlannedReparentShardResponse} message PlannedReparentShardResponse message or plain object to encode + * @param {vtctldata.IRefreshStateByShardResponse} message RefreshStateByShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PlannedReparentShardResponse.encode = function encode(message, writer) { + RefreshStateByShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.promoted_primary != null && Object.hasOwnProperty.call(message, "promoted_primary")) - $root.topodata.TabletAlias.encode(message.promoted_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.is_partial_refresh != null && Object.hasOwnProperty.call(message, "is_partial_refresh")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.is_partial_refresh); + if (message.partial_refresh_details != null && Object.hasOwnProperty.call(message, "partial_refresh_details")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.partial_refresh_details); return writer; }; /** - * Encodes the specified PlannedReparentShardResponse message, length delimited. Does not implicitly {@link vtctldata.PlannedReparentShardResponse.verify|verify} messages. + * Encodes the specified RefreshStateByShardResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static - * @param {vtctldata.IPlannedReparentShardResponse} message PlannedReparentShardResponse message or plain object to encode + * @param {vtctldata.IRefreshStateByShardResponse} message RefreshStateByShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - PlannedReparentShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + RefreshStateByShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a PlannedReparentShardResponse message from the specified reader or buffer. + * Decodes a RefreshStateByShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse + * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PlannedReparentShardResponse.decode = function decode(reader, length) { + RefreshStateByShardResponse.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.PlannedReparentShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RefreshStateByShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.is_partial_refresh = reader.bool(); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.promoted_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + message.partial_refresh_details = reader.string(); break; } default: @@ -125105,173 +131605,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a PlannedReparentShardResponse message from the specified reader or buffer, length delimited. + * Decodes a RefreshStateByShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse + * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - PlannedReparentShardResponse.decodeDelimited = function decodeDelimited(reader) { + RefreshStateByShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a PlannedReparentShardResponse message. + * Verifies a RefreshStateByShardResponse message. * @function verify - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - PlannedReparentShardResponse.verify = function verify(message) { + RefreshStateByShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) { - let error = $root.topodata.TabletAlias.verify(message.promoted_primary); - if (error) - return "promoted_primary." + error; - } - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.logutil.Event.verify(message.events[i]); - if (error) - return "events." + error; - } - } + if (message.is_partial_refresh != null && message.hasOwnProperty("is_partial_refresh")) + if (typeof message.is_partial_refresh !== "boolean") + return "is_partial_refresh: boolean expected"; + if (message.partial_refresh_details != null && message.hasOwnProperty("partial_refresh_details")) + if (!$util.isString(message.partial_refresh_details)) + return "partial_refresh_details: string expected"; return null; }; /** - * Creates a PlannedReparentShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RefreshStateByShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.PlannedReparentShardResponse} PlannedReparentShardResponse + * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse */ - PlannedReparentShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.PlannedReparentShardResponse) + RefreshStateByShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RefreshStateByShardResponse) return object; - let message = new $root.vtctldata.PlannedReparentShardResponse(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.promoted_primary != null) { - if (typeof object.promoted_primary !== "object") - throw TypeError(".vtctldata.PlannedReparentShardResponse.promoted_primary: object expected"); - message.promoted_primary = $root.topodata.TabletAlias.fromObject(object.promoted_primary); - } - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".vtctldata.PlannedReparentShardResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".vtctldata.PlannedReparentShardResponse.events: object expected"); - message.events[i] = $root.logutil.Event.fromObject(object.events[i]); - } - } + let message = new $root.vtctldata.RefreshStateByShardResponse(); + if (object.is_partial_refresh != null) + message.is_partial_refresh = Boolean(object.is_partial_refresh); + if (object.partial_refresh_details != null) + message.partial_refresh_details = String(object.partial_refresh_details); return message; }; /** - * Creates a plain object from a PlannedReparentShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a RefreshStateByShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static - * @param {vtctldata.PlannedReparentShardResponse} message PlannedReparentShardResponse + * @param {vtctldata.RefreshStateByShardResponse} message RefreshStateByShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - PlannedReparentShardResponse.toObject = function toObject(message, options) { + RefreshStateByShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.events = []; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.promoted_primary = null; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.promoted_primary != null && message.hasOwnProperty("promoted_primary")) - object.promoted_primary = $root.topodata.TabletAlias.toObject(message.promoted_primary, options); - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + object.is_partial_refresh = false; + object.partial_refresh_details = ""; } + if (message.is_partial_refresh != null && message.hasOwnProperty("is_partial_refresh")) + object.is_partial_refresh = message.is_partial_refresh; + if (message.partial_refresh_details != null && message.hasOwnProperty("partial_refresh_details")) + object.partial_refresh_details = message.partial_refresh_details; return object; }; /** - * Converts this PlannedReparentShardResponse to JSON. + * Converts this RefreshStateByShardResponse to JSON. * @function toJSON - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @instance * @returns {Object.} JSON object */ - PlannedReparentShardResponse.prototype.toJSON = function toJSON() { + RefreshStateByShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for PlannedReparentShardResponse + * Gets the default type url for RefreshStateByShardResponse * @function getTypeUrl - * @memberof vtctldata.PlannedReparentShardResponse + * @memberof vtctldata.RefreshStateByShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - PlannedReparentShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RefreshStateByShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.PlannedReparentShardResponse"; + return typeUrlPrefix + "/vtctldata.RefreshStateByShardResponse"; }; - return PlannedReparentShardResponse; + return RefreshStateByShardResponse; })(); - vtctldata.RebuildKeyspaceGraphRequest = (function() { + vtctldata.ReloadSchemaRequest = (function() { /** - * Properties of a RebuildKeyspaceGraphRequest. + * Properties of a ReloadSchemaRequest. * @memberof vtctldata - * @interface IRebuildKeyspaceGraphRequest - * @property {string|null} [keyspace] RebuildKeyspaceGraphRequest keyspace - * @property {Array.|null} [cells] RebuildKeyspaceGraphRequest cells - * @property {boolean|null} [allow_partial] RebuildKeyspaceGraphRequest allow_partial + * @interface IReloadSchemaRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] ReloadSchemaRequest tablet_alias */ /** - * Constructs a new RebuildKeyspaceGraphRequest. + * Constructs a new ReloadSchemaRequest. * @memberof vtctldata - * @classdesc Represents a RebuildKeyspaceGraphRequest. - * @implements IRebuildKeyspaceGraphRequest + * @classdesc Represents a ReloadSchemaRequest. + * @implements IReloadSchemaRequest * @constructor - * @param {vtctldata.IRebuildKeyspaceGraphRequest=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaRequest=} [properties] Properties to set */ - function RebuildKeyspaceGraphRequest(properties) { - this.cells = []; + function ReloadSchemaRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -125279,106 +131737,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RebuildKeyspaceGraphRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.RebuildKeyspaceGraphRequest - * @instance - */ - RebuildKeyspaceGraphRequest.prototype.keyspace = ""; - - /** - * RebuildKeyspaceGraphRequest cells. - * @member {Array.} cells - * @memberof vtctldata.RebuildKeyspaceGraphRequest - * @instance - */ - RebuildKeyspaceGraphRequest.prototype.cells = $util.emptyArray; - - /** - * RebuildKeyspaceGraphRequest allow_partial. - * @member {boolean} allow_partial - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * ReloadSchemaRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ReloadSchemaRequest * @instance */ - RebuildKeyspaceGraphRequest.prototype.allow_partial = false; + ReloadSchemaRequest.prototype.tablet_alias = null; /** - * Creates a new RebuildKeyspaceGraphRequest instance using the specified properties. + * Creates a new ReloadSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static - * @param {vtctldata.IRebuildKeyspaceGraphRequest=} [properties] Properties to set - * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest instance + * @param {vtctldata.IReloadSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest instance */ - RebuildKeyspaceGraphRequest.create = function create(properties) { - return new RebuildKeyspaceGraphRequest(properties); + ReloadSchemaRequest.create = function create(properties) { + return new ReloadSchemaRequest(properties); }; /** - * Encodes the specified RebuildKeyspaceGraphRequest message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaRequest message. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static - * @param {vtctldata.IRebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaRequest} message ReloadSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildKeyspaceGraphRequest.encode = function encode(message, writer) { + ReloadSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); - if (message.allow_partial != null && Object.hasOwnProperty.call(message, "allow_partial")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.allow_partial); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RebuildKeyspaceGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static - * @param {vtctldata.IRebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaRequest} message ReloadSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildKeyspaceGraphRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest + * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildKeyspaceGraphRequest.decode = function decode(reader, length) { + ReloadSchemaRequest.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.RebuildKeyspaceGraphRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); - break; - } - case 3: { - message.allow_partial = reader.bool(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -125390,151 +131817,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RebuildKeyspaceGraphRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest + * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildKeyspaceGraphRequest.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RebuildKeyspaceGraphRequest message. + * Verifies a ReloadSchemaRequest message. * @function verify - * @memberof vtctldata.RebuildKeyspaceGraphRequest - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - RebuildKeyspaceGraphRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; + * @memberof vtctldata.ReloadSchemaRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReloadSchemaRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; } - if (message.allow_partial != null && message.hasOwnProperty("allow_partial")) - if (typeof message.allow_partial !== "boolean") - return "allow_partial: boolean expected"; return null; }; /** - * Creates a RebuildKeyspaceGraphRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RebuildKeyspaceGraphRequest} RebuildKeyspaceGraphRequest + * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest */ - RebuildKeyspaceGraphRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RebuildKeyspaceGraphRequest) + ReloadSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaRequest) return object; - let message = new $root.vtctldata.RebuildKeyspaceGraphRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.RebuildKeyspaceGraphRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); + let message = new $root.vtctldata.ReloadSchemaRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ReloadSchemaRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.allow_partial != null) - message.allow_partial = Boolean(object.allow_partial); return message; }; /** - * Creates a plain object from a RebuildKeyspaceGraphRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static - * @param {vtctldata.RebuildKeyspaceGraphRequest} message RebuildKeyspaceGraphRequest + * @param {vtctldata.ReloadSchemaRequest} message ReloadSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RebuildKeyspaceGraphRequest.toObject = function toObject(message, options) { + ReloadSchemaRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; - if (options.defaults) { - object.keyspace = ""; - object.allow_partial = false; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } - if (message.allow_partial != null && message.hasOwnProperty("allow_partial")) - object.allow_partial = message.allow_partial; + if (options.defaults) + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this RebuildKeyspaceGraphRequest to JSON. + * Converts this ReloadSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @instance * @returns {Object.} JSON object */ - RebuildKeyspaceGraphRequest.prototype.toJSON = function toJSON() { + ReloadSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RebuildKeyspaceGraphRequest + * Gets the default type url for ReloadSchemaRequest * @function getTypeUrl - * @memberof vtctldata.RebuildKeyspaceGraphRequest + * @memberof vtctldata.ReloadSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RebuildKeyspaceGraphRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RebuildKeyspaceGraphRequest"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaRequest"; }; - return RebuildKeyspaceGraphRequest; + return ReloadSchemaRequest; })(); - vtctldata.RebuildKeyspaceGraphResponse = (function() { + vtctldata.ReloadSchemaResponse = (function() { /** - * Properties of a RebuildKeyspaceGraphResponse. + * Properties of a ReloadSchemaResponse. * @memberof vtctldata - * @interface IRebuildKeyspaceGraphResponse + * @interface IReloadSchemaResponse */ /** - * Constructs a new RebuildKeyspaceGraphResponse. + * Constructs a new ReloadSchemaResponse. * @memberof vtctldata - * @classdesc Represents a RebuildKeyspaceGraphResponse. - * @implements IRebuildKeyspaceGraphResponse + * @classdesc Represents a ReloadSchemaResponse. + * @implements IReloadSchemaResponse * @constructor - * @param {vtctldata.IRebuildKeyspaceGraphResponse=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaResponse=} [properties] Properties to set */ - function RebuildKeyspaceGraphResponse(properties) { + function ReloadSchemaResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -125542,60 +131944,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RebuildKeyspaceGraphResponse instance using the specified properties. + * Creates a new ReloadSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static - * @param {vtctldata.IRebuildKeyspaceGraphResponse=} [properties] Properties to set - * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse instance + * @param {vtctldata.IReloadSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse instance */ - RebuildKeyspaceGraphResponse.create = function create(properties) { - return new RebuildKeyspaceGraphResponse(properties); + ReloadSchemaResponse.create = function create(properties) { + return new ReloadSchemaResponse(properties); }; /** - * Encodes the specified RebuildKeyspaceGraphResponse message. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaResponse message. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static - * @param {vtctldata.IRebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaResponse} message ReloadSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildKeyspaceGraphResponse.encode = function encode(message, writer) { + ReloadSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified RebuildKeyspaceGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildKeyspaceGraphResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static - * @param {vtctldata.IRebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaResponse} message ReloadSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildKeyspaceGraphResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse + * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildKeyspaceGraphResponse.decode = function decode(reader, length) { + ReloadSchemaResponse.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.RebuildKeyspaceGraphResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -125608,110 +132010,112 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RebuildKeyspaceGraphResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse + * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildKeyspaceGraphResponse.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RebuildKeyspaceGraphResponse message. + * Verifies a ReloadSchemaResponse message. * @function verify - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RebuildKeyspaceGraphResponse.verify = function verify(message) { + ReloadSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a RebuildKeyspaceGraphResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RebuildKeyspaceGraphResponse} RebuildKeyspaceGraphResponse + * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse */ - RebuildKeyspaceGraphResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RebuildKeyspaceGraphResponse) + ReloadSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaResponse) return object; - return new $root.vtctldata.RebuildKeyspaceGraphResponse(); + return new $root.vtctldata.ReloadSchemaResponse(); }; /** - * Creates a plain object from a RebuildKeyspaceGraphResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static - * @param {vtctldata.RebuildKeyspaceGraphResponse} message RebuildKeyspaceGraphResponse + * @param {vtctldata.ReloadSchemaResponse} message ReloadSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RebuildKeyspaceGraphResponse.toObject = function toObject() { + ReloadSchemaResponse.toObject = function toObject() { return {}; }; /** - * Converts this RebuildKeyspaceGraphResponse to JSON. + * Converts this ReloadSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @instance * @returns {Object.} JSON object */ - RebuildKeyspaceGraphResponse.prototype.toJSON = function toJSON() { + ReloadSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RebuildKeyspaceGraphResponse + * Gets the default type url for ReloadSchemaResponse * @function getTypeUrl - * @memberof vtctldata.RebuildKeyspaceGraphResponse + * @memberof vtctldata.ReloadSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RebuildKeyspaceGraphResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RebuildKeyspaceGraphResponse"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaResponse"; }; - return RebuildKeyspaceGraphResponse; + return ReloadSchemaResponse; })(); - vtctldata.RebuildVSchemaGraphRequest = (function() { + vtctldata.ReloadSchemaKeyspaceRequest = (function() { /** - * Properties of a RebuildVSchemaGraphRequest. + * Properties of a ReloadSchemaKeyspaceRequest. * @memberof vtctldata - * @interface IRebuildVSchemaGraphRequest - * @property {Array.|null} [cells] RebuildVSchemaGraphRequest cells + * @interface IReloadSchemaKeyspaceRequest + * @property {string|null} [keyspace] ReloadSchemaKeyspaceRequest keyspace + * @property {string|null} [wait_position] ReloadSchemaKeyspaceRequest wait_position + * @property {boolean|null} [include_primary] ReloadSchemaKeyspaceRequest include_primary + * @property {number|null} [concurrency] ReloadSchemaKeyspaceRequest concurrency */ /** - * Constructs a new RebuildVSchemaGraphRequest. + * Constructs a new ReloadSchemaKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a RebuildVSchemaGraphRequest. - * @implements IRebuildVSchemaGraphRequest + * @classdesc Represents a ReloadSchemaKeyspaceRequest. + * @implements IReloadSchemaKeyspaceRequest * @constructor - * @param {vtctldata.IRebuildVSchemaGraphRequest=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaKeyspaceRequest=} [properties] Properties to set */ - function RebuildVSchemaGraphRequest(properties) { - this.cells = []; + function ReloadSchemaKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -125719,78 +132123,117 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RebuildVSchemaGraphRequest cells. - * @member {Array.} cells - * @memberof vtctldata.RebuildVSchemaGraphRequest + * ReloadSchemaKeyspaceRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @instance */ - RebuildVSchemaGraphRequest.prototype.cells = $util.emptyArray; + ReloadSchemaKeyspaceRequest.prototype.keyspace = ""; /** - * Creates a new RebuildVSchemaGraphRequest instance using the specified properties. + * ReloadSchemaKeyspaceRequest wait_position. + * @member {string} wait_position + * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @instance + */ + ReloadSchemaKeyspaceRequest.prototype.wait_position = ""; + + /** + * ReloadSchemaKeyspaceRequest include_primary. + * @member {boolean} include_primary + * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @instance + */ + ReloadSchemaKeyspaceRequest.prototype.include_primary = false; + + /** + * ReloadSchemaKeyspaceRequest concurrency. + * @member {number} concurrency + * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @instance + */ + ReloadSchemaKeyspaceRequest.prototype.concurrency = 0; + + /** + * Creates a new ReloadSchemaKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static - * @param {vtctldata.IRebuildVSchemaGraphRequest=} [properties] Properties to set - * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest instance + * @param {vtctldata.IReloadSchemaKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest instance */ - RebuildVSchemaGraphRequest.create = function create(properties) { - return new RebuildVSchemaGraphRequest(properties); + ReloadSchemaKeyspaceRequest.create = function create(properties) { + return new ReloadSchemaKeyspaceRequest(properties); }; /** - * Encodes the specified RebuildVSchemaGraphRequest message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static - * @param {vtctldata.IRebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildVSchemaGraphRequest.encode = function encode(message, writer) { + ReloadSchemaKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.cells[i]); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.wait_position != null && Object.hasOwnProperty.call(message, "wait_position")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.wait_position); + if (message.include_primary != null && Object.hasOwnProperty.call(message, "include_primary")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.include_primary); + if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) + writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.concurrency); return writer; }; /** - * Encodes the specified RebuildVSchemaGraphRequest message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static - * @param {vtctldata.IRebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildVSchemaGraphRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest + * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildVSchemaGraphRequest.decode = function decode(reader, length) { + ReloadSchemaKeyspaceRequest.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.RebuildVSchemaGraphRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.wait_position = reader.string(); + break; + } + case 3: { + message.include_primary = reader.bool(); + break; + } + case 4: { + message.concurrency = reader.uint32(); break; } default: @@ -125802,133 +132245,148 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RebuildVSchemaGraphRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest + * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildVSchemaGraphRequest.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RebuildVSchemaGraphRequest message. + * Verifies a ReloadSchemaKeyspaceRequest message. * @function verify - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RebuildVSchemaGraphRequest.verify = function verify(message) { + ReloadSchemaKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.wait_position != null && message.hasOwnProperty("wait_position")) + if (!$util.isString(message.wait_position)) + return "wait_position: string expected"; + if (message.include_primary != null && message.hasOwnProperty("include_primary")) + if (typeof message.include_primary !== "boolean") + return "include_primary: boolean expected"; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (!$util.isInteger(message.concurrency)) + return "concurrency: integer expected"; return null; }; /** - * Creates a RebuildVSchemaGraphRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RebuildVSchemaGraphRequest} RebuildVSchemaGraphRequest + * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest */ - RebuildVSchemaGraphRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RebuildVSchemaGraphRequest) + ReloadSchemaKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaKeyspaceRequest) return object; - let message = new $root.vtctldata.RebuildVSchemaGraphRequest(); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.RebuildVSchemaGraphRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } + let message = new $root.vtctldata.ReloadSchemaKeyspaceRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.wait_position != null) + message.wait_position = String(object.wait_position); + if (object.include_primary != null) + message.include_primary = Boolean(object.include_primary); + if (object.concurrency != null) + message.concurrency = object.concurrency >>> 0; return message; }; /** - * Creates a plain object from a RebuildVSchemaGraphRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static - * @param {vtctldata.RebuildVSchemaGraphRequest} message RebuildVSchemaGraphRequest + * @param {vtctldata.ReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RebuildVSchemaGraphRequest.toObject = function toObject(message, options) { + ReloadSchemaKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; + if (options.defaults) { + object.keyspace = ""; + object.wait_position = ""; + object.include_primary = false; + object.concurrency = 0; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.wait_position != null && message.hasOwnProperty("wait_position")) + object.wait_position = message.wait_position; + if (message.include_primary != null && message.hasOwnProperty("include_primary")) + object.include_primary = message.include_primary; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + object.concurrency = message.concurrency; return object; }; /** - * Converts this RebuildVSchemaGraphRequest to JSON. + * Converts this ReloadSchemaKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @instance * @returns {Object.} JSON object */ - RebuildVSchemaGraphRequest.prototype.toJSON = function toJSON() { + ReloadSchemaKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RebuildVSchemaGraphRequest + * Gets the default type url for ReloadSchemaKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.RebuildVSchemaGraphRequest + * @memberof vtctldata.ReloadSchemaKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RebuildVSchemaGraphRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RebuildVSchemaGraphRequest"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaKeyspaceRequest"; }; - return RebuildVSchemaGraphRequest; + return ReloadSchemaKeyspaceRequest; })(); - vtctldata.RebuildVSchemaGraphResponse = (function() { + vtctldata.ReloadSchemaKeyspaceResponse = (function() { /** - * Properties of a RebuildVSchemaGraphResponse. + * Properties of a ReloadSchemaKeyspaceResponse. * @memberof vtctldata - * @interface IRebuildVSchemaGraphResponse + * @interface IReloadSchemaKeyspaceResponse + * @property {Array.|null} [events] ReloadSchemaKeyspaceResponse events */ /** - * Constructs a new RebuildVSchemaGraphResponse. + * Constructs a new ReloadSchemaKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a RebuildVSchemaGraphResponse. - * @implements IRebuildVSchemaGraphResponse + * @classdesc Represents a ReloadSchemaKeyspaceResponse. + * @implements IReloadSchemaKeyspaceResponse * @constructor - * @param {vtctldata.IRebuildVSchemaGraphResponse=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaKeyspaceResponse=} [properties] Properties to set */ - function RebuildVSchemaGraphResponse(properties) { + function ReloadSchemaKeyspaceResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -125936,63 +132394,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RebuildVSchemaGraphResponse instance using the specified properties. + * ReloadSchemaKeyspaceResponse events. + * @member {Array.} events + * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @instance + */ + ReloadSchemaKeyspaceResponse.prototype.events = $util.emptyArray; + + /** + * Creates a new ReloadSchemaKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static - * @param {vtctldata.IRebuildVSchemaGraphResponse=} [properties] Properties to set - * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse instance + * @param {vtctldata.IReloadSchemaKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse instance */ - RebuildVSchemaGraphResponse.create = function create(properties) { - return new RebuildVSchemaGraphResponse(properties); + ReloadSchemaKeyspaceResponse.create = function create(properties) { + return new ReloadSchemaKeyspaceResponse(properties); }; /** - * Encodes the specified RebuildVSchemaGraphResponse message. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static - * @param {vtctldata.IRebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildVSchemaGraphResponse.encode = function encode(message, writer) { + ReloadSchemaKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RebuildVSchemaGraphResponse message, length delimited. Does not implicitly {@link vtctldata.RebuildVSchemaGraphResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static - * @param {vtctldata.IRebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RebuildVSchemaGraphResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse + * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildVSchemaGraphResponse.decode = function decode(reader, length) { + ReloadSchemaKeyspaceResponse.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.RebuildVSchemaGraphResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaKeyspaceResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + break; + } default: reader.skipType(tag & 7); break; @@ -126002,109 +132477,143 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RebuildVSchemaGraphResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse + * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RebuildVSchemaGraphResponse.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RebuildVSchemaGraphResponse message. + * Verifies a ReloadSchemaKeyspaceResponse message. * @function verify - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RebuildVSchemaGraphResponse.verify = function verify(message) { + ReloadSchemaKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.logutil.Event.verify(message.events[i]); + if (error) + return "events." + error; + } + } return null; }; /** - * Creates a RebuildVSchemaGraphResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RebuildVSchemaGraphResponse} RebuildVSchemaGraphResponse + * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse */ - RebuildVSchemaGraphResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RebuildVSchemaGraphResponse) + ReloadSchemaKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaKeyspaceResponse) return object; - return new $root.vtctldata.RebuildVSchemaGraphResponse(); + let message = new $root.vtctldata.ReloadSchemaKeyspaceResponse(); + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".vtctldata.ReloadSchemaKeyspaceResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".vtctldata.ReloadSchemaKeyspaceResponse.events: object expected"); + message.events[i] = $root.logutil.Event.fromObject(object.events[i]); + } + } + return message; }; /** - * Creates a plain object from a RebuildVSchemaGraphResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static - * @param {vtctldata.RebuildVSchemaGraphResponse} message RebuildVSchemaGraphResponse + * @param {vtctldata.ReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RebuildVSchemaGraphResponse.toObject = function toObject() { - return {}; + ReloadSchemaKeyspaceResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.events = []; + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + } + return object; }; /** - * Converts this RebuildVSchemaGraphResponse to JSON. + * Converts this ReloadSchemaKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @instance * @returns {Object.} JSON object */ - RebuildVSchemaGraphResponse.prototype.toJSON = function toJSON() { + ReloadSchemaKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RebuildVSchemaGraphResponse + * Gets the default type url for ReloadSchemaKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.RebuildVSchemaGraphResponse + * @memberof vtctldata.ReloadSchemaKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RebuildVSchemaGraphResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RebuildVSchemaGraphResponse"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaKeyspaceResponse"; }; - return RebuildVSchemaGraphResponse; + return ReloadSchemaKeyspaceResponse; })(); - vtctldata.RefreshStateRequest = (function() { + vtctldata.ReloadSchemaShardRequest = (function() { /** - * Properties of a RefreshStateRequest. + * Properties of a ReloadSchemaShardRequest. * @memberof vtctldata - * @interface IRefreshStateRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] RefreshStateRequest tablet_alias + * @interface IReloadSchemaShardRequest + * @property {string|null} [keyspace] ReloadSchemaShardRequest keyspace + * @property {string|null} [shard] ReloadSchemaShardRequest shard + * @property {string|null} [wait_position] ReloadSchemaShardRequest wait_position + * @property {boolean|null} [include_primary] ReloadSchemaShardRequest include_primary + * @property {number|null} [concurrency] ReloadSchemaShardRequest concurrency */ /** - * Constructs a new RefreshStateRequest. + * Constructs a new ReloadSchemaShardRequest. * @memberof vtctldata - * @classdesc Represents a RefreshStateRequest. - * @implements IRefreshStateRequest + * @classdesc Represents a ReloadSchemaShardRequest. + * @implements IReloadSchemaShardRequest * @constructor - * @param {vtctldata.IRefreshStateRequest=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaShardRequest=} [properties] Properties to set */ - function RefreshStateRequest(properties) { + function ReloadSchemaShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -126112,75 +132621,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RefreshStateRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.RefreshStateRequest + * ReloadSchemaShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ReloadSchemaShardRequest * @instance */ - RefreshStateRequest.prototype.tablet_alias = null; + ReloadSchemaShardRequest.prototype.keyspace = ""; /** - * Creates a new RefreshStateRequest instance using the specified properties. + * ReloadSchemaShardRequest shard. + * @member {string} shard + * @memberof vtctldata.ReloadSchemaShardRequest + * @instance + */ + ReloadSchemaShardRequest.prototype.shard = ""; + + /** + * ReloadSchemaShardRequest wait_position. + * @member {string} wait_position + * @memberof vtctldata.ReloadSchemaShardRequest + * @instance + */ + ReloadSchemaShardRequest.prototype.wait_position = ""; + + /** + * ReloadSchemaShardRequest include_primary. + * @member {boolean} include_primary + * @memberof vtctldata.ReloadSchemaShardRequest + * @instance + */ + ReloadSchemaShardRequest.prototype.include_primary = false; + + /** + * ReloadSchemaShardRequest concurrency. + * @member {number} concurrency + * @memberof vtctldata.ReloadSchemaShardRequest + * @instance + */ + ReloadSchemaShardRequest.prototype.concurrency = 0; + + /** + * Creates a new ReloadSchemaShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static - * @param {vtctldata.IRefreshStateRequest=} [properties] Properties to set - * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest instance + * @param {vtctldata.IReloadSchemaShardRequest=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest instance */ - RefreshStateRequest.create = function create(properties) { - return new RefreshStateRequest(properties); + ReloadSchemaShardRequest.create = function create(properties) { + return new ReloadSchemaShardRequest(properties); }; /** - * Encodes the specified RefreshStateRequest message. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaShardRequest message. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static - * @param {vtctldata.IRefreshStateRequest} message RefreshStateRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaShardRequest} message ReloadSchemaShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateRequest.encode = function encode(message, writer) { + ReloadSchemaShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.wait_position != null && Object.hasOwnProperty.call(message, "wait_position")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.wait_position); + if (message.include_primary != null && Object.hasOwnProperty.call(message, "include_primary")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_primary); + if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) + writer.uint32(/* id 5, wireType 0 =*/40).uint32(message.concurrency); return writer; }; /** - * Encodes the specified RefreshStateRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateRequest.verify|verify} messages. + * Encodes the specified ReloadSchemaShardRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static - * @param {vtctldata.IRefreshStateRequest} message RefreshStateRequest message or plain object to encode + * @param {vtctldata.IReloadSchemaShardRequest} message ReloadSchemaShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RefreshStateRequest message from the specified reader or buffer. + * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest + * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateRequest.decode = function decode(reader, length) { + ReloadSchemaShardRequest.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.RefreshStateRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.wait_position = reader.string(); + break; + } + case 4: { + message.include_primary = reader.bool(); + break; + } + case 5: { + message.concurrency = reader.uint32(); break; } default: @@ -126192,126 +132757,156 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RefreshStateRequest message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest + * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateRequest.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RefreshStateRequest message. + * Verifies a ReloadSchemaShardRequest message. * @function verify - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RefreshStateRequest.verify = function verify(message) { + ReloadSchemaShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.wait_position != null && message.hasOwnProperty("wait_position")) + if (!$util.isString(message.wait_position)) + return "wait_position: string expected"; + if (message.include_primary != null && message.hasOwnProperty("include_primary")) + if (typeof message.include_primary !== "boolean") + return "include_primary: boolean expected"; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + if (!$util.isInteger(message.concurrency)) + return "concurrency: integer expected"; return null; }; /** - * Creates a RefreshStateRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RefreshStateRequest} RefreshStateRequest + * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest */ - RefreshStateRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RefreshStateRequest) + ReloadSchemaShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaShardRequest) return object; - let message = new $root.vtctldata.RefreshStateRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.RefreshStateRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.ReloadSchemaShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.wait_position != null) + message.wait_position = String(object.wait_position); + if (object.include_primary != null) + message.include_primary = Boolean(object.include_primary); + if (object.concurrency != null) + message.concurrency = object.concurrency >>> 0; return message; }; /** - * Creates a plain object from a RefreshStateRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static - * @param {vtctldata.RefreshStateRequest} message RefreshStateRequest + * @param {vtctldata.ReloadSchemaShardRequest} message ReloadSchemaShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RefreshStateRequest.toObject = function toObject(message, options) { + ReloadSchemaShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.wait_position = ""; + object.include_primary = false; + object.concurrency = 0; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.wait_position != null && message.hasOwnProperty("wait_position")) + object.wait_position = message.wait_position; + if (message.include_primary != null && message.hasOwnProperty("include_primary")) + object.include_primary = message.include_primary; + if (message.concurrency != null && message.hasOwnProperty("concurrency")) + object.concurrency = message.concurrency; return object; }; /** - * Converts this RefreshStateRequest to JSON. + * Converts this ReloadSchemaShardRequest to JSON. * @function toJSON - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @instance * @returns {Object.} JSON object */ - RefreshStateRequest.prototype.toJSON = function toJSON() { + ReloadSchemaShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RefreshStateRequest + * Gets the default type url for ReloadSchemaShardRequest * @function getTypeUrl - * @memberof vtctldata.RefreshStateRequest + * @memberof vtctldata.ReloadSchemaShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RefreshStateRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RefreshStateRequest"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaShardRequest"; }; - return RefreshStateRequest; + return ReloadSchemaShardRequest; })(); - vtctldata.RefreshStateResponse = (function() { + vtctldata.ReloadSchemaShardResponse = (function() { /** - * Properties of a RefreshStateResponse. + * Properties of a ReloadSchemaShardResponse. * @memberof vtctldata - * @interface IRefreshStateResponse + * @interface IReloadSchemaShardResponse + * @property {Array.|null} [events] ReloadSchemaShardResponse events */ /** - * Constructs a new RefreshStateResponse. + * Constructs a new ReloadSchemaShardResponse. * @memberof vtctldata - * @classdesc Represents a RefreshStateResponse. - * @implements IRefreshStateResponse + * @classdesc Represents a ReloadSchemaShardResponse. + * @implements IReloadSchemaShardResponse * @constructor - * @param {vtctldata.IRefreshStateResponse=} [properties] Properties to set + * @param {vtctldata.IReloadSchemaShardResponse=} [properties] Properties to set */ - function RefreshStateResponse(properties) { + function ReloadSchemaShardResponse(properties) { + this.events = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -126319,63 +132914,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RefreshStateResponse instance using the specified properties. + * ReloadSchemaShardResponse events. + * @member {Array.} events + * @memberof vtctldata.ReloadSchemaShardResponse + * @instance + */ + ReloadSchemaShardResponse.prototype.events = $util.emptyArray; + + /** + * Creates a new ReloadSchemaShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static - * @param {vtctldata.IRefreshStateResponse=} [properties] Properties to set - * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse instance + * @param {vtctldata.IReloadSchemaShardResponse=} [properties] Properties to set + * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse instance */ - RefreshStateResponse.create = function create(properties) { - return new RefreshStateResponse(properties); + ReloadSchemaShardResponse.create = function create(properties) { + return new ReloadSchemaShardResponse(properties); }; /** - * Encodes the specified RefreshStateResponse message. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaShardResponse message. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static - * @param {vtctldata.IRefreshStateResponse} message RefreshStateResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaShardResponse} message ReloadSchemaShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateResponse.encode = function encode(message, writer) { + ReloadSchemaShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.events != null && message.events.length) + for (let i = 0; i < message.events.length; ++i) + $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified RefreshStateResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateResponse.verify|verify} messages. + * Encodes the specified ReloadSchemaShardResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static - * @param {vtctldata.IRefreshStateResponse} message RefreshStateResponse message or plain object to encode + * @param {vtctldata.IReloadSchemaShardResponse} message ReloadSchemaShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReloadSchemaShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RefreshStateResponse message from the specified reader or buffer. + * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse + * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateResponse.decode = function decode(reader, length) { + ReloadSchemaShardResponse.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.RefreshStateResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReloadSchemaShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 2: { + if (!(message.events && message.events.length)) + message.events = []; + message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + break; + } default: reader.skipType(tag & 7); break; @@ -126385,112 +132997,141 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RefreshStateResponse message from the specified reader or buffer, length delimited. + * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse + * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateResponse.decodeDelimited = function decodeDelimited(reader) { + ReloadSchemaShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RefreshStateResponse message. + * Verifies a ReloadSchemaShardResponse message. * @function verify - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RefreshStateResponse.verify = function verify(message) { + ReloadSchemaShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.events != null && message.hasOwnProperty("events")) { + if (!Array.isArray(message.events)) + return "events: array expected"; + for (let i = 0; i < message.events.length; ++i) { + let error = $root.logutil.Event.verify(message.events[i]); + if (error) + return "events." + error; + } + } return null; }; /** - * Creates a RefreshStateResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReloadSchemaShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RefreshStateResponse} RefreshStateResponse + * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse */ - RefreshStateResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RefreshStateResponse) + ReloadSchemaShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReloadSchemaShardResponse) return object; - return new $root.vtctldata.RefreshStateResponse(); + let message = new $root.vtctldata.ReloadSchemaShardResponse(); + if (object.events) { + if (!Array.isArray(object.events)) + throw TypeError(".vtctldata.ReloadSchemaShardResponse.events: array expected"); + message.events = []; + for (let i = 0; i < object.events.length; ++i) { + if (typeof object.events[i] !== "object") + throw TypeError(".vtctldata.ReloadSchemaShardResponse.events: object expected"); + message.events[i] = $root.logutil.Event.fromObject(object.events[i]); + } + } + return message; }; /** - * Creates a plain object from a RefreshStateResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReloadSchemaShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static - * @param {vtctldata.RefreshStateResponse} message RefreshStateResponse + * @param {vtctldata.ReloadSchemaShardResponse} message ReloadSchemaShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RefreshStateResponse.toObject = function toObject() { - return {}; + ReloadSchemaShardResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.events = []; + if (message.events && message.events.length) { + object.events = []; + for (let j = 0; j < message.events.length; ++j) + object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + } + return object; }; /** - * Converts this RefreshStateResponse to JSON. + * Converts this ReloadSchemaShardResponse to JSON. * @function toJSON - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @instance * @returns {Object.} JSON object */ - RefreshStateResponse.prototype.toJSON = function toJSON() { + ReloadSchemaShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RefreshStateResponse + * Gets the default type url for ReloadSchemaShardResponse * @function getTypeUrl - * @memberof vtctldata.RefreshStateResponse + * @memberof vtctldata.ReloadSchemaShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RefreshStateResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReloadSchemaShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RefreshStateResponse"; + return typeUrlPrefix + "/vtctldata.ReloadSchemaShardResponse"; }; - return RefreshStateResponse; + return ReloadSchemaShardResponse; })(); - vtctldata.RefreshStateByShardRequest = (function() { + vtctldata.RemoveBackupRequest = (function() { /** - * Properties of a RefreshStateByShardRequest. + * Properties of a RemoveBackupRequest. * @memberof vtctldata - * @interface IRefreshStateByShardRequest - * @property {string|null} [keyspace] RefreshStateByShardRequest keyspace - * @property {string|null} [shard] RefreshStateByShardRequest shard - * @property {Array.|null} [cells] RefreshStateByShardRequest cells + * @interface IRemoveBackupRequest + * @property {string|null} [keyspace] RemoveBackupRequest keyspace + * @property {string|null} [shard] RemoveBackupRequest shard + * @property {string|null} [name] RemoveBackupRequest name */ /** - * Constructs a new RefreshStateByShardRequest. + * Constructs a new RemoveBackupRequest. * @memberof vtctldata - * @classdesc Represents a RefreshStateByShardRequest. - * @implements IRefreshStateByShardRequest + * @classdesc Represents a RemoveBackupRequest. + * @implements IRemoveBackupRequest * @constructor - * @param {vtctldata.IRefreshStateByShardRequest=} [properties] Properties to set + * @param {vtctldata.IRemoveBackupRequest=} [properties] Properties to set */ - function RefreshStateByShardRequest(properties) { - this.cells = []; + function RemoveBackupRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -126498,91 +133139,90 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RefreshStateByShardRequest keyspace. + * RemoveBackupRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @instance */ - RefreshStateByShardRequest.prototype.keyspace = ""; + RemoveBackupRequest.prototype.keyspace = ""; /** - * RefreshStateByShardRequest shard. + * RemoveBackupRequest shard. * @member {string} shard - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @instance */ - RefreshStateByShardRequest.prototype.shard = ""; + RemoveBackupRequest.prototype.shard = ""; /** - * RefreshStateByShardRequest cells. - * @member {Array.} cells - * @memberof vtctldata.RefreshStateByShardRequest + * RemoveBackupRequest name. + * @member {string} name + * @memberof vtctldata.RemoveBackupRequest * @instance */ - RefreshStateByShardRequest.prototype.cells = $util.emptyArray; + RemoveBackupRequest.prototype.name = ""; /** - * Creates a new RefreshStateByShardRequest instance using the specified properties. + * Creates a new RemoveBackupRequest instance using the specified properties. * @function create - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static - * @param {vtctldata.IRefreshStateByShardRequest=} [properties] Properties to set - * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest instance + * @param {vtctldata.IRemoveBackupRequest=} [properties] Properties to set + * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest instance */ - RefreshStateByShardRequest.create = function create(properties) { - return new RefreshStateByShardRequest(properties); + RemoveBackupRequest.create = function create(properties) { + return new RemoveBackupRequest(properties); }; /** - * Encodes the specified RefreshStateByShardRequest message. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. + * Encodes the specified RemoveBackupRequest message. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static - * @param {vtctldata.IRefreshStateByShardRequest} message RefreshStateByShardRequest message or plain object to encode + * @param {vtctldata.IRemoveBackupRequest} message RemoveBackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateByShardRequest.encode = function encode(message, writer) { + RemoveBackupRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.name); return writer; }; /** - * Encodes the specified RefreshStateByShardRequest message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardRequest.verify|verify} messages. + * Encodes the specified RemoveBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static - * @param {vtctldata.IRefreshStateByShardRequest} message RefreshStateByShardRequest message or plain object to encode + * @param {vtctldata.IRemoveBackupRequest} message RemoveBackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateByShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + RemoveBackupRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RefreshStateByShardRequest message from the specified reader or buffer. + * Decodes a RemoveBackupRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest + * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateByShardRequest.decode = function decode(reader, length) { + RemoveBackupRequest.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.RefreshStateByShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveBackupRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -126595,9 +133235,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 3: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + message.name = reader.string(); break; } default: @@ -126609,30 +133247,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RefreshStateByShardRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveBackupRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest + * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateByShardRequest.decodeDelimited = function decodeDelimited(reader) { + RemoveBackupRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RefreshStateByShardRequest message. + * Verifies a RemoveBackupRequest message. * @function verify - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RefreshStateByShardRequest.verify = function verify(message) { + RemoveBackupRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -126641,121 +133279,106 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; return null; }; /** - * Creates a RefreshStateByShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveBackupRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RefreshStateByShardRequest} RefreshStateByShardRequest + * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest */ - RefreshStateByShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RefreshStateByShardRequest) + RemoveBackupRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveBackupRequest) return object; - let message = new $root.vtctldata.RefreshStateByShardRequest(); + let message = new $root.vtctldata.RemoveBackupRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.RefreshStateByShardRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } + if (object.name != null) + message.name = String(object.name); return message; }; /** - * Creates a plain object from a RefreshStateByShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a RemoveBackupRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static - * @param {vtctldata.RefreshStateByShardRequest} message RefreshStateByShardRequest + * @param {vtctldata.RemoveBackupRequest} message RemoveBackupRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RefreshStateByShardRequest.toObject = function toObject(message, options) { + RemoveBackupRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; if (options.defaults) { object.keyspace = ""; object.shard = ""; + object.name = ""; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; return object; }; /** - * Converts this RefreshStateByShardRequest to JSON. + * Converts this RemoveBackupRequest to JSON. * @function toJSON - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @instance * @returns {Object.} JSON object */ - RefreshStateByShardRequest.prototype.toJSON = function toJSON() { + RemoveBackupRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RefreshStateByShardRequest + * Gets the default type url for RemoveBackupRequest * @function getTypeUrl - * @memberof vtctldata.RefreshStateByShardRequest + * @memberof vtctldata.RemoveBackupRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RefreshStateByShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveBackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RefreshStateByShardRequest"; + return typeUrlPrefix + "/vtctldata.RemoveBackupRequest"; }; - return RefreshStateByShardRequest; + return RemoveBackupRequest; })(); - vtctldata.RefreshStateByShardResponse = (function() { + vtctldata.RemoveBackupResponse = (function() { /** - * Properties of a RefreshStateByShardResponse. + * Properties of a RemoveBackupResponse. * @memberof vtctldata - * @interface IRefreshStateByShardResponse - * @property {boolean|null} [is_partial_refresh] RefreshStateByShardResponse is_partial_refresh - * @property {string|null} [partial_refresh_details] RefreshStateByShardResponse partial_refresh_details + * @interface IRemoveBackupResponse */ /** - * Constructs a new RefreshStateByShardResponse. + * Constructs a new RemoveBackupResponse. * @memberof vtctldata - * @classdesc Represents a RefreshStateByShardResponse. - * @implements IRefreshStateByShardResponse + * @classdesc Represents a RemoveBackupResponse. + * @implements IRemoveBackupResponse * @constructor - * @param {vtctldata.IRefreshStateByShardResponse=} [properties] Properties to set + * @param {vtctldata.IRemoveBackupResponse=} [properties] Properties to set */ - function RefreshStateByShardResponse(properties) { + function RemoveBackupResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -126763,91 +133386,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RefreshStateByShardResponse is_partial_refresh. - * @member {boolean} is_partial_refresh - * @memberof vtctldata.RefreshStateByShardResponse - * @instance - */ - RefreshStateByShardResponse.prototype.is_partial_refresh = false; - - /** - * RefreshStateByShardResponse partial_refresh_details. - * @member {string} partial_refresh_details - * @memberof vtctldata.RefreshStateByShardResponse - * @instance - */ - RefreshStateByShardResponse.prototype.partial_refresh_details = ""; - - /** - * Creates a new RefreshStateByShardResponse instance using the specified properties. + * Creates a new RemoveBackupResponse instance using the specified properties. * @function create - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static - * @param {vtctldata.IRefreshStateByShardResponse=} [properties] Properties to set - * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse instance + * @param {vtctldata.IRemoveBackupResponse=} [properties] Properties to set + * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse instance */ - RefreshStateByShardResponse.create = function create(properties) { - return new RefreshStateByShardResponse(properties); + RemoveBackupResponse.create = function create(properties) { + return new RemoveBackupResponse(properties); }; /** - * Encodes the specified RefreshStateByShardResponse message. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. + * Encodes the specified RemoveBackupResponse message. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static - * @param {vtctldata.IRefreshStateByShardResponse} message RefreshStateByShardResponse message or plain object to encode + * @param {vtctldata.IRemoveBackupResponse} message RemoveBackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateByShardResponse.encode = function encode(message, writer) { + RemoveBackupResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.is_partial_refresh != null && Object.hasOwnProperty.call(message, "is_partial_refresh")) - writer.uint32(/* id 1, wireType 0 =*/8).bool(message.is_partial_refresh); - if (message.partial_refresh_details != null && Object.hasOwnProperty.call(message, "partial_refresh_details")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.partial_refresh_details); return writer; }; /** - * Encodes the specified RefreshStateByShardResponse message, length delimited. Does not implicitly {@link vtctldata.RefreshStateByShardResponse.verify|verify} messages. + * Encodes the specified RemoveBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static - * @param {vtctldata.IRefreshStateByShardResponse} message RefreshStateByShardResponse message or plain object to encode + * @param {vtctldata.IRemoveBackupResponse} message RemoveBackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RefreshStateByShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + RemoveBackupResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RefreshStateByShardResponse message from the specified reader or buffer. + * Decodes a RemoveBackupResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse + * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateByShardResponse.decode = function decode(reader, length) { + RemoveBackupResponse.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.RefreshStateByShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveBackupResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.is_partial_refresh = reader.bool(); - break; - } - case 2: { - message.partial_refresh_details = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -126857,131 +133452,112 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RefreshStateByShardResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveBackupResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse + * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RefreshStateByShardResponse.decodeDelimited = function decodeDelimited(reader) { + RemoveBackupResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RefreshStateByShardResponse message. + * Verifies a RemoveBackupResponse message. * @function verify - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RefreshStateByShardResponse.verify = function verify(message) { + RemoveBackupResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.is_partial_refresh != null && message.hasOwnProperty("is_partial_refresh")) - if (typeof message.is_partial_refresh !== "boolean") - return "is_partial_refresh: boolean expected"; - if (message.partial_refresh_details != null && message.hasOwnProperty("partial_refresh_details")) - if (!$util.isString(message.partial_refresh_details)) - return "partial_refresh_details: string expected"; return null; }; /** - * Creates a RefreshStateByShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveBackupResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RefreshStateByShardResponse} RefreshStateByShardResponse + * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse */ - RefreshStateByShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RefreshStateByShardResponse) + RemoveBackupResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveBackupResponse) return object; - let message = new $root.vtctldata.RefreshStateByShardResponse(); - if (object.is_partial_refresh != null) - message.is_partial_refresh = Boolean(object.is_partial_refresh); - if (object.partial_refresh_details != null) - message.partial_refresh_details = String(object.partial_refresh_details); - return message; + return new $root.vtctldata.RemoveBackupResponse(); }; /** - * Creates a plain object from a RefreshStateByShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a RemoveBackupResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static - * @param {vtctldata.RefreshStateByShardResponse} message RefreshStateByShardResponse + * @param {vtctldata.RemoveBackupResponse} message RemoveBackupResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RefreshStateByShardResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.is_partial_refresh = false; - object.partial_refresh_details = ""; - } - if (message.is_partial_refresh != null && message.hasOwnProperty("is_partial_refresh")) - object.is_partial_refresh = message.is_partial_refresh; - if (message.partial_refresh_details != null && message.hasOwnProperty("partial_refresh_details")) - object.partial_refresh_details = message.partial_refresh_details; - return object; + RemoveBackupResponse.toObject = function toObject() { + return {}; }; /** - * Converts this RefreshStateByShardResponse to JSON. + * Converts this RemoveBackupResponse to JSON. * @function toJSON - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @instance * @returns {Object.} JSON object */ - RefreshStateByShardResponse.prototype.toJSON = function toJSON() { + RemoveBackupResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RefreshStateByShardResponse + * Gets the default type url for RemoveBackupResponse * @function getTypeUrl - * @memberof vtctldata.RefreshStateByShardResponse + * @memberof vtctldata.RemoveBackupResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RefreshStateByShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveBackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RefreshStateByShardResponse"; + return typeUrlPrefix + "/vtctldata.RemoveBackupResponse"; }; - return RefreshStateByShardResponse; + return RemoveBackupResponse; })(); - vtctldata.ReloadSchemaRequest = (function() { + vtctldata.RemoveKeyspaceCellRequest = (function() { /** - * Properties of a ReloadSchemaRequest. + * Properties of a RemoveKeyspaceCellRequest. * @memberof vtctldata - * @interface IReloadSchemaRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] ReloadSchemaRequest tablet_alias + * @interface IRemoveKeyspaceCellRequest + * @property {string|null} [keyspace] RemoveKeyspaceCellRequest keyspace + * @property {string|null} [cell] RemoveKeyspaceCellRequest cell + * @property {boolean|null} [force] RemoveKeyspaceCellRequest force + * @property {boolean|null} [recursive] RemoveKeyspaceCellRequest recursive */ /** - * Constructs a new ReloadSchemaRequest. + * Constructs a new RemoveKeyspaceCellRequest. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaRequest. - * @implements IReloadSchemaRequest + * @classdesc Represents a RemoveKeyspaceCellRequest. + * @implements IRemoveKeyspaceCellRequest * @constructor - * @param {vtctldata.IReloadSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IRemoveKeyspaceCellRequest=} [properties] Properties to set */ - function ReloadSchemaRequest(properties) { + function RemoveKeyspaceCellRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -126989,75 +133565,117 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReloadSchemaRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ReloadSchemaRequest + * RemoveKeyspaceCellRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.RemoveKeyspaceCellRequest * @instance */ - ReloadSchemaRequest.prototype.tablet_alias = null; + RemoveKeyspaceCellRequest.prototype.keyspace = ""; /** - * Creates a new ReloadSchemaRequest instance using the specified properties. + * RemoveKeyspaceCellRequest cell. + * @member {string} cell + * @memberof vtctldata.RemoveKeyspaceCellRequest + * @instance + */ + RemoveKeyspaceCellRequest.prototype.cell = ""; + + /** + * RemoveKeyspaceCellRequest force. + * @member {boolean} force + * @memberof vtctldata.RemoveKeyspaceCellRequest + * @instance + */ + RemoveKeyspaceCellRequest.prototype.force = false; + + /** + * RemoveKeyspaceCellRequest recursive. + * @member {boolean} recursive + * @memberof vtctldata.RemoveKeyspaceCellRequest + * @instance + */ + RemoveKeyspaceCellRequest.prototype.recursive = false; + + /** + * Creates a new RemoveKeyspaceCellRequest instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static - * @param {vtctldata.IReloadSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest instance + * @param {vtctldata.IRemoveKeyspaceCellRequest=} [properties] Properties to set + * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest instance */ - ReloadSchemaRequest.create = function create(properties) { - return new ReloadSchemaRequest(properties); + RemoveKeyspaceCellRequest.create = function create(properties) { + return new RemoveKeyspaceCellRequest(properties); }; /** - * Encodes the specified ReloadSchemaRequest message. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. + * Encodes the specified RemoveKeyspaceCellRequest message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static - * @param {vtctldata.IReloadSchemaRequest} message ReloadSchemaRequest message or plain object to encode + * @param {vtctldata.IRemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaRequest.encode = function encode(message, writer) { + RemoveKeyspaceCellRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cell); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); + if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.recursive); return writer; }; /** - * Encodes the specified ReloadSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaRequest.verify|verify} messages. + * Encodes the specified RemoveKeyspaceCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static - * @param {vtctldata.IReloadSchemaRequest} message ReloadSchemaRequest message or plain object to encode + * @param {vtctldata.IRemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + RemoveKeyspaceCellRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaRequest message from the specified reader or buffer. + * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest + * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaRequest.decode = function decode(reader, length) { + RemoveKeyspaceCellRequest.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.ReloadSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveKeyspaceCellRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.cell = reader.string(); + break; + } + case 3: { + message.force = reader.bool(); + break; + } + case 4: { + message.recursive = reader.bool(); break; } default: @@ -127069,126 +133687,146 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest + * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + RemoveKeyspaceCellRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaRequest message. + * Verifies a RemoveKeyspaceCellRequest message. * @function verify - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaRequest.verify = function verify(message) { + RemoveKeyspaceCellRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; + if (message.recursive != null && message.hasOwnProperty("recursive")) + if (typeof message.recursive !== "boolean") + return "recursive: boolean expected"; return null; }; /** - * Creates a ReloadSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveKeyspaceCellRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaRequest} ReloadSchemaRequest + * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest */ - ReloadSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaRequest) + RemoveKeyspaceCellRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveKeyspaceCellRequest) return object; - let message = new $root.vtctldata.ReloadSchemaRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ReloadSchemaRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.RemoveKeyspaceCellRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.cell != null) + message.cell = String(object.cell); + if (object.force != null) + message.force = Boolean(object.force); + if (object.recursive != null) + message.recursive = Boolean(object.recursive); return message; }; /** - * Creates a plain object from a ReloadSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a RemoveKeyspaceCellRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static - * @param {vtctldata.ReloadSchemaRequest} message ReloadSchemaRequest + * @param {vtctldata.RemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaRequest.toObject = function toObject(message, options) { + RemoveKeyspaceCellRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) { + object.keyspace = ""; + object.cell = ""; + object.force = false; + object.recursive = false; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; + if (message.recursive != null && message.hasOwnProperty("recursive")) + object.recursive = message.recursive; return object; }; /** - * Converts this ReloadSchemaRequest to JSON. + * Converts this RemoveKeyspaceCellRequest to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @instance * @returns {Object.} JSON object */ - ReloadSchemaRequest.prototype.toJSON = function toJSON() { + RemoveKeyspaceCellRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaRequest + * Gets the default type url for RemoveKeyspaceCellRequest * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaRequest + * @memberof vtctldata.RemoveKeyspaceCellRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveKeyspaceCellRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaRequest"; + return typeUrlPrefix + "/vtctldata.RemoveKeyspaceCellRequest"; }; - return ReloadSchemaRequest; + return RemoveKeyspaceCellRequest; })(); - vtctldata.ReloadSchemaResponse = (function() { + vtctldata.RemoveKeyspaceCellResponse = (function() { /** - * Properties of a ReloadSchemaResponse. + * Properties of a RemoveKeyspaceCellResponse. * @memberof vtctldata - * @interface IReloadSchemaResponse + * @interface IRemoveKeyspaceCellResponse */ /** - * Constructs a new ReloadSchemaResponse. + * Constructs a new RemoveKeyspaceCellResponse. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaResponse. - * @implements IReloadSchemaResponse + * @classdesc Represents a RemoveKeyspaceCellResponse. + * @implements IRemoveKeyspaceCellResponse * @constructor - * @param {vtctldata.IReloadSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IRemoveKeyspaceCellResponse=} [properties] Properties to set */ - function ReloadSchemaResponse(properties) { + function RemoveKeyspaceCellResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -127196,60 +133834,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new ReloadSchemaResponse instance using the specified properties. + * Creates a new RemoveKeyspaceCellResponse instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static - * @param {vtctldata.IReloadSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse instance + * @param {vtctldata.IRemoveKeyspaceCellResponse=} [properties] Properties to set + * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse instance */ - ReloadSchemaResponse.create = function create(properties) { - return new ReloadSchemaResponse(properties); + RemoveKeyspaceCellResponse.create = function create(properties) { + return new RemoveKeyspaceCellResponse(properties); }; /** - * Encodes the specified ReloadSchemaResponse message. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. + * Encodes the specified RemoveKeyspaceCellResponse message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static - * @param {vtctldata.IReloadSchemaResponse} message ReloadSchemaResponse message or plain object to encode + * @param {vtctldata.IRemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaResponse.encode = function encode(message, writer) { + RemoveKeyspaceCellResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified ReloadSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaResponse.verify|verify} messages. + * Encodes the specified RemoveKeyspaceCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static - * @param {vtctldata.IReloadSchemaResponse} message ReloadSchemaResponse message or plain object to encode + * @param {vtctldata.IRemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + RemoveKeyspaceCellResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaResponse message from the specified reader or buffer. + * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse + * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaResponse.decode = function decode(reader, length) { + RemoveKeyspaceCellResponse.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.ReloadSchemaResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveKeyspaceCellResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -127262,112 +133900,113 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse + * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + RemoveKeyspaceCellResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaResponse message. + * Verifies a RemoveKeyspaceCellResponse message. * @function verify - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaResponse.verify = function verify(message) { + RemoveKeyspaceCellResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a ReloadSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveKeyspaceCellResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaResponse} ReloadSchemaResponse + * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse */ - ReloadSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaResponse) + RemoveKeyspaceCellResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveKeyspaceCellResponse) return object; - return new $root.vtctldata.ReloadSchemaResponse(); + return new $root.vtctldata.RemoveKeyspaceCellResponse(); }; /** - * Creates a plain object from a ReloadSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a RemoveKeyspaceCellResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static - * @param {vtctldata.ReloadSchemaResponse} message ReloadSchemaResponse + * @param {vtctldata.RemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaResponse.toObject = function toObject() { + RemoveKeyspaceCellResponse.toObject = function toObject() { return {}; }; /** - * Converts this ReloadSchemaResponse to JSON. + * Converts this RemoveKeyspaceCellResponse to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @instance * @returns {Object.} JSON object */ - ReloadSchemaResponse.prototype.toJSON = function toJSON() { + RemoveKeyspaceCellResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaResponse + * Gets the default type url for RemoveKeyspaceCellResponse * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaResponse + * @memberof vtctldata.RemoveKeyspaceCellResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveKeyspaceCellResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaResponse"; + return typeUrlPrefix + "/vtctldata.RemoveKeyspaceCellResponse"; }; - return ReloadSchemaResponse; + return RemoveKeyspaceCellResponse; })(); - vtctldata.ReloadSchemaKeyspaceRequest = (function() { + vtctldata.RemoveShardCellRequest = (function() { /** - * Properties of a ReloadSchemaKeyspaceRequest. + * Properties of a RemoveShardCellRequest. * @memberof vtctldata - * @interface IReloadSchemaKeyspaceRequest - * @property {string|null} [keyspace] ReloadSchemaKeyspaceRequest keyspace - * @property {string|null} [wait_position] ReloadSchemaKeyspaceRequest wait_position - * @property {boolean|null} [include_primary] ReloadSchemaKeyspaceRequest include_primary - * @property {number|null} [concurrency] ReloadSchemaKeyspaceRequest concurrency + * @interface IRemoveShardCellRequest + * @property {string|null} [keyspace] RemoveShardCellRequest keyspace + * @property {string|null} [shard_name] RemoveShardCellRequest shard_name + * @property {string|null} [cell] RemoveShardCellRequest cell + * @property {boolean|null} [force] RemoveShardCellRequest force + * @property {boolean|null} [recursive] RemoveShardCellRequest recursive */ /** - * Constructs a new ReloadSchemaKeyspaceRequest. + * Constructs a new RemoveShardCellRequest. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaKeyspaceRequest. - * @implements IReloadSchemaKeyspaceRequest + * @classdesc Represents a RemoveShardCellRequest. + * @implements IRemoveShardCellRequest * @constructor - * @param {vtctldata.IReloadSchemaKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IRemoveShardCellRequest=} [properties] Properties to set */ - function ReloadSchemaKeyspaceRequest(properties) { + function RemoveShardCellRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -127375,100 +134014,110 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReloadSchemaKeyspaceRequest keyspace. + * RemoveShardCellRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @instance */ - ReloadSchemaKeyspaceRequest.prototype.keyspace = ""; + RemoveShardCellRequest.prototype.keyspace = ""; /** - * ReloadSchemaKeyspaceRequest wait_position. - * @member {string} wait_position - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * RemoveShardCellRequest shard_name. + * @member {string} shard_name + * @memberof vtctldata.RemoveShardCellRequest * @instance */ - ReloadSchemaKeyspaceRequest.prototype.wait_position = ""; + RemoveShardCellRequest.prototype.shard_name = ""; /** - * ReloadSchemaKeyspaceRequest include_primary. - * @member {boolean} include_primary - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * RemoveShardCellRequest cell. + * @member {string} cell + * @memberof vtctldata.RemoveShardCellRequest * @instance */ - ReloadSchemaKeyspaceRequest.prototype.include_primary = false; + RemoveShardCellRequest.prototype.cell = ""; /** - * ReloadSchemaKeyspaceRequest concurrency. - * @member {number} concurrency - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * RemoveShardCellRequest force. + * @member {boolean} force + * @memberof vtctldata.RemoveShardCellRequest * @instance */ - ReloadSchemaKeyspaceRequest.prototype.concurrency = 0; + RemoveShardCellRequest.prototype.force = false; /** - * Creates a new ReloadSchemaKeyspaceRequest instance using the specified properties. + * RemoveShardCellRequest recursive. + * @member {boolean} recursive + * @memberof vtctldata.RemoveShardCellRequest + * @instance + */ + RemoveShardCellRequest.prototype.recursive = false; + + /** + * Creates a new RemoveShardCellRequest instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static - * @param {vtctldata.IReloadSchemaKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest instance + * @param {vtctldata.IRemoveShardCellRequest=} [properties] Properties to set + * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest instance */ - ReloadSchemaKeyspaceRequest.create = function create(properties) { - return new ReloadSchemaKeyspaceRequest(properties); + RemoveShardCellRequest.create = function create(properties) { + return new RemoveShardCellRequest(properties); }; /** - * Encodes the specified ReloadSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. + * Encodes the specified RemoveShardCellRequest message. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static - * @param {vtctldata.IReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest message or plain object to encode + * @param {vtctldata.IRemoveShardCellRequest} message RemoveShardCellRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaKeyspaceRequest.encode = function encode(message, writer) { + RemoveShardCellRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.wait_position != null && Object.hasOwnProperty.call(message, "wait_position")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.wait_position); - if (message.include_primary != null && Object.hasOwnProperty.call(message, "include_primary")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.include_primary); - if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) - writer.uint32(/* id 4, wireType 0 =*/32).uint32(message.concurrency); + if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cell); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); + if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.recursive); return writer; }; /** - * Encodes the specified ReloadSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceRequest.verify|verify} messages. + * Encodes the specified RemoveShardCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static - * @param {vtctldata.IReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest message or plain object to encode + * @param {vtctldata.IRemoveShardCellRequest} message RemoveShardCellRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + RemoveShardCellRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer. + * Decodes a RemoveShardCellRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest + * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaKeyspaceRequest.decode = function decode(reader, length) { + RemoveShardCellRequest.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.ReloadSchemaKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveShardCellRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -127477,15 +134126,19 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.wait_position = reader.string(); + message.shard_name = reader.string(); break; } case 3: { - message.include_primary = reader.bool(); + message.cell = reader.string(); break; } case 4: { - message.concurrency = reader.uint32(); + message.force = reader.bool(); + break; + } + case 5: { + message.recursive = reader.bool(); break; } default: @@ -127497,148 +134150,154 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a RemoveShardCellRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest + * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + RemoveShardCellRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaKeyspaceRequest message. + * Verifies a RemoveShardCellRequest message. * @function verify - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaKeyspaceRequest.verify = function verify(message) { + RemoveShardCellRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.wait_position != null && message.hasOwnProperty("wait_position")) - if (!$util.isString(message.wait_position)) - return "wait_position: string expected"; - if (message.include_primary != null && message.hasOwnProperty("include_primary")) - if (typeof message.include_primary !== "boolean") - return "include_primary: boolean expected"; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (!$util.isInteger(message.concurrency)) - return "concurrency: integer expected"; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + if (!$util.isString(message.shard_name)) + return "shard_name: string expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; + if (message.recursive != null && message.hasOwnProperty("recursive")) + if (typeof message.recursive !== "boolean") + return "recursive: boolean expected"; return null; }; /** - * Creates a ReloadSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveShardCellRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaKeyspaceRequest} ReloadSchemaKeyspaceRequest + * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest */ - ReloadSchemaKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaKeyspaceRequest) + RemoveShardCellRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveShardCellRequest) return object; - let message = new $root.vtctldata.ReloadSchemaKeyspaceRequest(); + let message = new $root.vtctldata.RemoveShardCellRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.wait_position != null) - message.wait_position = String(object.wait_position); - if (object.include_primary != null) - message.include_primary = Boolean(object.include_primary); - if (object.concurrency != null) - message.concurrency = object.concurrency >>> 0; + if (object.shard_name != null) + message.shard_name = String(object.shard_name); + if (object.cell != null) + message.cell = String(object.cell); + if (object.force != null) + message.force = Boolean(object.force); + if (object.recursive != null) + message.recursive = Boolean(object.recursive); return message; }; /** - * Creates a plain object from a ReloadSchemaKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a RemoveShardCellRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static - * @param {vtctldata.ReloadSchemaKeyspaceRequest} message ReloadSchemaKeyspaceRequest + * @param {vtctldata.RemoveShardCellRequest} message RemoveShardCellRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaKeyspaceRequest.toObject = function toObject(message, options) { + RemoveShardCellRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; - object.wait_position = ""; - object.include_primary = false; - object.concurrency = 0; + object.shard_name = ""; + object.cell = ""; + object.force = false; + object.recursive = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.wait_position != null && message.hasOwnProperty("wait_position")) - object.wait_position = message.wait_position; - if (message.include_primary != null && message.hasOwnProperty("include_primary")) - object.include_primary = message.include_primary; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - object.concurrency = message.concurrency; + if (message.shard_name != null && message.hasOwnProperty("shard_name")) + object.shard_name = message.shard_name; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; + if (message.recursive != null && message.hasOwnProperty("recursive")) + object.recursive = message.recursive; return object; }; /** - * Converts this ReloadSchemaKeyspaceRequest to JSON. + * Converts this RemoveShardCellRequest to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @instance * @returns {Object.} JSON object */ - ReloadSchemaKeyspaceRequest.prototype.toJSON = function toJSON() { + RemoveShardCellRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaKeyspaceRequest + * Gets the default type url for RemoveShardCellRequest * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaKeyspaceRequest + * @memberof vtctldata.RemoveShardCellRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveShardCellRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.RemoveShardCellRequest"; }; - return ReloadSchemaKeyspaceRequest; + return RemoveShardCellRequest; })(); - vtctldata.ReloadSchemaKeyspaceResponse = (function() { + vtctldata.RemoveShardCellResponse = (function() { /** - * Properties of a ReloadSchemaKeyspaceResponse. + * Properties of a RemoveShardCellResponse. * @memberof vtctldata - * @interface IReloadSchemaKeyspaceResponse - * @property {Array.|null} [events] ReloadSchemaKeyspaceResponse events + * @interface IRemoveShardCellResponse */ /** - * Constructs a new ReloadSchemaKeyspaceResponse. + * Constructs a new RemoveShardCellResponse. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaKeyspaceResponse. - * @implements IReloadSchemaKeyspaceResponse + * @classdesc Represents a RemoveShardCellResponse. + * @implements IRemoveShardCellResponse * @constructor - * @param {vtctldata.IReloadSchemaKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IRemoveShardCellResponse=} [properties] Properties to set */ - function ReloadSchemaKeyspaceResponse(properties) { - this.events = []; + function RemoveShardCellResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -127646,80 +134305,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReloadSchemaKeyspaceResponse events. - * @member {Array.} events - * @memberof vtctldata.ReloadSchemaKeyspaceResponse - * @instance - */ - ReloadSchemaKeyspaceResponse.prototype.events = $util.emptyArray; - - /** - * Creates a new ReloadSchemaKeyspaceResponse instance using the specified properties. + * Creates a new RemoveShardCellResponse instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static - * @param {vtctldata.IReloadSchemaKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse instance + * @param {vtctldata.IRemoveShardCellResponse=} [properties] Properties to set + * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse instance */ - ReloadSchemaKeyspaceResponse.create = function create(properties) { - return new ReloadSchemaKeyspaceResponse(properties); + RemoveShardCellResponse.create = function create(properties) { + return new RemoveShardCellResponse(properties); }; /** - * Encodes the specified ReloadSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. + * Encodes the specified RemoveShardCellResponse message. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static - * @param {vtctldata.IReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse message or plain object to encode + * @param {vtctldata.IRemoveShardCellResponse} message RemoveShardCellResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaKeyspaceResponse.encode = function encode(message, writer) { + RemoveShardCellResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReloadSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaKeyspaceResponse.verify|verify} messages. + * Encodes the specified RemoveShardCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static - * @param {vtctldata.IReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse message or plain object to encode + * @param {vtctldata.IRemoveShardCellResponse} message RemoveShardCellResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + RemoveShardCellResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer. + * Decodes a RemoveShardCellResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse + * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaKeyspaceResponse.decode = function decode(reader, length) { + RemoveShardCellResponse.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.ReloadSchemaKeyspaceResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RemoveShardCellResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.logutil.Event.decode(reader, reader.uint32())); - break; - } default: reader.skipType(tag & 7); break; @@ -127729,143 +134371,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a RemoveShardCellResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse + * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + RemoveShardCellResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaKeyspaceResponse message. + * Verifies a RemoveShardCellResponse message. * @function verify - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaKeyspaceResponse.verify = function verify(message) { + RemoveShardCellResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.logutil.Event.verify(message.events[i]); - if (error) - return "events." + error; - } - } return null; }; /** - * Creates a ReloadSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RemoveShardCellResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaKeyspaceResponse} ReloadSchemaKeyspaceResponse + * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse */ - ReloadSchemaKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaKeyspaceResponse) + RemoveShardCellResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RemoveShardCellResponse) return object; - let message = new $root.vtctldata.ReloadSchemaKeyspaceResponse(); - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".vtctldata.ReloadSchemaKeyspaceResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".vtctldata.ReloadSchemaKeyspaceResponse.events: object expected"); - message.events[i] = $root.logutil.Event.fromObject(object.events[i]); - } - } - return message; + return new $root.vtctldata.RemoveShardCellResponse(); }; /** - * Creates a plain object from a ReloadSchemaKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a RemoveShardCellResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static - * @param {vtctldata.ReloadSchemaKeyspaceResponse} message ReloadSchemaKeyspaceResponse + * @param {vtctldata.RemoveShardCellResponse} message RemoveShardCellResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaKeyspaceResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.events = []; - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.logutil.Event.toObject(message.events[j], options); - } - return object; + RemoveShardCellResponse.toObject = function toObject() { + return {}; }; /** - * Converts this ReloadSchemaKeyspaceResponse to JSON. + * Converts this RemoveShardCellResponse to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @instance * @returns {Object.} JSON object */ - ReloadSchemaKeyspaceResponse.prototype.toJSON = function toJSON() { + RemoveShardCellResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaKeyspaceResponse + * Gets the default type url for RemoveShardCellResponse * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaKeyspaceResponse + * @memberof vtctldata.RemoveShardCellResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RemoveShardCellResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.RemoveShardCellResponse"; }; - return ReloadSchemaKeyspaceResponse; + return RemoveShardCellResponse; })(); - vtctldata.ReloadSchemaShardRequest = (function() { + vtctldata.ReparentTabletRequest = (function() { /** - * Properties of a ReloadSchemaShardRequest. + * Properties of a ReparentTabletRequest. * @memberof vtctldata - * @interface IReloadSchemaShardRequest - * @property {string|null} [keyspace] ReloadSchemaShardRequest keyspace - * @property {string|null} [shard] ReloadSchemaShardRequest shard - * @property {string|null} [wait_position] ReloadSchemaShardRequest wait_position - * @property {boolean|null} [include_primary] ReloadSchemaShardRequest include_primary - * @property {number|null} [concurrency] ReloadSchemaShardRequest concurrency + * @interface IReparentTabletRequest + * @property {topodata.ITabletAlias|null} [tablet] ReparentTabletRequest tablet */ /** - * Constructs a new ReloadSchemaShardRequest. + * Constructs a new ReparentTabletRequest. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaShardRequest. - * @implements IReloadSchemaShardRequest + * @classdesc Represents a ReparentTabletRequest. + * @implements IReparentTabletRequest * @constructor - * @param {vtctldata.IReloadSchemaShardRequest=} [properties] Properties to set + * @param {vtctldata.IReparentTabletRequest=} [properties] Properties to set */ - function ReloadSchemaShardRequest(properties) { + function ReparentTabletRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -127873,131 +134481,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReloadSchemaShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ReloadSchemaShardRequest - * @instance - */ - ReloadSchemaShardRequest.prototype.keyspace = ""; - - /** - * ReloadSchemaShardRequest shard. - * @member {string} shard - * @memberof vtctldata.ReloadSchemaShardRequest - * @instance - */ - ReloadSchemaShardRequest.prototype.shard = ""; - - /** - * ReloadSchemaShardRequest wait_position. - * @member {string} wait_position - * @memberof vtctldata.ReloadSchemaShardRequest - * @instance - */ - ReloadSchemaShardRequest.prototype.wait_position = ""; - - /** - * ReloadSchemaShardRequest include_primary. - * @member {boolean} include_primary - * @memberof vtctldata.ReloadSchemaShardRequest - * @instance - */ - ReloadSchemaShardRequest.prototype.include_primary = false; - - /** - * ReloadSchemaShardRequest concurrency. - * @member {number} concurrency - * @memberof vtctldata.ReloadSchemaShardRequest + * ReparentTabletRequest tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.ReparentTabletRequest * @instance */ - ReloadSchemaShardRequest.prototype.concurrency = 0; + ReparentTabletRequest.prototype.tablet = null; /** - * Creates a new ReloadSchemaShardRequest instance using the specified properties. + * Creates a new ReparentTabletRequest instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static - * @param {vtctldata.IReloadSchemaShardRequest=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest instance + * @param {vtctldata.IReparentTabletRequest=} [properties] Properties to set + * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest instance */ - ReloadSchemaShardRequest.create = function create(properties) { - return new ReloadSchemaShardRequest(properties); + ReparentTabletRequest.create = function create(properties) { + return new ReparentTabletRequest(properties); }; /** - * Encodes the specified ReloadSchemaShardRequest message. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. + * Encodes the specified ReparentTabletRequest message. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static - * @param {vtctldata.IReloadSchemaShardRequest} message ReloadSchemaShardRequest message or plain object to encode + * @param {vtctldata.IReparentTabletRequest} message ReparentTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaShardRequest.encode = function encode(message, writer) { + ReparentTabletRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.wait_position != null && Object.hasOwnProperty.call(message, "wait_position")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.wait_position); - if (message.include_primary != null && Object.hasOwnProperty.call(message, "include_primary")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_primary); - if (message.concurrency != null && Object.hasOwnProperty.call(message, "concurrency")) - writer.uint32(/* id 5, wireType 0 =*/40).uint32(message.concurrency); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReloadSchemaShardRequest message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardRequest.verify|verify} messages. + * Encodes the specified ReparentTabletRequest message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static - * @param {vtctldata.IReloadSchemaShardRequest} message ReloadSchemaShardRequest message or plain object to encode + * @param {vtctldata.IReparentTabletRequest} message ReparentTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + ReparentTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer. + * Decodes a ReparentTabletRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest + * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaShardRequest.decode = function decode(reader, length) { + ReparentTabletRequest.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.ReloadSchemaShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReparentTabletRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.wait_position = reader.string(); - break; - } - case 4: { - message.include_primary = reader.bool(); - break; - } - case 5: { - message.concurrency = reader.uint32(); + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -128009,156 +134561,129 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaShardRequest message from the specified reader or buffer, length delimited. + * Decodes a ReparentTabletRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest + * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaShardRequest.decodeDelimited = function decodeDelimited(reader) { + ReparentTabletRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaShardRequest message. + * Verifies a ReparentTabletRequest message. * @function verify - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaShardRequest.verify = function verify(message) { + ReparentTabletRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.wait_position != null && message.hasOwnProperty("wait_position")) - if (!$util.isString(message.wait_position)) - return "wait_position: string expected"; - if (message.include_primary != null && message.hasOwnProperty("include_primary")) - if (typeof message.include_primary !== "boolean") - return "include_primary: boolean expected"; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - if (!$util.isInteger(message.concurrency)) - return "concurrency: integer expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } return null; }; /** - * Creates a ReloadSchemaShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ReparentTabletRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaShardRequest} ReloadSchemaShardRequest + * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest */ - ReloadSchemaShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaShardRequest) + ReparentTabletRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReparentTabletRequest) return object; - let message = new $root.vtctldata.ReloadSchemaShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.wait_position != null) - message.wait_position = String(object.wait_position); - if (object.include_primary != null) - message.include_primary = Boolean(object.include_primary); - if (object.concurrency != null) - message.concurrency = object.concurrency >>> 0; + let message = new $root.vtctldata.ReparentTabletRequest(); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.ReparentTabletRequest.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } return message; }; /** - * Creates a plain object from a ReloadSchemaShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a ReparentTabletRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static - * @param {vtctldata.ReloadSchemaShardRequest} message ReloadSchemaShardRequest + * @param {vtctldata.ReparentTabletRequest} message ReparentTabletRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaShardRequest.toObject = function toObject(message, options) { + ReparentTabletRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.wait_position = ""; - object.include_primary = false; - object.concurrency = 0; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.wait_position != null && message.hasOwnProperty("wait_position")) - object.wait_position = message.wait_position; - if (message.include_primary != null && message.hasOwnProperty("include_primary")) - object.include_primary = message.include_primary; - if (message.concurrency != null && message.hasOwnProperty("concurrency")) - object.concurrency = message.concurrency; + if (options.defaults) + object.tablet = null; + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); return object; }; /** - * Converts this ReloadSchemaShardRequest to JSON. + * Converts this ReparentTabletRequest to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @instance * @returns {Object.} JSON object */ - ReloadSchemaShardRequest.prototype.toJSON = function toJSON() { + ReparentTabletRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaShardRequest + * Gets the default type url for ReparentTabletRequest * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaShardRequest + * @memberof vtctldata.ReparentTabletRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReparentTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaShardRequest"; + return typeUrlPrefix + "/vtctldata.ReparentTabletRequest"; }; - return ReloadSchemaShardRequest; + return ReparentTabletRequest; })(); - vtctldata.ReloadSchemaShardResponse = (function() { + vtctldata.ReparentTabletResponse = (function() { /** - * Properties of a ReloadSchemaShardResponse. + * Properties of a ReparentTabletResponse. * @memberof vtctldata - * @interface IReloadSchemaShardResponse - * @property {Array.|null} [events] ReloadSchemaShardResponse events + * @interface IReparentTabletResponse + * @property {string|null} [keyspace] ReparentTabletResponse keyspace + * @property {string|null} [shard] ReparentTabletResponse shard + * @property {topodata.ITabletAlias|null} [primary] ReparentTabletResponse primary */ /** - * Constructs a new ReloadSchemaShardResponse. + * Constructs a new ReparentTabletResponse. * @memberof vtctldata - * @classdesc Represents a ReloadSchemaShardResponse. - * @implements IReloadSchemaShardResponse + * @classdesc Represents a ReparentTabletResponse. + * @implements IReparentTabletResponse * @constructor - * @param {vtctldata.IReloadSchemaShardResponse=} [properties] Properties to set + * @param {vtctldata.IReparentTabletResponse=} [properties] Properties to set */ - function ReloadSchemaShardResponse(properties) { - this.events = []; + function ReparentTabletResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -128166,78 +134691,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReloadSchemaShardResponse events. - * @member {Array.} events - * @memberof vtctldata.ReloadSchemaShardResponse + * ReparentTabletResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.ReparentTabletResponse * @instance */ - ReloadSchemaShardResponse.prototype.events = $util.emptyArray; + ReparentTabletResponse.prototype.keyspace = ""; /** - * Creates a new ReloadSchemaShardResponse instance using the specified properties. + * ReparentTabletResponse shard. + * @member {string} shard + * @memberof vtctldata.ReparentTabletResponse + * @instance + */ + ReparentTabletResponse.prototype.shard = ""; + + /** + * ReparentTabletResponse primary. + * @member {topodata.ITabletAlias|null|undefined} primary + * @memberof vtctldata.ReparentTabletResponse + * @instance + */ + ReparentTabletResponse.prototype.primary = null; + + /** + * Creates a new ReparentTabletResponse instance using the specified properties. * @function create - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static - * @param {vtctldata.IReloadSchemaShardResponse=} [properties] Properties to set - * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse instance + * @param {vtctldata.IReparentTabletResponse=} [properties] Properties to set + * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse instance */ - ReloadSchemaShardResponse.create = function create(properties) { - return new ReloadSchemaShardResponse(properties); + ReparentTabletResponse.create = function create(properties) { + return new ReparentTabletResponse(properties); }; /** - * Encodes the specified ReloadSchemaShardResponse message. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. + * Encodes the specified ReparentTabletResponse message. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static - * @param {vtctldata.IReloadSchemaShardResponse} message ReloadSchemaShardResponse message or plain object to encode + * @param {vtctldata.IReparentTabletResponse} message ReparentTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaShardResponse.encode = function encode(message, writer) { + ReparentTabletResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.events != null && message.events.length) - for (let i = 0; i < message.events.length; ++i) - $root.logutil.Event.encode(message.events[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.primary != null && Object.hasOwnProperty.call(message, "primary")) + $root.topodata.TabletAlias.encode(message.primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReloadSchemaShardResponse message, length delimited. Does not implicitly {@link vtctldata.ReloadSchemaShardResponse.verify|verify} messages. + * Encodes the specified ReparentTabletResponse message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static - * @param {vtctldata.IReloadSchemaShardResponse} message ReloadSchemaShardResponse message or plain object to encode + * @param {vtctldata.IReparentTabletResponse} message ReparentTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReloadSchemaShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + ReparentTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer. + * Decodes a ReparentTabletResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse + * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaShardResponse.decode = function decode(reader, length) { + ReparentTabletResponse.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.ReloadSchemaShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ReparentTabletResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } case 2: { - if (!(message.events && message.events.length)) - message.events = []; - message.events.push($root.logutil.Event.decode(reader, reader.uint32())); + message.shard = reader.string(); + break; + } + case 3: { + message.primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -128249,141 +134799,148 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReloadSchemaShardResponse message from the specified reader or buffer, length delimited. + * Decodes a ReparentTabletResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse + * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReloadSchemaShardResponse.decodeDelimited = function decodeDelimited(reader) { + ReparentTabletResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReloadSchemaShardResponse message. + * Verifies a ReparentTabletResponse message. * @function verify - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReloadSchemaShardResponse.verify = function verify(message) { + ReparentTabletResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.events != null && message.hasOwnProperty("events")) { - if (!Array.isArray(message.events)) - return "events: array expected"; - for (let i = 0; i < message.events.length; ++i) { - let error = $root.logutil.Event.verify(message.events[i]); - if (error) - return "events." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.primary != null && message.hasOwnProperty("primary")) { + let error = $root.topodata.TabletAlias.verify(message.primary); + if (error) + return "primary." + error; } return null; }; /** - * Creates a ReloadSchemaShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ReparentTabletResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReloadSchemaShardResponse} ReloadSchemaShardResponse + * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse */ - ReloadSchemaShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReloadSchemaShardResponse) + ReparentTabletResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ReparentTabletResponse) return object; - let message = new $root.vtctldata.ReloadSchemaShardResponse(); - if (object.events) { - if (!Array.isArray(object.events)) - throw TypeError(".vtctldata.ReloadSchemaShardResponse.events: array expected"); - message.events = []; - for (let i = 0; i < object.events.length; ++i) { - if (typeof object.events[i] !== "object") - throw TypeError(".vtctldata.ReloadSchemaShardResponse.events: object expected"); - message.events[i] = $root.logutil.Event.fromObject(object.events[i]); - } + let message = new $root.vtctldata.ReparentTabletResponse(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.primary != null) { + if (typeof object.primary !== "object") + throw TypeError(".vtctldata.ReparentTabletResponse.primary: object expected"); + message.primary = $root.topodata.TabletAlias.fromObject(object.primary); } return message; }; /** - * Creates a plain object from a ReloadSchemaShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a ReparentTabletResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static - * @param {vtctldata.ReloadSchemaShardResponse} message ReloadSchemaShardResponse + * @param {vtctldata.ReparentTabletResponse} message ReparentTabletResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReloadSchemaShardResponse.toObject = function toObject(message, options) { + ReparentTabletResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.events = []; - if (message.events && message.events.length) { - object.events = []; - for (let j = 0; j < message.events.length; ++j) - object.events[j] = $root.logutil.Event.toObject(message.events[j], options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.primary = null; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.primary != null && message.hasOwnProperty("primary")) + object.primary = $root.topodata.TabletAlias.toObject(message.primary, options); return object; }; /** - * Converts this ReloadSchemaShardResponse to JSON. + * Converts this ReparentTabletResponse to JSON. * @function toJSON - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @instance * @returns {Object.} JSON object */ - ReloadSchemaShardResponse.prototype.toJSON = function toJSON() { + ReparentTabletResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReloadSchemaShardResponse + * Gets the default type url for ReparentTabletResponse * @function getTypeUrl - * @memberof vtctldata.ReloadSchemaShardResponse + * @memberof vtctldata.ReparentTabletResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReloadSchemaShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ReparentTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReloadSchemaShardResponse"; + return typeUrlPrefix + "/vtctldata.ReparentTabletResponse"; }; - return ReloadSchemaShardResponse; + return ReparentTabletResponse; })(); - vtctldata.RemoveBackupRequest = (function() { + vtctldata.RestoreFromBackupRequest = (function() { /** - * Properties of a RemoveBackupRequest. + * Properties of a RestoreFromBackupRequest. * @memberof vtctldata - * @interface IRemoveBackupRequest - * @property {string|null} [keyspace] RemoveBackupRequest keyspace - * @property {string|null} [shard] RemoveBackupRequest shard - * @property {string|null} [name] RemoveBackupRequest name + * @interface IRestoreFromBackupRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] RestoreFromBackupRequest tablet_alias + * @property {vttime.ITime|null} [backup_time] RestoreFromBackupRequest backup_time + * @property {string|null} [restore_to_pos] RestoreFromBackupRequest restore_to_pos + * @property {boolean|null} [dry_run] RestoreFromBackupRequest dry_run + * @property {vttime.ITime|null} [restore_to_timestamp] RestoreFromBackupRequest restore_to_timestamp */ /** - * Constructs a new RemoveBackupRequest. + * Constructs a new RestoreFromBackupRequest. * @memberof vtctldata - * @classdesc Represents a RemoveBackupRequest. - * @implements IRemoveBackupRequest + * @classdesc Represents a RestoreFromBackupRequest. + * @implements IRestoreFromBackupRequest * @constructor - * @param {vtctldata.IRemoveBackupRequest=} [properties] Properties to set + * @param {vtctldata.IRestoreFromBackupRequest=} [properties] Properties to set */ - function RemoveBackupRequest(properties) { + function RestoreFromBackupRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -128391,103 +134948,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RemoveBackupRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.RemoveBackupRequest + * RestoreFromBackupRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.RestoreFromBackupRequest * @instance */ - RemoveBackupRequest.prototype.keyspace = ""; + RestoreFromBackupRequest.prototype.tablet_alias = null; /** - * RemoveBackupRequest shard. - * @member {string} shard - * @memberof vtctldata.RemoveBackupRequest + * RestoreFromBackupRequest backup_time. + * @member {vttime.ITime|null|undefined} backup_time + * @memberof vtctldata.RestoreFromBackupRequest * @instance */ - RemoveBackupRequest.prototype.shard = ""; + RestoreFromBackupRequest.prototype.backup_time = null; /** - * RemoveBackupRequest name. - * @member {string} name - * @memberof vtctldata.RemoveBackupRequest + * RestoreFromBackupRequest restore_to_pos. + * @member {string} restore_to_pos + * @memberof vtctldata.RestoreFromBackupRequest * @instance */ - RemoveBackupRequest.prototype.name = ""; + RestoreFromBackupRequest.prototype.restore_to_pos = ""; /** - * Creates a new RemoveBackupRequest instance using the specified properties. + * RestoreFromBackupRequest dry_run. + * @member {boolean} dry_run + * @memberof vtctldata.RestoreFromBackupRequest + * @instance + */ + RestoreFromBackupRequest.prototype.dry_run = false; + + /** + * RestoreFromBackupRequest restore_to_timestamp. + * @member {vttime.ITime|null|undefined} restore_to_timestamp + * @memberof vtctldata.RestoreFromBackupRequest + * @instance + */ + RestoreFromBackupRequest.prototype.restore_to_timestamp = null; + + /** + * Creates a new RestoreFromBackupRequest instance using the specified properties. * @function create - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static - * @param {vtctldata.IRemoveBackupRequest=} [properties] Properties to set - * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest instance + * @param {vtctldata.IRestoreFromBackupRequest=} [properties] Properties to set + * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest instance */ - RemoveBackupRequest.create = function create(properties) { - return new RemoveBackupRequest(properties); + RestoreFromBackupRequest.create = function create(properties) { + return new RestoreFromBackupRequest(properties); }; /** - * Encodes the specified RemoveBackupRequest message. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. + * Encodes the specified RestoreFromBackupRequest message. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static - * @param {vtctldata.IRemoveBackupRequest} message RemoveBackupRequest message or plain object to encode + * @param {vtctldata.IRestoreFromBackupRequest} message RestoreFromBackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveBackupRequest.encode = function encode(message, writer) { + RestoreFromBackupRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.name); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.backup_time != null && Object.hasOwnProperty.call(message, "backup_time")) + $root.vttime.Time.encode(message.backup_time, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.restore_to_pos != null && Object.hasOwnProperty.call(message, "restore_to_pos")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.restore_to_pos); + if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.dry_run); + if (message.restore_to_timestamp != null && Object.hasOwnProperty.call(message, "restore_to_timestamp")) + $root.vttime.Time.encode(message.restore_to_timestamp, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; /** - * Encodes the specified RemoveBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupRequest.verify|verify} messages. + * Encodes the specified RestoreFromBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static - * @param {vtctldata.IRemoveBackupRequest} message RemoveBackupRequest message or plain object to encode + * @param {vtctldata.IRestoreFromBackupRequest} message RestoreFromBackupRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveBackupRequest.encodeDelimited = function encodeDelimited(message, writer) { + RestoreFromBackupRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveBackupRequest message from the specified reader or buffer. + * Decodes a RestoreFromBackupRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest + * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveBackupRequest.decode = function decode(reader, length) { + RestoreFromBackupRequest.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.RemoveBackupRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RestoreFromBackupRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.shard = reader.string(); + message.backup_time = $root.vttime.Time.decode(reader, reader.uint32()); break; } case 3: { - message.name = reader.string(); + message.restore_to_pos = reader.string(); + break; + } + case 4: { + message.dry_run = reader.bool(); + break; + } + case 5: { + message.restore_to_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); break; } default: @@ -128499,138 +135084,173 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveBackupRequest message from the specified reader or buffer, length delimited. + * Decodes a RestoreFromBackupRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest + * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveBackupRequest.decodeDelimited = function decodeDelimited(reader) { + RestoreFromBackupRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveBackupRequest message. + * Verifies a RestoreFromBackupRequest message. * @function verify - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveBackupRequest.verify = function verify(message) { + RestoreFromBackupRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.backup_time != null && message.hasOwnProperty("backup_time")) { + let error = $root.vttime.Time.verify(message.backup_time); + if (error) + return "backup_time." + error; + } + if (message.restore_to_pos != null && message.hasOwnProperty("restore_to_pos")) + if (!$util.isString(message.restore_to_pos)) + return "restore_to_pos: string expected"; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + if (typeof message.dry_run !== "boolean") + return "dry_run: boolean expected"; + if (message.restore_to_timestamp != null && message.hasOwnProperty("restore_to_timestamp")) { + let error = $root.vttime.Time.verify(message.restore_to_timestamp); + if (error) + return "restore_to_timestamp." + error; + } return null; }; /** - * Creates a RemoveBackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RestoreFromBackupRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveBackupRequest} RemoveBackupRequest + * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest */ - RemoveBackupRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveBackupRequest) + RestoreFromBackupRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RestoreFromBackupRequest) return object; - let message = new $root.vtctldata.RemoveBackupRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.name != null) - message.name = String(object.name); + let message = new $root.vtctldata.RestoreFromBackupRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.RestoreFromBackupRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.backup_time != null) { + if (typeof object.backup_time !== "object") + throw TypeError(".vtctldata.RestoreFromBackupRequest.backup_time: object expected"); + message.backup_time = $root.vttime.Time.fromObject(object.backup_time); + } + if (object.restore_to_pos != null) + message.restore_to_pos = String(object.restore_to_pos); + if (object.dry_run != null) + message.dry_run = Boolean(object.dry_run); + if (object.restore_to_timestamp != null) { + if (typeof object.restore_to_timestamp !== "object") + throw TypeError(".vtctldata.RestoreFromBackupRequest.restore_to_timestamp: object expected"); + message.restore_to_timestamp = $root.vttime.Time.fromObject(object.restore_to_timestamp); + } return message; }; /** - * Creates a plain object from a RemoveBackupRequest message. Also converts values to other types if specified. + * Creates a plain object from a RestoreFromBackupRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static - * @param {vtctldata.RemoveBackupRequest} message RemoveBackupRequest + * @param {vtctldata.RestoreFromBackupRequest} message RestoreFromBackupRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveBackupRequest.toObject = function toObject(message, options) { + RestoreFromBackupRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.name = ""; + object.tablet_alias = null; + object.backup_time = null; + object.restore_to_pos = ""; + object.dry_run = false; + object.restore_to_timestamp = null; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.backup_time != null && message.hasOwnProperty("backup_time")) + object.backup_time = $root.vttime.Time.toObject(message.backup_time, options); + if (message.restore_to_pos != null && message.hasOwnProperty("restore_to_pos")) + object.restore_to_pos = message.restore_to_pos; + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + object.dry_run = message.dry_run; + if (message.restore_to_timestamp != null && message.hasOwnProperty("restore_to_timestamp")) + object.restore_to_timestamp = $root.vttime.Time.toObject(message.restore_to_timestamp, options); return object; }; /** - * Converts this RemoveBackupRequest to JSON. + * Converts this RestoreFromBackupRequest to JSON. * @function toJSON - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @instance * @returns {Object.} JSON object */ - RemoveBackupRequest.prototype.toJSON = function toJSON() { + RestoreFromBackupRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveBackupRequest + * Gets the default type url for RestoreFromBackupRequest * @function getTypeUrl - * @memberof vtctldata.RemoveBackupRequest + * @memberof vtctldata.RestoreFromBackupRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveBackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RestoreFromBackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveBackupRequest"; + return typeUrlPrefix + "/vtctldata.RestoreFromBackupRequest"; }; - return RemoveBackupRequest; + return RestoreFromBackupRequest; })(); - vtctldata.RemoveBackupResponse = (function() { + vtctldata.RestoreFromBackupResponse = (function() { /** - * Properties of a RemoveBackupResponse. + * Properties of a RestoreFromBackupResponse. * @memberof vtctldata - * @interface IRemoveBackupResponse + * @interface IRestoreFromBackupResponse + * @property {topodata.ITabletAlias|null} [tablet_alias] RestoreFromBackupResponse tablet_alias + * @property {string|null} [keyspace] RestoreFromBackupResponse keyspace + * @property {string|null} [shard] RestoreFromBackupResponse shard + * @property {logutil.IEvent|null} [event] RestoreFromBackupResponse event */ /** - * Constructs a new RemoveBackupResponse. + * Constructs a new RestoreFromBackupResponse. * @memberof vtctldata - * @classdesc Represents a RemoveBackupResponse. - * @implements IRemoveBackupResponse + * @classdesc Represents a RestoreFromBackupResponse. + * @implements IRestoreFromBackupResponse * @constructor - * @param {vtctldata.IRemoveBackupResponse=} [properties] Properties to set + * @param {vtctldata.IRestoreFromBackupResponse=} [properties] Properties to set */ - function RemoveBackupResponse(properties) { + function RestoreFromBackupResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -128638,63 +135258,119 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RemoveBackupResponse instance using the specified properties. + * RestoreFromBackupResponse tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.RestoreFromBackupResponse + * @instance + */ + RestoreFromBackupResponse.prototype.tablet_alias = null; + + /** + * RestoreFromBackupResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.RestoreFromBackupResponse + * @instance + */ + RestoreFromBackupResponse.prototype.keyspace = ""; + + /** + * RestoreFromBackupResponse shard. + * @member {string} shard + * @memberof vtctldata.RestoreFromBackupResponse + * @instance + */ + RestoreFromBackupResponse.prototype.shard = ""; + + /** + * RestoreFromBackupResponse event. + * @member {logutil.IEvent|null|undefined} event + * @memberof vtctldata.RestoreFromBackupResponse + * @instance + */ + RestoreFromBackupResponse.prototype.event = null; + + /** + * Creates a new RestoreFromBackupResponse instance using the specified properties. * @function create - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static - * @param {vtctldata.IRemoveBackupResponse=} [properties] Properties to set - * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse instance + * @param {vtctldata.IRestoreFromBackupResponse=} [properties] Properties to set + * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse instance */ - RemoveBackupResponse.create = function create(properties) { - return new RemoveBackupResponse(properties); + RestoreFromBackupResponse.create = function create(properties) { + return new RestoreFromBackupResponse(properties); }; /** - * Encodes the specified RemoveBackupResponse message. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. + * Encodes the specified RestoreFromBackupResponse message. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static - * @param {vtctldata.IRemoveBackupResponse} message RemoveBackupResponse message or plain object to encode + * @param {vtctldata.IRestoreFromBackupResponse} message RestoreFromBackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveBackupResponse.encode = function encode(message, writer) { + RestoreFromBackupResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); + if (message.event != null && Object.hasOwnProperty.call(message, "event")) + $root.logutil.Event.encode(message.event, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified RemoveBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveBackupResponse.verify|verify} messages. + * Encodes the specified RestoreFromBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static - * @param {vtctldata.IRemoveBackupResponse} message RemoveBackupResponse message or plain object to encode + * @param {vtctldata.IRestoreFromBackupResponse} message RestoreFromBackupResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveBackupResponse.encodeDelimited = function encodeDelimited(message, writer) { + RestoreFromBackupResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveBackupResponse message from the specified reader or buffer. + * Decodes a RestoreFromBackupResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse + * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveBackupResponse.decode = function decode(reader, length) { + RestoreFromBackupResponse.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.RemoveBackupResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RestoreFromBackupResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.keyspace = reader.string(); + break; + } + case 3: { + message.shard = reader.string(); + break; + } + case 4: { + message.event = $root.logutil.Event.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -128704,112 +135380,157 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveBackupResponse message from the specified reader or buffer, length delimited. + * Decodes a RestoreFromBackupResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse + * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveBackupResponse.decodeDelimited = function decodeDelimited(reader) { + RestoreFromBackupResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveBackupResponse message. + * Verifies a RestoreFromBackupResponse message. * @function verify - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveBackupResponse.verify = function verify(message) { + RestoreFromBackupResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.event != null && message.hasOwnProperty("event")) { + let error = $root.logutil.Event.verify(message.event); + if (error) + return "event." + error; + } return null; }; /** - * Creates a RemoveBackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RestoreFromBackupResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveBackupResponse} RemoveBackupResponse + * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse */ - RemoveBackupResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveBackupResponse) + RestoreFromBackupResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RestoreFromBackupResponse) return object; - return new $root.vtctldata.RemoveBackupResponse(); + let message = new $root.vtctldata.RestoreFromBackupResponse(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.RestoreFromBackupResponse.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.event != null) { + if (typeof object.event !== "object") + throw TypeError(".vtctldata.RestoreFromBackupResponse.event: object expected"); + message.event = $root.logutil.Event.fromObject(object.event); + } + return message; }; /** - * Creates a plain object from a RemoveBackupResponse message. Also converts values to other types if specified. + * Creates a plain object from a RestoreFromBackupResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static - * @param {vtctldata.RemoveBackupResponse} message RemoveBackupResponse + * @param {vtctldata.RestoreFromBackupResponse} message RestoreFromBackupResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveBackupResponse.toObject = function toObject() { - return {}; + RestoreFromBackupResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet_alias = null; + object.keyspace = ""; + object.shard = ""; + object.event = null; + } + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.event != null && message.hasOwnProperty("event")) + object.event = $root.logutil.Event.toObject(message.event, options); + return object; }; /** - * Converts this RemoveBackupResponse to JSON. + * Converts this RestoreFromBackupResponse to JSON. * @function toJSON - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @instance * @returns {Object.} JSON object */ - RemoveBackupResponse.prototype.toJSON = function toJSON() { + RestoreFromBackupResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveBackupResponse + * Gets the default type url for RestoreFromBackupResponse * @function getTypeUrl - * @memberof vtctldata.RemoveBackupResponse + * @memberof vtctldata.RestoreFromBackupResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveBackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RestoreFromBackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveBackupResponse"; + return typeUrlPrefix + "/vtctldata.RestoreFromBackupResponse"; }; - return RemoveBackupResponse; + return RestoreFromBackupResponse; })(); - vtctldata.RemoveKeyspaceCellRequest = (function() { + vtctldata.RunHealthCheckRequest = (function() { /** - * Properties of a RemoveKeyspaceCellRequest. + * Properties of a RunHealthCheckRequest. * @memberof vtctldata - * @interface IRemoveKeyspaceCellRequest - * @property {string|null} [keyspace] RemoveKeyspaceCellRequest keyspace - * @property {string|null} [cell] RemoveKeyspaceCellRequest cell - * @property {boolean|null} [force] RemoveKeyspaceCellRequest force - * @property {boolean|null} [recursive] RemoveKeyspaceCellRequest recursive + * @interface IRunHealthCheckRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] RunHealthCheckRequest tablet_alias */ /** - * Constructs a new RemoveKeyspaceCellRequest. + * Constructs a new RunHealthCheckRequest. * @memberof vtctldata - * @classdesc Represents a RemoveKeyspaceCellRequest. - * @implements IRemoveKeyspaceCellRequest + * @classdesc Represents a RunHealthCheckRequest. + * @implements IRunHealthCheckRequest * @constructor - * @param {vtctldata.IRemoveKeyspaceCellRequest=} [properties] Properties to set + * @param {vtctldata.IRunHealthCheckRequest=} [properties] Properties to set */ - function RemoveKeyspaceCellRequest(properties) { + function RunHealthCheckRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -128817,117 +135538,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RemoveKeyspaceCellRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.RemoveKeyspaceCellRequest - * @instance - */ - RemoveKeyspaceCellRequest.prototype.keyspace = ""; - - /** - * RemoveKeyspaceCellRequest cell. - * @member {string} cell - * @memberof vtctldata.RemoveKeyspaceCellRequest - * @instance - */ - RemoveKeyspaceCellRequest.prototype.cell = ""; - - /** - * RemoveKeyspaceCellRequest force. - * @member {boolean} force - * @memberof vtctldata.RemoveKeyspaceCellRequest - * @instance - */ - RemoveKeyspaceCellRequest.prototype.force = false; - - /** - * RemoveKeyspaceCellRequest recursive. - * @member {boolean} recursive - * @memberof vtctldata.RemoveKeyspaceCellRequest + * RunHealthCheckRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.RunHealthCheckRequest * @instance */ - RemoveKeyspaceCellRequest.prototype.recursive = false; + RunHealthCheckRequest.prototype.tablet_alias = null; /** - * Creates a new RemoveKeyspaceCellRequest instance using the specified properties. + * Creates a new RunHealthCheckRequest instance using the specified properties. * @function create - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static - * @param {vtctldata.IRemoveKeyspaceCellRequest=} [properties] Properties to set - * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest instance + * @param {vtctldata.IRunHealthCheckRequest=} [properties] Properties to set + * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest instance */ - RemoveKeyspaceCellRequest.create = function create(properties) { - return new RemoveKeyspaceCellRequest(properties); + RunHealthCheckRequest.create = function create(properties) { + return new RunHealthCheckRequest(properties); }; /** - * Encodes the specified RemoveKeyspaceCellRequest message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. + * Encodes the specified RunHealthCheckRequest message. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static - * @param {vtctldata.IRemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest message or plain object to encode + * @param {vtctldata.IRunHealthCheckRequest} message RunHealthCheckRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveKeyspaceCellRequest.encode = function encode(message, writer) { + RunHealthCheckRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cell); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.force); - if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.recursive); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RemoveKeyspaceCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellRequest.verify|verify} messages. + * Encodes the specified RunHealthCheckRequest message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static - * @param {vtctldata.IRemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest message or plain object to encode + * @param {vtctldata.IRunHealthCheckRequest} message RunHealthCheckRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveKeyspaceCellRequest.encodeDelimited = function encodeDelimited(message, writer) { + RunHealthCheckRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer. + * Decodes a RunHealthCheckRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest + * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveKeyspaceCellRequest.decode = function decode(reader, length) { + RunHealthCheckRequest.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.RemoveKeyspaceCellRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RunHealthCheckRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.cell = reader.string(); - break; - } - case 3: { - message.force = reader.bool(); - break; - } - case 4: { - message.recursive = reader.bool(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -128939,146 +135618,126 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveKeyspaceCellRequest message from the specified reader or buffer, length delimited. + * Decodes a RunHealthCheckRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest + * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveKeyspaceCellRequest.decodeDelimited = function decodeDelimited(reader) { + RunHealthCheckRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveKeyspaceCellRequest message. + * Verifies a RunHealthCheckRequest message. * @function verify - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveKeyspaceCellRequest.verify = function verify(message) { + RunHealthCheckRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; - if (message.recursive != null && message.hasOwnProperty("recursive")) - if (typeof message.recursive !== "boolean") - return "recursive: boolean expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a RemoveKeyspaceCellRequest message from a plain object. Also converts values to their respective internal types. + * Creates a RunHealthCheckRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveKeyspaceCellRequest} RemoveKeyspaceCellRequest + * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest */ - RemoveKeyspaceCellRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveKeyspaceCellRequest) - return object; - let message = new $root.vtctldata.RemoveKeyspaceCellRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.cell != null) - message.cell = String(object.cell); - if (object.force != null) - message.force = Boolean(object.force); - if (object.recursive != null) - message.recursive = Boolean(object.recursive); + RunHealthCheckRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RunHealthCheckRequest) + return object; + let message = new $root.vtctldata.RunHealthCheckRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.RunHealthCheckRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a RemoveKeyspaceCellRequest message. Also converts values to other types if specified. + * Creates a plain object from a RunHealthCheckRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static - * @param {vtctldata.RemoveKeyspaceCellRequest} message RemoveKeyspaceCellRequest + * @param {vtctldata.RunHealthCheckRequest} message RunHealthCheckRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveKeyspaceCellRequest.toObject = function toObject(message, options) { + RunHealthCheckRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.cell = ""; - object.force = false; - object.recursive = false; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; - if (message.recursive != null && message.hasOwnProperty("recursive")) - object.recursive = message.recursive; + if (options.defaults) + object.tablet_alias = null; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this RemoveKeyspaceCellRequest to JSON. + * Converts this RunHealthCheckRequest to JSON. * @function toJSON - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @instance * @returns {Object.} JSON object */ - RemoveKeyspaceCellRequest.prototype.toJSON = function toJSON() { + RunHealthCheckRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveKeyspaceCellRequest + * Gets the default type url for RunHealthCheckRequest * @function getTypeUrl - * @memberof vtctldata.RemoveKeyspaceCellRequest + * @memberof vtctldata.RunHealthCheckRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveKeyspaceCellRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RunHealthCheckRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveKeyspaceCellRequest"; + return typeUrlPrefix + "/vtctldata.RunHealthCheckRequest"; }; - return RemoveKeyspaceCellRequest; + return RunHealthCheckRequest; })(); - vtctldata.RemoveKeyspaceCellResponse = (function() { + vtctldata.RunHealthCheckResponse = (function() { /** - * Properties of a RemoveKeyspaceCellResponse. + * Properties of a RunHealthCheckResponse. * @memberof vtctldata - * @interface IRemoveKeyspaceCellResponse + * @interface IRunHealthCheckResponse */ /** - * Constructs a new RemoveKeyspaceCellResponse. + * Constructs a new RunHealthCheckResponse. * @memberof vtctldata - * @classdesc Represents a RemoveKeyspaceCellResponse. - * @implements IRemoveKeyspaceCellResponse + * @classdesc Represents a RunHealthCheckResponse. + * @implements IRunHealthCheckResponse * @constructor - * @param {vtctldata.IRemoveKeyspaceCellResponse=} [properties] Properties to set + * @param {vtctldata.IRunHealthCheckResponse=} [properties] Properties to set */ - function RemoveKeyspaceCellResponse(properties) { + function RunHealthCheckResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -129086,60 +135745,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RemoveKeyspaceCellResponse instance using the specified properties. + * Creates a new RunHealthCheckResponse instance using the specified properties. * @function create - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static - * @param {vtctldata.IRemoveKeyspaceCellResponse=} [properties] Properties to set - * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse instance + * @param {vtctldata.IRunHealthCheckResponse=} [properties] Properties to set + * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse instance */ - RemoveKeyspaceCellResponse.create = function create(properties) { - return new RemoveKeyspaceCellResponse(properties); + RunHealthCheckResponse.create = function create(properties) { + return new RunHealthCheckResponse(properties); }; /** - * Encodes the specified RemoveKeyspaceCellResponse message. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. + * Encodes the specified RunHealthCheckResponse message. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static - * @param {vtctldata.IRemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse message or plain object to encode + * @param {vtctldata.IRunHealthCheckResponse} message RunHealthCheckResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveKeyspaceCellResponse.encode = function encode(message, writer) { + RunHealthCheckResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified RemoveKeyspaceCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveKeyspaceCellResponse.verify|verify} messages. + * Encodes the specified RunHealthCheckResponse message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static - * @param {vtctldata.IRemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse message or plain object to encode + * @param {vtctldata.IRunHealthCheckResponse} message RunHealthCheckResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveKeyspaceCellResponse.encodeDelimited = function encodeDelimited(message, writer) { + RunHealthCheckResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer. + * Decodes a RunHealthCheckResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse + * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveKeyspaceCellResponse.decode = function decode(reader, length) { + RunHealthCheckResponse.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.RemoveKeyspaceCellResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.RunHealthCheckResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -129152,113 +135811,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveKeyspaceCellResponse message from the specified reader or buffer, length delimited. + * Decodes a RunHealthCheckResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse + * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveKeyspaceCellResponse.decodeDelimited = function decodeDelimited(reader) { + RunHealthCheckResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveKeyspaceCellResponse message. + * Verifies a RunHealthCheckResponse message. * @function verify - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveKeyspaceCellResponse.verify = function verify(message) { + RunHealthCheckResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a RemoveKeyspaceCellResponse message from a plain object. Also converts values to their respective internal types. + * Creates a RunHealthCheckResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveKeyspaceCellResponse} RemoveKeyspaceCellResponse + * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse */ - RemoveKeyspaceCellResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveKeyspaceCellResponse) + RunHealthCheckResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.RunHealthCheckResponse) return object; - return new $root.vtctldata.RemoveKeyspaceCellResponse(); + return new $root.vtctldata.RunHealthCheckResponse(); }; /** - * Creates a plain object from a RemoveKeyspaceCellResponse message. Also converts values to other types if specified. + * Creates a plain object from a RunHealthCheckResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static - * @param {vtctldata.RemoveKeyspaceCellResponse} message RemoveKeyspaceCellResponse + * @param {vtctldata.RunHealthCheckResponse} message RunHealthCheckResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveKeyspaceCellResponse.toObject = function toObject() { + RunHealthCheckResponse.toObject = function toObject() { return {}; }; /** - * Converts this RemoveKeyspaceCellResponse to JSON. + * Converts this RunHealthCheckResponse to JSON. * @function toJSON - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @instance * @returns {Object.} JSON object */ - RemoveKeyspaceCellResponse.prototype.toJSON = function toJSON() { + RunHealthCheckResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveKeyspaceCellResponse + * Gets the default type url for RunHealthCheckResponse * @function getTypeUrl - * @memberof vtctldata.RemoveKeyspaceCellResponse + * @memberof vtctldata.RunHealthCheckResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveKeyspaceCellResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + RunHealthCheckResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveKeyspaceCellResponse"; + return typeUrlPrefix + "/vtctldata.RunHealthCheckResponse"; }; - return RemoveKeyspaceCellResponse; + return RunHealthCheckResponse; })(); - vtctldata.RemoveShardCellRequest = (function() { + vtctldata.SetKeyspaceDurabilityPolicyRequest = (function() { /** - * Properties of a RemoveShardCellRequest. + * Properties of a SetKeyspaceDurabilityPolicyRequest. * @memberof vtctldata - * @interface IRemoveShardCellRequest - * @property {string|null} [keyspace] RemoveShardCellRequest keyspace - * @property {string|null} [shard_name] RemoveShardCellRequest shard_name - * @property {string|null} [cell] RemoveShardCellRequest cell - * @property {boolean|null} [force] RemoveShardCellRequest force - * @property {boolean|null} [recursive] RemoveShardCellRequest recursive + * @interface ISetKeyspaceDurabilityPolicyRequest + * @property {string|null} [keyspace] SetKeyspaceDurabilityPolicyRequest keyspace + * @property {string|null} [durability_policy] SetKeyspaceDurabilityPolicyRequest durability_policy */ /** - * Constructs a new RemoveShardCellRequest. + * Constructs a new SetKeyspaceDurabilityPolicyRequest. * @memberof vtctldata - * @classdesc Represents a RemoveShardCellRequest. - * @implements IRemoveShardCellRequest + * @classdesc Represents a SetKeyspaceDurabilityPolicyRequest. + * @implements ISetKeyspaceDurabilityPolicyRequest * @constructor - * @param {vtctldata.IRemoveShardCellRequest=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest=} [properties] Properties to set */ - function RemoveShardCellRequest(properties) { + function SetKeyspaceDurabilityPolicyRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -129266,110 +135922,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RemoveShardCellRequest keyspace. + * SetKeyspaceDurabilityPolicyRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.RemoveShardCellRequest - * @instance - */ - RemoveShardCellRequest.prototype.keyspace = ""; - - /** - * RemoveShardCellRequest shard_name. - * @member {string} shard_name - * @memberof vtctldata.RemoveShardCellRequest - * @instance - */ - RemoveShardCellRequest.prototype.shard_name = ""; - - /** - * RemoveShardCellRequest cell. - * @member {string} cell - * @memberof vtctldata.RemoveShardCellRequest - * @instance - */ - RemoveShardCellRequest.prototype.cell = ""; - - /** - * RemoveShardCellRequest force. - * @member {boolean} force - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @instance */ - RemoveShardCellRequest.prototype.force = false; + SetKeyspaceDurabilityPolicyRequest.prototype.keyspace = ""; /** - * RemoveShardCellRequest recursive. - * @member {boolean} recursive - * @memberof vtctldata.RemoveShardCellRequest + * SetKeyspaceDurabilityPolicyRequest durability_policy. + * @member {string} durability_policy + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @instance */ - RemoveShardCellRequest.prototype.recursive = false; + SetKeyspaceDurabilityPolicyRequest.prototype.durability_policy = ""; /** - * Creates a new RemoveShardCellRequest instance using the specified properties. + * Creates a new SetKeyspaceDurabilityPolicyRequest instance using the specified properties. * @function create - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static - * @param {vtctldata.IRemoveShardCellRequest=} [properties] Properties to set - * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest instance + * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest instance */ - RemoveShardCellRequest.create = function create(properties) { - return new RemoveShardCellRequest(properties); + SetKeyspaceDurabilityPolicyRequest.create = function create(properties) { + return new SetKeyspaceDurabilityPolicyRequest(properties); }; /** - * Encodes the specified RemoveShardCellRequest message. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceDurabilityPolicyRequest message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static - * @param {vtctldata.IRemoveShardCellRequest} message RemoveShardCellRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveShardCellRequest.encode = function encode(message, writer) { + SetKeyspaceDurabilityPolicyRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard_name != null && Object.hasOwnProperty.call(message, "shard_name")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard_name); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cell); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); - if (message.recursive != null && Object.hasOwnProperty.call(message, "recursive")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.recursive); + if (message.durability_policy != null && Object.hasOwnProperty.call(message, "durability_policy")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.durability_policy); return writer; }; /** - * Encodes the specified RemoveShardCellRequest message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceDurabilityPolicyRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static - * @param {vtctldata.IRemoveShardCellRequest} message RemoveShardCellRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveShardCellRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceDurabilityPolicyRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveShardCellRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest + * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveShardCellRequest.decode = function decode(reader, length) { + SetKeyspaceDurabilityPolicyRequest.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.RemoveShardCellRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceDurabilityPolicyRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -129378,19 +136004,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.shard_name = reader.string(); - break; - } - case 3: { - message.cell = reader.string(); - break; - } - case 4: { - message.force = reader.bool(); - break; - } - case 5: { - message.recursive = reader.bool(); + message.durability_policy = reader.string(); break; } default: @@ -129402,154 +136016,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveShardCellRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest + * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveShardCellRequest.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceDurabilityPolicyRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveShardCellRequest message. + * Verifies a SetKeyspaceDurabilityPolicyRequest message. * @function verify - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveShardCellRequest.verify = function verify(message) { + SetKeyspaceDurabilityPolicyRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - if (!$util.isString(message.shard_name)) - return "shard_name: string expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; - if (message.recursive != null && message.hasOwnProperty("recursive")) - if (typeof message.recursive !== "boolean") - return "recursive: boolean expected"; + if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) + if (!$util.isString(message.durability_policy)) + return "durability_policy: string expected"; return null; }; /** - * Creates a RemoveShardCellRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceDurabilityPolicyRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveShardCellRequest} RemoveShardCellRequest + * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest */ - RemoveShardCellRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveShardCellRequest) + SetKeyspaceDurabilityPolicyRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceDurabilityPolicyRequest) return object; - let message = new $root.vtctldata.RemoveShardCellRequest(); + let message = new $root.vtctldata.SetKeyspaceDurabilityPolicyRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.shard_name != null) - message.shard_name = String(object.shard_name); - if (object.cell != null) - message.cell = String(object.cell); - if (object.force != null) - message.force = Boolean(object.force); - if (object.recursive != null) - message.recursive = Boolean(object.recursive); + if (object.durability_policy != null) + message.durability_policy = String(object.durability_policy); return message; }; /** - * Creates a plain object from a RemoveShardCellRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceDurabilityPolicyRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static - * @param {vtctldata.RemoveShardCellRequest} message RemoveShardCellRequest + * @param {vtctldata.SetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveShardCellRequest.toObject = function toObject(message, options) { + SetKeyspaceDurabilityPolicyRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; - object.shard_name = ""; - object.cell = ""; - object.force = false; - object.recursive = false; + object.durability_policy = ""; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.shard_name != null && message.hasOwnProperty("shard_name")) - object.shard_name = message.shard_name; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; - if (message.recursive != null && message.hasOwnProperty("recursive")) - object.recursive = message.recursive; + if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) + object.durability_policy = message.durability_policy; return object; }; /** - * Converts this RemoveShardCellRequest to JSON. + * Converts this SetKeyspaceDurabilityPolicyRequest to JSON. * @function toJSON - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @instance * @returns {Object.} JSON object */ - RemoveShardCellRequest.prototype.toJSON = function toJSON() { + SetKeyspaceDurabilityPolicyRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveShardCellRequest + * Gets the default type url for SetKeyspaceDurabilityPolicyRequest * @function getTypeUrl - * @memberof vtctldata.RemoveShardCellRequest + * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveShardCellRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceDurabilityPolicyRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveShardCellRequest"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceDurabilityPolicyRequest"; }; - return RemoveShardCellRequest; + return SetKeyspaceDurabilityPolicyRequest; })(); - vtctldata.RemoveShardCellResponse = (function() { + vtctldata.SetKeyspaceDurabilityPolicyResponse = (function() { /** - * Properties of a RemoveShardCellResponse. + * Properties of a SetKeyspaceDurabilityPolicyResponse. * @memberof vtctldata - * @interface IRemoveShardCellResponse + * @interface ISetKeyspaceDurabilityPolicyResponse + * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceDurabilityPolicyResponse keyspace */ /** - * Constructs a new RemoveShardCellResponse. + * Constructs a new SetKeyspaceDurabilityPolicyResponse. * @memberof vtctldata - * @classdesc Represents a RemoveShardCellResponse. - * @implements IRemoveShardCellResponse + * @classdesc Represents a SetKeyspaceDurabilityPolicyResponse. + * @implements ISetKeyspaceDurabilityPolicyResponse * @constructor - * @param {vtctldata.IRemoveShardCellResponse=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse=} [properties] Properties to set */ - function RemoveShardCellResponse(properties) { + function SetKeyspaceDurabilityPolicyResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -129557,63 +136148,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RemoveShardCellResponse instance using the specified properties. + * SetKeyspaceDurabilityPolicyResponse keyspace. + * @member {topodata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @instance + */ + SetKeyspaceDurabilityPolicyResponse.prototype.keyspace = null; + + /** + * Creates a new SetKeyspaceDurabilityPolicyResponse instance using the specified properties. * @function create - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static - * @param {vtctldata.IRemoveShardCellResponse=} [properties] Properties to set - * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse instance + * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse instance */ - RemoveShardCellResponse.create = function create(properties) { - return new RemoveShardCellResponse(properties); + SetKeyspaceDurabilityPolicyResponse.create = function create(properties) { + return new SetKeyspaceDurabilityPolicyResponse(properties); }; /** - * Encodes the specified RemoveShardCellResponse message. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceDurabilityPolicyResponse message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static - * @param {vtctldata.IRemoveShardCellResponse} message RemoveShardCellResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveShardCellResponse.encode = function encode(message, writer) { + SetKeyspaceDurabilityPolicyResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RemoveShardCellResponse message, length delimited. Does not implicitly {@link vtctldata.RemoveShardCellResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceDurabilityPolicyResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static - * @param {vtctldata.IRemoveShardCellResponse} message RemoveShardCellResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RemoveShardCellResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceDurabilityPolicyResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RemoveShardCellResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse + * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveShardCellResponse.decode = function decode(reader, length) { + SetKeyspaceDurabilityPolicyResponse.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.RemoveShardCellResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceDurabilityPolicyResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -129623,109 +136228,132 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RemoveShardCellResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse + * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RemoveShardCellResponse.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceDurabilityPolicyResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RemoveShardCellResponse message. + * Verifies a SetKeyspaceDurabilityPolicyResponse message. * @function verify - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RemoveShardCellResponse.verify = function verify(message) { + SetKeyspaceDurabilityPolicyResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.topodata.Keyspace.verify(message.keyspace); + if (error) + return "keyspace." + error; + } return null; }; /** - * Creates a RemoveShardCellResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceDurabilityPolicyResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RemoveShardCellResponse} RemoveShardCellResponse + * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse */ - RemoveShardCellResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RemoveShardCellResponse) + SetKeyspaceDurabilityPolicyResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceDurabilityPolicyResponse) return object; - return new $root.vtctldata.RemoveShardCellResponse(); + let message = new $root.vtctldata.SetKeyspaceDurabilityPolicyResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace: object expected"); + message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); + } + return message; }; /** - * Creates a plain object from a RemoveShardCellResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceDurabilityPolicyResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static - * @param {vtctldata.RemoveShardCellResponse} message RemoveShardCellResponse + * @param {vtctldata.SetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RemoveShardCellResponse.toObject = function toObject() { - return {}; + SetKeyspaceDurabilityPolicyResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.keyspace = null; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); + return object; }; /** - * Converts this RemoveShardCellResponse to JSON. + * Converts this SetKeyspaceDurabilityPolicyResponse to JSON. * @function toJSON - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @instance * @returns {Object.} JSON object */ - RemoveShardCellResponse.prototype.toJSON = function toJSON() { + SetKeyspaceDurabilityPolicyResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RemoveShardCellResponse + * Gets the default type url for SetKeyspaceDurabilityPolicyResponse * @function getTypeUrl - * @memberof vtctldata.RemoveShardCellResponse + * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RemoveShardCellResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceDurabilityPolicyResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RemoveShardCellResponse"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceDurabilityPolicyResponse"; }; - return RemoveShardCellResponse; + return SetKeyspaceDurabilityPolicyResponse; })(); - vtctldata.ReparentTabletRequest = (function() { + vtctldata.SetKeyspaceServedFromRequest = (function() { /** - * Properties of a ReparentTabletRequest. + * Properties of a SetKeyspaceServedFromRequest. * @memberof vtctldata - * @interface IReparentTabletRequest - * @property {topodata.ITabletAlias|null} [tablet] ReparentTabletRequest tablet + * @interface ISetKeyspaceServedFromRequest + * @property {string|null} [keyspace] SetKeyspaceServedFromRequest keyspace + * @property {topodata.TabletType|null} [tablet_type] SetKeyspaceServedFromRequest tablet_type + * @property {Array.|null} [cells] SetKeyspaceServedFromRequest cells + * @property {boolean|null} [remove] SetKeyspaceServedFromRequest remove + * @property {string|null} [source_keyspace] SetKeyspaceServedFromRequest source_keyspace */ /** - * Constructs a new ReparentTabletRequest. + * Constructs a new SetKeyspaceServedFromRequest. * @memberof vtctldata - * @classdesc Represents a ReparentTabletRequest. - * @implements IReparentTabletRequest + * @classdesc Represents a SetKeyspaceServedFromRequest. + * @implements ISetKeyspaceServedFromRequest * @constructor - * @param {vtctldata.IReparentTabletRequest=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceServedFromRequest=} [properties] Properties to set */ - function ReparentTabletRequest(properties) { + function SetKeyspaceServedFromRequest(properties) { + this.cells = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -129733,75 +136361,134 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReparentTabletRequest tablet. - * @member {topodata.ITabletAlias|null|undefined} tablet - * @memberof vtctldata.ReparentTabletRequest + * SetKeyspaceServedFromRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.SetKeyspaceServedFromRequest * @instance */ - ReparentTabletRequest.prototype.tablet = null; + SetKeyspaceServedFromRequest.prototype.keyspace = ""; /** - * Creates a new ReparentTabletRequest instance using the specified properties. + * SetKeyspaceServedFromRequest tablet_type. + * @member {topodata.TabletType} tablet_type + * @memberof vtctldata.SetKeyspaceServedFromRequest + * @instance + */ + SetKeyspaceServedFromRequest.prototype.tablet_type = 0; + + /** + * SetKeyspaceServedFromRequest cells. + * @member {Array.} cells + * @memberof vtctldata.SetKeyspaceServedFromRequest + * @instance + */ + SetKeyspaceServedFromRequest.prototype.cells = $util.emptyArray; + + /** + * SetKeyspaceServedFromRequest remove. + * @member {boolean} remove + * @memberof vtctldata.SetKeyspaceServedFromRequest + * @instance + */ + SetKeyspaceServedFromRequest.prototype.remove = false; + + /** + * SetKeyspaceServedFromRequest source_keyspace. + * @member {string} source_keyspace + * @memberof vtctldata.SetKeyspaceServedFromRequest + * @instance + */ + SetKeyspaceServedFromRequest.prototype.source_keyspace = ""; + + /** + * Creates a new SetKeyspaceServedFromRequest instance using the specified properties. * @function create - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static - * @param {vtctldata.IReparentTabletRequest=} [properties] Properties to set - * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest instance + * @param {vtctldata.ISetKeyspaceServedFromRequest=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest instance */ - ReparentTabletRequest.create = function create(properties) { - return new ReparentTabletRequest(properties); + SetKeyspaceServedFromRequest.create = function create(properties) { + return new SetKeyspaceServedFromRequest(properties); }; /** - * Encodes the specified ReparentTabletRequest message. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceServedFromRequest message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static - * @param {vtctldata.IReparentTabletRequest} message ReparentTabletRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReparentTabletRequest.encode = function encode(message, writer) { + SetKeyspaceServedFromRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) - $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.tablet_type); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); + if (message.remove != null && Object.hasOwnProperty.call(message, "remove")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.remove); + if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_keyspace); return writer; }; /** - * Encodes the specified ReparentTabletRequest message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceServedFromRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static - * @param {vtctldata.IReparentTabletRequest} message ReparentTabletRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReparentTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceServedFromRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReparentTabletRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest + * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReparentTabletRequest.decode = function decode(reader, length) { + SetKeyspaceServedFromRequest.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.ReparentTabletRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceServedFromRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.tablet_type = reader.int32(); + break; + } + case 3: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 4: { + message.remove = reader.bool(); + break; + } + case 5: { + message.source_keyspace = reader.string(); break; } default: @@ -129813,129 +136500,232 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReparentTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest + * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReparentTabletRequest.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceServedFromRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReparentTabletRequest message. + * Verifies a SetKeyspaceServedFromRequest message. * @function verify - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReparentTabletRequest.verify = function verify(message) { + SetKeyspaceServedFromRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet != null && message.hasOwnProperty("tablet")) { - let error = $root.topodata.TabletAlias.verify(message.tablet); - if (error) - return "tablet." + error; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + switch (message.tablet_type) { + default: + return "tablet_type: enum value expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; } + if (message.remove != null && message.hasOwnProperty("remove")) + if (typeof message.remove !== "boolean") + return "remove: boolean expected"; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + if (!$util.isString(message.source_keyspace)) + return "source_keyspace: string expected"; return null; }; /** - * Creates a ReparentTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceServedFromRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReparentTabletRequest} ReparentTabletRequest + * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest */ - ReparentTabletRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReparentTabletRequest) + SetKeyspaceServedFromRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceServedFromRequest) return object; - let message = new $root.vtctldata.ReparentTabletRequest(); - if (object.tablet != null) { - if (typeof object.tablet !== "object") - throw TypeError(".vtctldata.ReparentTabletRequest.tablet: object expected"); - message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + let message = new $root.vtctldata.SetKeyspaceServedFromRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + switch (object.tablet_type) { + default: + if (typeof object.tablet_type === "number") { + message.tablet_type = object.tablet_type; + break; + } + break; + case "UNKNOWN": + case 0: + message.tablet_type = 0; + break; + case "PRIMARY": + case 1: + message.tablet_type = 1; + break; + case "MASTER": + case 1: + message.tablet_type = 1; + break; + case "REPLICA": + case 2: + message.tablet_type = 2; + break; + case "RDONLY": + case 3: + message.tablet_type = 3; + break; + case "BATCH": + case 3: + message.tablet_type = 3; + break; + case "SPARE": + case 4: + message.tablet_type = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_type = 5; + break; + case "BACKUP": + case 6: + message.tablet_type = 6; + break; + case "RESTORE": + case 7: + message.tablet_type = 7; + break; + case "DRAINED": + case 8: + message.tablet_type = 8; + break; + } + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.SetKeyspaceServedFromRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); } + if (object.remove != null) + message.remove = Boolean(object.remove); + if (object.source_keyspace != null) + message.source_keyspace = String(object.source_keyspace); return message; }; /** - * Creates a plain object from a ReparentTabletRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceServedFromRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static - * @param {vtctldata.ReparentTabletRequest} message ReparentTabletRequest + * @param {vtctldata.SetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReparentTabletRequest.toObject = function toObject(message, options) { + SetKeyspaceServedFromRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet = null; - if (message.tablet != null && message.hasOwnProperty("tablet")) - object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (options.arrays || options.defaults) + object.cells = []; + if (options.defaults) { + object.keyspace = ""; + object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + object.remove = false; + object.source_keyspace = ""; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.remove != null && message.hasOwnProperty("remove")) + object.remove = message.remove; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + object.source_keyspace = message.source_keyspace; return object; }; /** - * Converts this ReparentTabletRequest to JSON. + * Converts this SetKeyspaceServedFromRequest to JSON. * @function toJSON - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @instance * @returns {Object.} JSON object */ - ReparentTabletRequest.prototype.toJSON = function toJSON() { + SetKeyspaceServedFromRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReparentTabletRequest + * Gets the default type url for SetKeyspaceServedFromRequest * @function getTypeUrl - * @memberof vtctldata.ReparentTabletRequest + * @memberof vtctldata.SetKeyspaceServedFromRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReparentTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceServedFromRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReparentTabletRequest"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceServedFromRequest"; }; - return ReparentTabletRequest; + return SetKeyspaceServedFromRequest; })(); - vtctldata.ReparentTabletResponse = (function() { + vtctldata.SetKeyspaceServedFromResponse = (function() { /** - * Properties of a ReparentTabletResponse. + * Properties of a SetKeyspaceServedFromResponse. * @memberof vtctldata - * @interface IReparentTabletResponse - * @property {string|null} [keyspace] ReparentTabletResponse keyspace - * @property {string|null} [shard] ReparentTabletResponse shard - * @property {topodata.ITabletAlias|null} [primary] ReparentTabletResponse primary + * @interface ISetKeyspaceServedFromResponse + * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceServedFromResponse keyspace */ /** - * Constructs a new ReparentTabletResponse. + * Constructs a new SetKeyspaceServedFromResponse. * @memberof vtctldata - * @classdesc Represents a ReparentTabletResponse. - * @implements IReparentTabletResponse + * @classdesc Represents a SetKeyspaceServedFromResponse. + * @implements ISetKeyspaceServedFromResponse * @constructor - * @param {vtctldata.IReparentTabletResponse=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceServedFromResponse=} [properties] Properties to set */ - function ReparentTabletResponse(properties) { + function SetKeyspaceServedFromResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -129943,103 +136733,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ReparentTabletResponse keyspace. - * @member {string} keyspace - * @memberof vtctldata.ReparentTabletResponse - * @instance - */ - ReparentTabletResponse.prototype.keyspace = ""; - - /** - * ReparentTabletResponse shard. - * @member {string} shard - * @memberof vtctldata.ReparentTabletResponse - * @instance - */ - ReparentTabletResponse.prototype.shard = ""; - - /** - * ReparentTabletResponse primary. - * @member {topodata.ITabletAlias|null|undefined} primary - * @memberof vtctldata.ReparentTabletResponse + * SetKeyspaceServedFromResponse keyspace. + * @member {topodata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.SetKeyspaceServedFromResponse * @instance */ - ReparentTabletResponse.prototype.primary = null; + SetKeyspaceServedFromResponse.prototype.keyspace = null; /** - * Creates a new ReparentTabletResponse instance using the specified properties. + * Creates a new SetKeyspaceServedFromResponse instance using the specified properties. * @function create - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static - * @param {vtctldata.IReparentTabletResponse=} [properties] Properties to set - * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse instance + * @param {vtctldata.ISetKeyspaceServedFromResponse=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse instance */ - ReparentTabletResponse.create = function create(properties) { - return new ReparentTabletResponse(properties); + SetKeyspaceServedFromResponse.create = function create(properties) { + return new SetKeyspaceServedFromResponse(properties); }; /** - * Encodes the specified ReparentTabletResponse message. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceServedFromResponse message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static - * @param {vtctldata.IReparentTabletResponse} message ReparentTabletResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReparentTabletResponse.encode = function encode(message, writer) { + SetKeyspaceServedFromResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.primary != null && Object.hasOwnProperty.call(message, "primary")) - $root.topodata.TabletAlias.encode(message.primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ReparentTabletResponse message, length delimited. Does not implicitly {@link vtctldata.ReparentTabletResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceServedFromResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static - * @param {vtctldata.IReparentTabletResponse} message ReparentTabletResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReparentTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceServedFromResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReparentTabletResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse + * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReparentTabletResponse.decode = function decode(reader, length) { + SetKeyspaceServedFromResponse.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.ReparentTabletResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceServedFromResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); break; } default: @@ -130051,148 +136813,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ReparentTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse + * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReparentTabletResponse.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceServedFromResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReparentTabletResponse message. + * Verifies a SetKeyspaceServedFromResponse message. * @function verify - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReparentTabletResponse.verify = function verify(message) { + SetKeyspaceServedFromResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.primary != null && message.hasOwnProperty("primary")) { - let error = $root.topodata.TabletAlias.verify(message.primary); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.topodata.Keyspace.verify(message.keyspace); if (error) - return "primary." + error; + return "keyspace." + error; } return null; }; /** - * Creates a ReparentTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceServedFromResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ReparentTabletResponse} ReparentTabletResponse + * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse */ - ReparentTabletResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ReparentTabletResponse) + SetKeyspaceServedFromResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceServedFromResponse) return object; - let message = new $root.vtctldata.ReparentTabletResponse(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.primary != null) { - if (typeof object.primary !== "object") - throw TypeError(".vtctldata.ReparentTabletResponse.primary: object expected"); - message.primary = $root.topodata.TabletAlias.fromObject(object.primary); + let message = new $root.vtctldata.SetKeyspaceServedFromResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.SetKeyspaceServedFromResponse.keyspace: object expected"); + message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); } return message; }; /** - * Creates a plain object from a ReparentTabletResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceServedFromResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static - * @param {vtctldata.ReparentTabletResponse} message ReparentTabletResponse + * @param {vtctldata.SetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReparentTabletResponse.toObject = function toObject(message, options) { + SetKeyspaceServedFromResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.primary = null; - } + if (options.defaults) + object.keyspace = null; if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.primary != null && message.hasOwnProperty("primary")) - object.primary = $root.topodata.TabletAlias.toObject(message.primary, options); + object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); return object; }; /** - * Converts this ReparentTabletResponse to JSON. + * Converts this SetKeyspaceServedFromResponse to JSON. * @function toJSON - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @instance * @returns {Object.} JSON object */ - ReparentTabletResponse.prototype.toJSON = function toJSON() { + SetKeyspaceServedFromResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReparentTabletResponse + * Gets the default type url for SetKeyspaceServedFromResponse * @function getTypeUrl - * @memberof vtctldata.ReparentTabletResponse + * @memberof vtctldata.SetKeyspaceServedFromResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReparentTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceServedFromResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ReparentTabletResponse"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceServedFromResponse"; }; - return ReparentTabletResponse; + return SetKeyspaceServedFromResponse; })(); - vtctldata.RestoreFromBackupRequest = (function() { + vtctldata.SetKeyspaceShardingInfoRequest = (function() { /** - * Properties of a RestoreFromBackupRequest. + * Properties of a SetKeyspaceShardingInfoRequest. * @memberof vtctldata - * @interface IRestoreFromBackupRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] RestoreFromBackupRequest tablet_alias - * @property {vttime.ITime|null} [backup_time] RestoreFromBackupRequest backup_time - * @property {string|null} [restore_to_pos] RestoreFromBackupRequest restore_to_pos - * @property {boolean|null} [dry_run] RestoreFromBackupRequest dry_run - * @property {vttime.ITime|null} [restore_to_timestamp] RestoreFromBackupRequest restore_to_timestamp + * @interface ISetKeyspaceShardingInfoRequest + * @property {string|null} [keyspace] SetKeyspaceShardingInfoRequest keyspace + * @property {boolean|null} [force] SetKeyspaceShardingInfoRequest force */ /** - * Constructs a new RestoreFromBackupRequest. + * Constructs a new SetKeyspaceShardingInfoRequest. * @memberof vtctldata - * @classdesc Represents a RestoreFromBackupRequest. - * @implements IRestoreFromBackupRequest + * @classdesc Represents a SetKeyspaceShardingInfoRequest. + * @implements ISetKeyspaceShardingInfoRequest * @constructor - * @param {vtctldata.IRestoreFromBackupRequest=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceShardingInfoRequest=} [properties] Properties to set */ - function RestoreFromBackupRequest(properties) { + function SetKeyspaceShardingInfoRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -130200,131 +136942,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RestoreFromBackupRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.RestoreFromBackupRequest - * @instance - */ - RestoreFromBackupRequest.prototype.tablet_alias = null; - - /** - * RestoreFromBackupRequest backup_time. - * @member {vttime.ITime|null|undefined} backup_time - * @memberof vtctldata.RestoreFromBackupRequest - * @instance - */ - RestoreFromBackupRequest.prototype.backup_time = null; - - /** - * RestoreFromBackupRequest restore_to_pos. - * @member {string} restore_to_pos - * @memberof vtctldata.RestoreFromBackupRequest - * @instance - */ - RestoreFromBackupRequest.prototype.restore_to_pos = ""; - - /** - * RestoreFromBackupRequest dry_run. - * @member {boolean} dry_run - * @memberof vtctldata.RestoreFromBackupRequest + * SetKeyspaceShardingInfoRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @instance */ - RestoreFromBackupRequest.prototype.dry_run = false; + SetKeyspaceShardingInfoRequest.prototype.keyspace = ""; /** - * RestoreFromBackupRequest restore_to_timestamp. - * @member {vttime.ITime|null|undefined} restore_to_timestamp - * @memberof vtctldata.RestoreFromBackupRequest + * SetKeyspaceShardingInfoRequest force. + * @member {boolean} force + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @instance */ - RestoreFromBackupRequest.prototype.restore_to_timestamp = null; + SetKeyspaceShardingInfoRequest.prototype.force = false; /** - * Creates a new RestoreFromBackupRequest instance using the specified properties. + * Creates a new SetKeyspaceShardingInfoRequest instance using the specified properties. * @function create - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static - * @param {vtctldata.IRestoreFromBackupRequest=} [properties] Properties to set - * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest instance + * @param {vtctldata.ISetKeyspaceShardingInfoRequest=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest instance */ - RestoreFromBackupRequest.create = function create(properties) { - return new RestoreFromBackupRequest(properties); + SetKeyspaceShardingInfoRequest.create = function create(properties) { + return new SetKeyspaceShardingInfoRequest(properties); }; /** - * Encodes the specified RestoreFromBackupRequest message. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceShardingInfoRequest message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static - * @param {vtctldata.IRestoreFromBackupRequest} message RestoreFromBackupRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RestoreFromBackupRequest.encode = function encode(message, writer) { + SetKeyspaceShardingInfoRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.backup_time != null && Object.hasOwnProperty.call(message, "backup_time")) - $root.vttime.Time.encode(message.backup_time, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.restore_to_pos != null && Object.hasOwnProperty.call(message, "restore_to_pos")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.restore_to_pos); - if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.dry_run); - if (message.restore_to_timestamp != null && Object.hasOwnProperty.call(message, "restore_to_timestamp")) - $root.vttime.Time.encode(message.restore_to_timestamp, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.force != null && Object.hasOwnProperty.call(message, "force")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); return writer; }; /** - * Encodes the specified RestoreFromBackupRequest message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupRequest.verify|verify} messages. + * Encodes the specified SetKeyspaceShardingInfoRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static - * @param {vtctldata.IRestoreFromBackupRequest} message RestoreFromBackupRequest message or plain object to encode + * @param {vtctldata.ISetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RestoreFromBackupRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceShardingInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RestoreFromBackupRequest message from the specified reader or buffer. + * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest + * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RestoreFromBackupRequest.decode = function decode(reader, length) { + SetKeyspaceShardingInfoRequest.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.RestoreFromBackupRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceShardingInfoRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - message.backup_time = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 3: { - message.restore_to_pos = reader.string(); + message.keyspace = reader.string(); break; } case 4: { - message.dry_run = reader.bool(); - break; - } - case 5: { - message.restore_to_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); + message.force = reader.bool(); break; } default: @@ -130336,173 +137036,131 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RestoreFromBackupRequest message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest + * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RestoreFromBackupRequest.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceShardingInfoRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RestoreFromBackupRequest message. + * Verifies a SetKeyspaceShardingInfoRequest message. * @function verify - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RestoreFromBackupRequest.verify = function verify(message) { + SetKeyspaceShardingInfoRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.backup_time != null && message.hasOwnProperty("backup_time")) { - let error = $root.vttime.Time.verify(message.backup_time); - if (error) - return "backup_time." + error; - } - if (message.restore_to_pos != null && message.hasOwnProperty("restore_to_pos")) - if (!$util.isString(message.restore_to_pos)) - return "restore_to_pos: string expected"; - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - if (typeof message.dry_run !== "boolean") - return "dry_run: boolean expected"; - if (message.restore_to_timestamp != null && message.hasOwnProperty("restore_to_timestamp")) { - let error = $root.vttime.Time.verify(message.restore_to_timestamp); - if (error) - return "restore_to_timestamp." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.force != null && message.hasOwnProperty("force")) + if (typeof message.force !== "boolean") + return "force: boolean expected"; return null; }; /** - * Creates a RestoreFromBackupRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceShardingInfoRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RestoreFromBackupRequest} RestoreFromBackupRequest + * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest */ - RestoreFromBackupRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RestoreFromBackupRequest) + SetKeyspaceShardingInfoRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceShardingInfoRequest) return object; - let message = new $root.vtctldata.RestoreFromBackupRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.RestoreFromBackupRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.backup_time != null) { - if (typeof object.backup_time !== "object") - throw TypeError(".vtctldata.RestoreFromBackupRequest.backup_time: object expected"); - message.backup_time = $root.vttime.Time.fromObject(object.backup_time); - } - if (object.restore_to_pos != null) - message.restore_to_pos = String(object.restore_to_pos); - if (object.dry_run != null) - message.dry_run = Boolean(object.dry_run); - if (object.restore_to_timestamp != null) { - if (typeof object.restore_to_timestamp !== "object") - throw TypeError(".vtctldata.RestoreFromBackupRequest.restore_to_timestamp: object expected"); - message.restore_to_timestamp = $root.vttime.Time.fromObject(object.restore_to_timestamp); - } + let message = new $root.vtctldata.SetKeyspaceShardingInfoRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.force != null) + message.force = Boolean(object.force); return message; }; /** - * Creates a plain object from a RestoreFromBackupRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceShardingInfoRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static - * @param {vtctldata.RestoreFromBackupRequest} message RestoreFromBackupRequest + * @param {vtctldata.SetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RestoreFromBackupRequest.toObject = function toObject(message, options) { + SetKeyspaceShardingInfoRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.tablet_alias = null; - object.backup_time = null; - object.restore_to_pos = ""; - object.dry_run = false; - object.restore_to_timestamp = null; + object.keyspace = ""; + object.force = false; } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.backup_time != null && message.hasOwnProperty("backup_time")) - object.backup_time = $root.vttime.Time.toObject(message.backup_time, options); - if (message.restore_to_pos != null && message.hasOwnProperty("restore_to_pos")) - object.restore_to_pos = message.restore_to_pos; - if (message.dry_run != null && message.hasOwnProperty("dry_run")) - object.dry_run = message.dry_run; - if (message.restore_to_timestamp != null && message.hasOwnProperty("restore_to_timestamp")) - object.restore_to_timestamp = $root.vttime.Time.toObject(message.restore_to_timestamp, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.force != null && message.hasOwnProperty("force")) + object.force = message.force; return object; }; /** - * Converts this RestoreFromBackupRequest to JSON. + * Converts this SetKeyspaceShardingInfoRequest to JSON. * @function toJSON - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @instance * @returns {Object.} JSON object */ - RestoreFromBackupRequest.prototype.toJSON = function toJSON() { + SetKeyspaceShardingInfoRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RestoreFromBackupRequest + * Gets the default type url for SetKeyspaceShardingInfoRequest * @function getTypeUrl - * @memberof vtctldata.RestoreFromBackupRequest + * @memberof vtctldata.SetKeyspaceShardingInfoRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RestoreFromBackupRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceShardingInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RestoreFromBackupRequest"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceShardingInfoRequest"; }; - return RestoreFromBackupRequest; + return SetKeyspaceShardingInfoRequest; })(); - vtctldata.RestoreFromBackupResponse = (function() { + vtctldata.SetKeyspaceShardingInfoResponse = (function() { /** - * Properties of a RestoreFromBackupResponse. + * Properties of a SetKeyspaceShardingInfoResponse. * @memberof vtctldata - * @interface IRestoreFromBackupResponse - * @property {topodata.ITabletAlias|null} [tablet_alias] RestoreFromBackupResponse tablet_alias - * @property {string|null} [keyspace] RestoreFromBackupResponse keyspace - * @property {string|null} [shard] RestoreFromBackupResponse shard - * @property {logutil.IEvent|null} [event] RestoreFromBackupResponse event + * @interface ISetKeyspaceShardingInfoResponse + * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceShardingInfoResponse keyspace */ /** - * Constructs a new RestoreFromBackupResponse. + * Constructs a new SetKeyspaceShardingInfoResponse. * @memberof vtctldata - * @classdesc Represents a RestoreFromBackupResponse. - * @implements IRestoreFromBackupResponse + * @classdesc Represents a SetKeyspaceShardingInfoResponse. + * @implements ISetKeyspaceShardingInfoResponse * @constructor - * @param {vtctldata.IRestoreFromBackupResponse=} [properties] Properties to set + * @param {vtctldata.ISetKeyspaceShardingInfoResponse=} [properties] Properties to set */ - function RestoreFromBackupResponse(properties) { + function SetKeyspaceShardingInfoResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -130510,117 +137168,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RestoreFromBackupResponse tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.RestoreFromBackupResponse - * @instance - */ - RestoreFromBackupResponse.prototype.tablet_alias = null; - - /** - * RestoreFromBackupResponse keyspace. - * @member {string} keyspace - * @memberof vtctldata.RestoreFromBackupResponse - * @instance - */ - RestoreFromBackupResponse.prototype.keyspace = ""; - - /** - * RestoreFromBackupResponse shard. - * @member {string} shard - * @memberof vtctldata.RestoreFromBackupResponse - * @instance - */ - RestoreFromBackupResponse.prototype.shard = ""; - - /** - * RestoreFromBackupResponse event. - * @member {logutil.IEvent|null|undefined} event - * @memberof vtctldata.RestoreFromBackupResponse + * SetKeyspaceShardingInfoResponse keyspace. + * @member {topodata.IKeyspace|null|undefined} keyspace + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @instance */ - RestoreFromBackupResponse.prototype.event = null; + SetKeyspaceShardingInfoResponse.prototype.keyspace = null; /** - * Creates a new RestoreFromBackupResponse instance using the specified properties. + * Creates a new SetKeyspaceShardingInfoResponse instance using the specified properties. * @function create - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static - * @param {vtctldata.IRestoreFromBackupResponse=} [properties] Properties to set - * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse instance + * @param {vtctldata.ISetKeyspaceShardingInfoResponse=} [properties] Properties to set + * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse instance */ - RestoreFromBackupResponse.create = function create(properties) { - return new RestoreFromBackupResponse(properties); + SetKeyspaceShardingInfoResponse.create = function create(properties) { + return new SetKeyspaceShardingInfoResponse(properties); }; /** - * Encodes the specified RestoreFromBackupResponse message. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceShardingInfoResponse message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static - * @param {vtctldata.IRestoreFromBackupResponse} message RestoreFromBackupResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RestoreFromBackupResponse.encode = function encode(message, writer) { + SetKeyspaceShardingInfoResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.shard); - if (message.event != null && Object.hasOwnProperty.call(message, "event")) - $root.logutil.Event.encode(message.event, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RestoreFromBackupResponse message, length delimited. Does not implicitly {@link vtctldata.RestoreFromBackupResponse.verify|verify} messages. + * Encodes the specified SetKeyspaceShardingInfoResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static - * @param {vtctldata.IRestoreFromBackupResponse} message RestoreFromBackupResponse message or plain object to encode + * @param {vtctldata.ISetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RestoreFromBackupResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetKeyspaceShardingInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RestoreFromBackupResponse message from the specified reader or buffer. + * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse + * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RestoreFromBackupResponse.decode = function decode(reader, length) { + SetKeyspaceShardingInfoResponse.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.RestoreFromBackupResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetKeyspaceShardingInfoResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 2: { - message.keyspace = reader.string(); - break; - } - case 3: { - message.shard = reader.string(); - break; - } - case 4: { - message.event = $root.logutil.Event.decode(reader, reader.uint32()); + message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); break; } default: @@ -130632,157 +137248,129 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RestoreFromBackupResponse message from the specified reader or buffer, length delimited. + * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse + * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RestoreFromBackupResponse.decodeDelimited = function decodeDelimited(reader) { + SetKeyspaceShardingInfoResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RestoreFromBackupResponse message. + * Verifies a SetKeyspaceShardingInfoResponse message. * @function verify - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RestoreFromBackupResponse.verify = function verify(message) { + SetKeyspaceShardingInfoResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.event != null && message.hasOwnProperty("event")) { - let error = $root.logutil.Event.verify(message.event); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) { + let error = $root.topodata.Keyspace.verify(message.keyspace); if (error) - return "event." + error; + return "keyspace." + error; } return null; }; /** - * Creates a RestoreFromBackupResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetKeyspaceShardingInfoResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RestoreFromBackupResponse} RestoreFromBackupResponse + * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse */ - RestoreFromBackupResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RestoreFromBackupResponse) + SetKeyspaceShardingInfoResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetKeyspaceShardingInfoResponse) return object; - let message = new $root.vtctldata.RestoreFromBackupResponse(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.RestoreFromBackupResponse.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.event != null) { - if (typeof object.event !== "object") - throw TypeError(".vtctldata.RestoreFromBackupResponse.event: object expected"); - message.event = $root.logutil.Event.fromObject(object.event); + let message = new $root.vtctldata.SetKeyspaceShardingInfoResponse(); + if (object.keyspace != null) { + if (typeof object.keyspace !== "object") + throw TypeError(".vtctldata.SetKeyspaceShardingInfoResponse.keyspace: object expected"); + message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); } return message; }; /** - * Creates a plain object from a RestoreFromBackupResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetKeyspaceShardingInfoResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static - * @param {vtctldata.RestoreFromBackupResponse} message RestoreFromBackupResponse + * @param {vtctldata.SetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RestoreFromBackupResponse.toObject = function toObject(message, options) { + SetKeyspaceShardingInfoResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.tablet_alias = null; - object.keyspace = ""; - object.shard = ""; - object.event = null; - } - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) + object.keyspace = null; if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.event != null && message.hasOwnProperty("event")) - object.event = $root.logutil.Event.toObject(message.event, options); + object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); return object; }; /** - * Converts this RestoreFromBackupResponse to JSON. + * Converts this SetKeyspaceShardingInfoResponse to JSON. * @function toJSON - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @instance * @returns {Object.} JSON object */ - RestoreFromBackupResponse.prototype.toJSON = function toJSON() { + SetKeyspaceShardingInfoResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RestoreFromBackupResponse + * Gets the default type url for SetKeyspaceShardingInfoResponse * @function getTypeUrl - * @memberof vtctldata.RestoreFromBackupResponse + * @memberof vtctldata.SetKeyspaceShardingInfoResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RestoreFromBackupResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetKeyspaceShardingInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RestoreFromBackupResponse"; + return typeUrlPrefix + "/vtctldata.SetKeyspaceShardingInfoResponse"; }; - return RestoreFromBackupResponse; + return SetKeyspaceShardingInfoResponse; })(); - vtctldata.RunHealthCheckRequest = (function() { + vtctldata.SetShardIsPrimaryServingRequest = (function() { /** - * Properties of a RunHealthCheckRequest. + * Properties of a SetShardIsPrimaryServingRequest. * @memberof vtctldata - * @interface IRunHealthCheckRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] RunHealthCheckRequest tablet_alias + * @interface ISetShardIsPrimaryServingRequest + * @property {string|null} [keyspace] SetShardIsPrimaryServingRequest keyspace + * @property {string|null} [shard] SetShardIsPrimaryServingRequest shard + * @property {boolean|null} [is_serving] SetShardIsPrimaryServingRequest is_serving */ /** - * Constructs a new RunHealthCheckRequest. + * Constructs a new SetShardIsPrimaryServingRequest. * @memberof vtctldata - * @classdesc Represents a RunHealthCheckRequest. - * @implements IRunHealthCheckRequest + * @classdesc Represents a SetShardIsPrimaryServingRequest. + * @implements ISetShardIsPrimaryServingRequest * @constructor - * @param {vtctldata.IRunHealthCheckRequest=} [properties] Properties to set + * @param {vtctldata.ISetShardIsPrimaryServingRequest=} [properties] Properties to set */ - function RunHealthCheckRequest(properties) { + function SetShardIsPrimaryServingRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -130790,75 +137378,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * RunHealthCheckRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.RunHealthCheckRequest + * SetShardIsPrimaryServingRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @instance */ - RunHealthCheckRequest.prototype.tablet_alias = null; + SetShardIsPrimaryServingRequest.prototype.keyspace = ""; /** - * Creates a new RunHealthCheckRequest instance using the specified properties. + * SetShardIsPrimaryServingRequest shard. + * @member {string} shard + * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @instance + */ + SetShardIsPrimaryServingRequest.prototype.shard = ""; + + /** + * SetShardIsPrimaryServingRequest is_serving. + * @member {boolean} is_serving + * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @instance + */ + SetShardIsPrimaryServingRequest.prototype.is_serving = false; + + /** + * Creates a new SetShardIsPrimaryServingRequest instance using the specified properties. * @function create - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static - * @param {vtctldata.IRunHealthCheckRequest=} [properties] Properties to set - * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest instance + * @param {vtctldata.ISetShardIsPrimaryServingRequest=} [properties] Properties to set + * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest instance */ - RunHealthCheckRequest.create = function create(properties) { - return new RunHealthCheckRequest(properties); + SetShardIsPrimaryServingRequest.create = function create(properties) { + return new SetShardIsPrimaryServingRequest(properties); }; /** - * Encodes the specified RunHealthCheckRequest message. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. + * Encodes the specified SetShardIsPrimaryServingRequest message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static - * @param {vtctldata.IRunHealthCheckRequest} message RunHealthCheckRequest message or plain object to encode + * @param {vtctldata.ISetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RunHealthCheckRequest.encode = function encode(message, writer) { + SetShardIsPrimaryServingRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.is_serving != null && Object.hasOwnProperty.call(message, "is_serving")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.is_serving); return writer; }; /** - * Encodes the specified RunHealthCheckRequest message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckRequest.verify|verify} messages. + * Encodes the specified SetShardIsPrimaryServingRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static - * @param {vtctldata.IRunHealthCheckRequest} message RunHealthCheckRequest message or plain object to encode + * @param {vtctldata.ISetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RunHealthCheckRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetShardIsPrimaryServingRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RunHealthCheckRequest message from the specified reader or buffer. + * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest + * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RunHealthCheckRequest.decode = function decode(reader, length) { + SetShardIsPrimaryServingRequest.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.RunHealthCheckRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetShardIsPrimaryServingRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.is_serving = reader.bool(); break; } default: @@ -130870,126 +137486,139 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RunHealthCheckRequest message from the specified reader or buffer, length delimited. + * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest + * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RunHealthCheckRequest.decodeDelimited = function decodeDelimited(reader) { + SetShardIsPrimaryServingRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RunHealthCheckRequest message. + * Verifies a SetShardIsPrimaryServingRequest message. * @function verify - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RunHealthCheckRequest.verify = function verify(message) { + SetShardIsPrimaryServingRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.is_serving != null && message.hasOwnProperty("is_serving")) + if (typeof message.is_serving !== "boolean") + return "is_serving: boolean expected"; return null; }; /** - * Creates a RunHealthCheckRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardIsPrimaryServingRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.RunHealthCheckRequest} RunHealthCheckRequest + * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest */ - RunHealthCheckRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RunHealthCheckRequest) + SetShardIsPrimaryServingRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetShardIsPrimaryServingRequest) return object; - let message = new $root.vtctldata.RunHealthCheckRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.RunHealthCheckRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.SetShardIsPrimaryServingRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.is_serving != null) + message.is_serving = Boolean(object.is_serving); return message; }; /** - * Creates a plain object from a RunHealthCheckRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetShardIsPrimaryServingRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static - * @param {vtctldata.RunHealthCheckRequest} message RunHealthCheckRequest + * @param {vtctldata.SetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RunHealthCheckRequest.toObject = function toObject(message, options) { + SetShardIsPrimaryServingRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.is_serving = false; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.is_serving != null && message.hasOwnProperty("is_serving")) + object.is_serving = message.is_serving; return object; }; /** - * Converts this RunHealthCheckRequest to JSON. + * Converts this SetShardIsPrimaryServingRequest to JSON. * @function toJSON - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @instance * @returns {Object.} JSON object */ - RunHealthCheckRequest.prototype.toJSON = function toJSON() { + SetShardIsPrimaryServingRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RunHealthCheckRequest + * Gets the default type url for SetShardIsPrimaryServingRequest * @function getTypeUrl - * @memberof vtctldata.RunHealthCheckRequest + * @memberof vtctldata.SetShardIsPrimaryServingRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RunHealthCheckRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetShardIsPrimaryServingRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RunHealthCheckRequest"; + return typeUrlPrefix + "/vtctldata.SetShardIsPrimaryServingRequest"; }; - return RunHealthCheckRequest; + return SetShardIsPrimaryServingRequest; })(); - vtctldata.RunHealthCheckResponse = (function() { + vtctldata.SetShardIsPrimaryServingResponse = (function() { /** - * Properties of a RunHealthCheckResponse. + * Properties of a SetShardIsPrimaryServingResponse. * @memberof vtctldata - * @interface IRunHealthCheckResponse + * @interface ISetShardIsPrimaryServingResponse + * @property {topodata.IShard|null} [shard] SetShardIsPrimaryServingResponse shard */ /** - * Constructs a new RunHealthCheckResponse. + * Constructs a new SetShardIsPrimaryServingResponse. * @memberof vtctldata - * @classdesc Represents a RunHealthCheckResponse. - * @implements IRunHealthCheckResponse + * @classdesc Represents a SetShardIsPrimaryServingResponse. + * @implements ISetShardIsPrimaryServingResponse * @constructor - * @param {vtctldata.IRunHealthCheckResponse=} [properties] Properties to set + * @param {vtctldata.ISetShardIsPrimaryServingResponse=} [properties] Properties to set */ - function RunHealthCheckResponse(properties) { + function SetShardIsPrimaryServingResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -130997,63 +137626,77 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new RunHealthCheckResponse instance using the specified properties. + * SetShardIsPrimaryServingResponse shard. + * @member {topodata.IShard|null|undefined} shard + * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @instance + */ + SetShardIsPrimaryServingResponse.prototype.shard = null; + + /** + * Creates a new SetShardIsPrimaryServingResponse instance using the specified properties. * @function create - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static - * @param {vtctldata.IRunHealthCheckResponse=} [properties] Properties to set - * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse instance + * @param {vtctldata.ISetShardIsPrimaryServingResponse=} [properties] Properties to set + * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse instance */ - RunHealthCheckResponse.create = function create(properties) { - return new RunHealthCheckResponse(properties); + SetShardIsPrimaryServingResponse.create = function create(properties) { + return new SetShardIsPrimaryServingResponse(properties); }; /** - * Encodes the specified RunHealthCheckResponse message. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. + * Encodes the specified SetShardIsPrimaryServingResponse message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static - * @param {vtctldata.IRunHealthCheckResponse} message RunHealthCheckResponse message or plain object to encode + * @param {vtctldata.ISetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RunHealthCheckResponse.encode = function encode(message, writer) { + SetShardIsPrimaryServingResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified RunHealthCheckResponse message, length delimited. Does not implicitly {@link vtctldata.RunHealthCheckResponse.verify|verify} messages. + * Encodes the specified SetShardIsPrimaryServingResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static - * @param {vtctldata.IRunHealthCheckResponse} message RunHealthCheckResponse message or plain object to encode + * @param {vtctldata.ISetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - RunHealthCheckResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetShardIsPrimaryServingResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a RunHealthCheckResponse message from the specified reader or buffer. + * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse + * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RunHealthCheckResponse.decode = function decode(reader, length) { + SetShardIsPrimaryServingResponse.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.RunHealthCheckResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetShardIsPrimaryServingResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -131063,110 +137706,135 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a RunHealthCheckResponse message from the specified reader or buffer, length delimited. + * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse + * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - RunHealthCheckResponse.decodeDelimited = function decodeDelimited(reader) { + SetShardIsPrimaryServingResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a RunHealthCheckResponse message. + * Verifies a SetShardIsPrimaryServingResponse message. * @function verify - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - RunHealthCheckResponse.verify = function verify(message) { + SetShardIsPrimaryServingResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.topodata.Shard.verify(message.shard); + if (error) + return "shard." + error; + } return null; }; /** - * Creates a RunHealthCheckResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardIsPrimaryServingResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.RunHealthCheckResponse} RunHealthCheckResponse + * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse */ - RunHealthCheckResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.RunHealthCheckResponse) + SetShardIsPrimaryServingResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetShardIsPrimaryServingResponse) return object; - return new $root.vtctldata.RunHealthCheckResponse(); + let message = new $root.vtctldata.SetShardIsPrimaryServingResponse(); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.SetShardIsPrimaryServingResponse.shard: object expected"); + message.shard = $root.topodata.Shard.fromObject(object.shard); + } + return message; }; /** - * Creates a plain object from a RunHealthCheckResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetShardIsPrimaryServingResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static - * @param {vtctldata.RunHealthCheckResponse} message RunHealthCheckResponse + * @param {vtctldata.SetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - RunHealthCheckResponse.toObject = function toObject() { - return {}; + SetShardIsPrimaryServingResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.shard = null; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.topodata.Shard.toObject(message.shard, options); + return object; }; /** - * Converts this RunHealthCheckResponse to JSON. + * Converts this SetShardIsPrimaryServingResponse to JSON. * @function toJSON - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @instance * @returns {Object.} JSON object */ - RunHealthCheckResponse.prototype.toJSON = function toJSON() { + SetShardIsPrimaryServingResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for RunHealthCheckResponse + * Gets the default type url for SetShardIsPrimaryServingResponse * @function getTypeUrl - * @memberof vtctldata.RunHealthCheckResponse + * @memberof vtctldata.SetShardIsPrimaryServingResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - RunHealthCheckResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetShardIsPrimaryServingResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.RunHealthCheckResponse"; + return typeUrlPrefix + "/vtctldata.SetShardIsPrimaryServingResponse"; }; - return RunHealthCheckResponse; + return SetShardIsPrimaryServingResponse; })(); - vtctldata.SetKeyspaceDurabilityPolicyRequest = (function() { + vtctldata.SetShardTabletControlRequest = (function() { /** - * Properties of a SetKeyspaceDurabilityPolicyRequest. + * Properties of a SetShardTabletControlRequest. * @memberof vtctldata - * @interface ISetKeyspaceDurabilityPolicyRequest - * @property {string|null} [keyspace] SetKeyspaceDurabilityPolicyRequest keyspace - * @property {string|null} [durability_policy] SetKeyspaceDurabilityPolicyRequest durability_policy + * @interface ISetShardTabletControlRequest + * @property {string|null} [keyspace] SetShardTabletControlRequest keyspace + * @property {string|null} [shard] SetShardTabletControlRequest shard + * @property {topodata.TabletType|null} [tablet_type] SetShardTabletControlRequest tablet_type + * @property {Array.|null} [cells] SetShardTabletControlRequest cells + * @property {Array.|null} [denied_tables] SetShardTabletControlRequest denied_tables + * @property {boolean|null} [disable_query_service] SetShardTabletControlRequest disable_query_service + * @property {boolean|null} [remove] SetShardTabletControlRequest remove */ /** - * Constructs a new SetKeyspaceDurabilityPolicyRequest. + * Constructs a new SetShardTabletControlRequest. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceDurabilityPolicyRequest. - * @implements ISetKeyspaceDurabilityPolicyRequest + * @classdesc Represents a SetShardTabletControlRequest. + * @implements ISetShardTabletControlRequest * @constructor - * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest=} [properties] Properties to set + * @param {vtctldata.ISetShardTabletControlRequest=} [properties] Properties to set */ - function SetKeyspaceDurabilityPolicyRequest(properties) { + function SetShardTabletControlRequest(properties) { + this.cells = []; + this.denied_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -131174,80 +137842,132 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceDurabilityPolicyRequest keyspace. + * SetShardTabletControlRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @instance */ - SetKeyspaceDurabilityPolicyRequest.prototype.keyspace = ""; + SetShardTabletControlRequest.prototype.keyspace = ""; /** - * SetKeyspaceDurabilityPolicyRequest durability_policy. - * @member {string} durability_policy - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * SetShardTabletControlRequest shard. + * @member {string} shard + * @memberof vtctldata.SetShardTabletControlRequest * @instance */ - SetKeyspaceDurabilityPolicyRequest.prototype.durability_policy = ""; + SetShardTabletControlRequest.prototype.shard = ""; /** - * Creates a new SetKeyspaceDurabilityPolicyRequest instance using the specified properties. + * SetShardTabletControlRequest tablet_type. + * @member {topodata.TabletType} tablet_type + * @memberof vtctldata.SetShardTabletControlRequest + * @instance + */ + SetShardTabletControlRequest.prototype.tablet_type = 0; + + /** + * SetShardTabletControlRequest cells. + * @member {Array.} cells + * @memberof vtctldata.SetShardTabletControlRequest + * @instance + */ + SetShardTabletControlRequest.prototype.cells = $util.emptyArray; + + /** + * SetShardTabletControlRequest denied_tables. + * @member {Array.} denied_tables + * @memberof vtctldata.SetShardTabletControlRequest + * @instance + */ + SetShardTabletControlRequest.prototype.denied_tables = $util.emptyArray; + + /** + * SetShardTabletControlRequest disable_query_service. + * @member {boolean} disable_query_service + * @memberof vtctldata.SetShardTabletControlRequest + * @instance + */ + SetShardTabletControlRequest.prototype.disable_query_service = false; + + /** + * SetShardTabletControlRequest remove. + * @member {boolean} remove + * @memberof vtctldata.SetShardTabletControlRequest + * @instance + */ + SetShardTabletControlRequest.prototype.remove = false; + + /** + * Creates a new SetShardTabletControlRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest instance + * @param {vtctldata.ISetShardTabletControlRequest=} [properties] Properties to set + * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest instance */ - SetKeyspaceDurabilityPolicyRequest.create = function create(properties) { - return new SetKeyspaceDurabilityPolicyRequest(properties); + SetShardTabletControlRequest.create = function create(properties) { + return new SetShardTabletControlRequest(properties); }; /** - * Encodes the specified SetKeyspaceDurabilityPolicyRequest message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. + * Encodes the specified SetShardTabletControlRequest message. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest message or plain object to encode + * @param {vtctldata.ISetShardTabletControlRequest} message SetShardTabletControlRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceDurabilityPolicyRequest.encode = function encode(message, writer) { + SetShardTabletControlRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.durability_policy != null && Object.hasOwnProperty.call(message, "durability_policy")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.durability_policy); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.cells[i]); + if (message.denied_tables != null && message.denied_tables.length) + for (let i = 0; i < message.denied_tables.length; ++i) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.denied_tables[i]); + if (message.disable_query_service != null && Object.hasOwnProperty.call(message, "disable_query_service")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.disable_query_service); + if (message.remove != null && Object.hasOwnProperty.call(message, "remove")) + writer.uint32(/* id 7, wireType 0 =*/56).bool(message.remove); return writer; }; /** - * Encodes the specified SetKeyspaceDurabilityPolicyRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyRequest.verify|verify} messages. + * Encodes the specified SetShardTabletControlRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest message or plain object to encode + * @param {vtctldata.ISetShardTabletControlRequest} message SetShardTabletControlRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceDurabilityPolicyRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetShardTabletControlRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer. + * Decodes a SetShardTabletControlRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest + * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceDurabilityPolicyRequest.decode = function decode(reader, length) { + SetShardTabletControlRequest.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.SetKeyspaceDurabilityPolicyRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetShardTabletControlRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -131256,7 +137976,31 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.durability_policy = reader.string(); + message.shard = reader.string(); + break; + } + case 3: { + message.tablet_type = reader.int32(); + break; + } + case 4: { + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 5: { + if (!(message.denied_tables && message.denied_tables.length)) + message.denied_tables = []; + message.denied_tables.push(reader.string()); + break; + } + case 6: { + message.disable_query_service = reader.bool(); + break; + } + case 7: { + message.remove = reader.bool(); break; } default: @@ -131268,131 +138012,261 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetKeyspaceDurabilityPolicyRequest message from the specified reader or buffer, length delimited. + * Decodes a SetShardTabletControlRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest + * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceDurabilityPolicyRequest.decodeDelimited = function decodeDelimited(reader) { + SetShardTabletControlRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetKeyspaceDurabilityPolicyRequest message. + * Verifies a SetShardTabletControlRequest message. * @function verify - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetKeyspaceDurabilityPolicyRequest.verify = function verify(message) { + SetShardTabletControlRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) - if (!$util.isString(message.durability_policy)) - return "durability_policy: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + switch (message.tablet_type) { + default: + return "tablet_type: enum value expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.denied_tables != null && message.hasOwnProperty("denied_tables")) { + if (!Array.isArray(message.denied_tables)) + return "denied_tables: array expected"; + for (let i = 0; i < message.denied_tables.length; ++i) + if (!$util.isString(message.denied_tables[i])) + return "denied_tables: string[] expected"; + } + if (message.disable_query_service != null && message.hasOwnProperty("disable_query_service")) + if (typeof message.disable_query_service !== "boolean") + return "disable_query_service: boolean expected"; + if (message.remove != null && message.hasOwnProperty("remove")) + if (typeof message.remove !== "boolean") + return "remove: boolean expected"; return null; }; /** - * Creates a SetKeyspaceDurabilityPolicyRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardTabletControlRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceDurabilityPolicyRequest} SetKeyspaceDurabilityPolicyRequest + * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest */ - SetKeyspaceDurabilityPolicyRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceDurabilityPolicyRequest) + SetShardTabletControlRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetShardTabletControlRequest) return object; - let message = new $root.vtctldata.SetKeyspaceDurabilityPolicyRequest(); + let message = new $root.vtctldata.SetShardTabletControlRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.durability_policy != null) - message.durability_policy = String(object.durability_policy); + if (object.shard != null) + message.shard = String(object.shard); + switch (object.tablet_type) { + default: + if (typeof object.tablet_type === "number") { + message.tablet_type = object.tablet_type; + break; + } + break; + case "UNKNOWN": + case 0: + message.tablet_type = 0; + break; + case "PRIMARY": + case 1: + message.tablet_type = 1; + break; + case "MASTER": + case 1: + message.tablet_type = 1; + break; + case "REPLICA": + case 2: + message.tablet_type = 2; + break; + case "RDONLY": + case 3: + message.tablet_type = 3; + break; + case "BATCH": + case 3: + message.tablet_type = 3; + break; + case "SPARE": + case 4: + message.tablet_type = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_type = 5; + break; + case "BACKUP": + case 6: + message.tablet_type = 6; + break; + case "RESTORE": + case 7: + message.tablet_type = 7; + break; + case "DRAINED": + case 8: + message.tablet_type = 8; + break; + } + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.SetShardTabletControlRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.denied_tables) { + if (!Array.isArray(object.denied_tables)) + throw TypeError(".vtctldata.SetShardTabletControlRequest.denied_tables: array expected"); + message.denied_tables = []; + for (let i = 0; i < object.denied_tables.length; ++i) + message.denied_tables[i] = String(object.denied_tables[i]); + } + if (object.disable_query_service != null) + message.disable_query_service = Boolean(object.disable_query_service); + if (object.remove != null) + message.remove = Boolean(object.remove); return message; }; /** - * Creates a plain object from a SetKeyspaceDurabilityPolicyRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetShardTabletControlRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static - * @param {vtctldata.SetKeyspaceDurabilityPolicyRequest} message SetKeyspaceDurabilityPolicyRequest + * @param {vtctldata.SetShardTabletControlRequest} message SetShardTabletControlRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceDurabilityPolicyRequest.toObject = function toObject(message, options) { + SetShardTabletControlRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.cells = []; + object.denied_tables = []; + } if (options.defaults) { object.keyspace = ""; - object.durability_policy = ""; + object.shard = ""; + object.tablet_type = options.enums === String ? "UNKNOWN" : 0; + object.disable_query_service = false; + object.remove = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.durability_policy != null && message.hasOwnProperty("durability_policy")) - object.durability_policy = message.durability_policy; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) + object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.denied_tables && message.denied_tables.length) { + object.denied_tables = []; + for (let j = 0; j < message.denied_tables.length; ++j) + object.denied_tables[j] = message.denied_tables[j]; + } + if (message.disable_query_service != null && message.hasOwnProperty("disable_query_service")) + object.disable_query_service = message.disable_query_service; + if (message.remove != null && message.hasOwnProperty("remove")) + object.remove = message.remove; return object; }; /** - * Converts this SetKeyspaceDurabilityPolicyRequest to JSON. + * Converts this SetShardTabletControlRequest to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @instance * @returns {Object.} JSON object */ - SetKeyspaceDurabilityPolicyRequest.prototype.toJSON = function toJSON() { + SetShardTabletControlRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceDurabilityPolicyRequest + * Gets the default type url for SetShardTabletControlRequest * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceDurabilityPolicyRequest + * @memberof vtctldata.SetShardTabletControlRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceDurabilityPolicyRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetShardTabletControlRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceDurabilityPolicyRequest"; + return typeUrlPrefix + "/vtctldata.SetShardTabletControlRequest"; }; - return SetKeyspaceDurabilityPolicyRequest; + return SetShardTabletControlRequest; })(); - vtctldata.SetKeyspaceDurabilityPolicyResponse = (function() { + vtctldata.SetShardTabletControlResponse = (function() { /** - * Properties of a SetKeyspaceDurabilityPolicyResponse. + * Properties of a SetShardTabletControlResponse. * @memberof vtctldata - * @interface ISetKeyspaceDurabilityPolicyResponse - * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceDurabilityPolicyResponse keyspace + * @interface ISetShardTabletControlResponse + * @property {topodata.IShard|null} [shard] SetShardTabletControlResponse shard */ /** - * Constructs a new SetKeyspaceDurabilityPolicyResponse. + * Constructs a new SetShardTabletControlResponse. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceDurabilityPolicyResponse. - * @implements ISetKeyspaceDurabilityPolicyResponse + * @classdesc Represents a SetShardTabletControlResponse. + * @implements ISetShardTabletControlResponse * @constructor - * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse=} [properties] Properties to set + * @param {vtctldata.ISetShardTabletControlResponse=} [properties] Properties to set */ - function SetKeyspaceDurabilityPolicyResponse(properties) { + function SetShardTabletControlResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -131400,75 +138274,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceDurabilityPolicyResponse keyspace. - * @member {topodata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * SetShardTabletControlResponse shard. + * @member {topodata.IShard|null|undefined} shard + * @memberof vtctldata.SetShardTabletControlResponse * @instance */ - SetKeyspaceDurabilityPolicyResponse.prototype.keyspace = null; + SetShardTabletControlResponse.prototype.shard = null; /** - * Creates a new SetKeyspaceDurabilityPolicyResponse instance using the specified properties. + * Creates a new SetShardTabletControlResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse instance + * @param {vtctldata.ISetShardTabletControlResponse=} [properties] Properties to set + * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse instance */ - SetKeyspaceDurabilityPolicyResponse.create = function create(properties) { - return new SetKeyspaceDurabilityPolicyResponse(properties); + SetShardTabletControlResponse.create = function create(properties) { + return new SetShardTabletControlResponse(properties); }; /** - * Encodes the specified SetKeyspaceDurabilityPolicyResponse message. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. + * Encodes the specified SetShardTabletControlResponse message. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse message or plain object to encode + * @param {vtctldata.ISetShardTabletControlResponse} message SetShardTabletControlResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceDurabilityPolicyResponse.encode = function encode(message, writer) { + SetShardTabletControlResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetKeyspaceDurabilityPolicyResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceDurabilityPolicyResponse.verify|verify} messages. + * Encodes the specified SetShardTabletControlResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static - * @param {vtctldata.ISetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse message or plain object to encode + * @param {vtctldata.ISetShardTabletControlResponse} message SetShardTabletControlResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceDurabilityPolicyResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetShardTabletControlResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer. + * Decodes a SetShardTabletControlResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse + * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceDurabilityPolicyResponse.decode = function decode(reader, length) { + SetShardTabletControlResponse.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.SetKeyspaceDurabilityPolicyResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetShardTabletControlResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); + message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); break; } default: @@ -131480,132 +138354,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetKeyspaceDurabilityPolicyResponse message from the specified reader or buffer, length delimited. + * Decodes a SetShardTabletControlResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse + * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceDurabilityPolicyResponse.decodeDelimited = function decodeDelimited(reader) { + SetShardTabletControlResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetKeyspaceDurabilityPolicyResponse message. + * Verifies a SetShardTabletControlResponse message. * @function verify - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetKeyspaceDurabilityPolicyResponse.verify = function verify(message) { + SetShardTabletControlResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.topodata.Keyspace.verify(message.keyspace); + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.topodata.Shard.verify(message.shard); if (error) - return "keyspace." + error; + return "shard." + error; } return null; }; /** - * Creates a SetKeyspaceDurabilityPolicyResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetShardTabletControlResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceDurabilityPolicyResponse} SetKeyspaceDurabilityPolicyResponse + * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse */ - SetKeyspaceDurabilityPolicyResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceDurabilityPolicyResponse) + SetShardTabletControlResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetShardTabletControlResponse) return object; - let message = new $root.vtctldata.SetKeyspaceDurabilityPolicyResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace: object expected"); - message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); + let message = new $root.vtctldata.SetShardTabletControlResponse(); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.SetShardTabletControlResponse.shard: object expected"); + message.shard = $root.topodata.Shard.fromObject(object.shard); } return message; }; /** - * Creates a plain object from a SetKeyspaceDurabilityPolicyResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetShardTabletControlResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static - * @param {vtctldata.SetKeyspaceDurabilityPolicyResponse} message SetKeyspaceDurabilityPolicyResponse + * @param {vtctldata.SetShardTabletControlResponse} message SetShardTabletControlResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceDurabilityPolicyResponse.toObject = function toObject(message, options) { + SetShardTabletControlResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.keyspace = null; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); + object.shard = null; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.topodata.Shard.toObject(message.shard, options); return object; }; /** - * Converts this SetKeyspaceDurabilityPolicyResponse to JSON. + * Converts this SetShardTabletControlResponse to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @instance * @returns {Object.} JSON object */ - SetKeyspaceDurabilityPolicyResponse.prototype.toJSON = function toJSON() { + SetShardTabletControlResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceDurabilityPolicyResponse + * Gets the default type url for SetShardTabletControlResponse * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceDurabilityPolicyResponse + * @memberof vtctldata.SetShardTabletControlResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceDurabilityPolicyResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetShardTabletControlResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceDurabilityPolicyResponse"; + return typeUrlPrefix + "/vtctldata.SetShardTabletControlResponse"; }; - return SetKeyspaceDurabilityPolicyResponse; + return SetShardTabletControlResponse; })(); - vtctldata.SetKeyspaceServedFromRequest = (function() { + vtctldata.SetWritableRequest = (function() { /** - * Properties of a SetKeyspaceServedFromRequest. + * Properties of a SetWritableRequest. * @memberof vtctldata - * @interface ISetKeyspaceServedFromRequest - * @property {string|null} [keyspace] SetKeyspaceServedFromRequest keyspace - * @property {topodata.TabletType|null} [tablet_type] SetKeyspaceServedFromRequest tablet_type - * @property {Array.|null} [cells] SetKeyspaceServedFromRequest cells - * @property {boolean|null} [remove] SetKeyspaceServedFromRequest remove - * @property {string|null} [source_keyspace] SetKeyspaceServedFromRequest source_keyspace + * @interface ISetWritableRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] SetWritableRequest tablet_alias + * @property {boolean|null} [writable] SetWritableRequest writable */ /** - * Constructs a new SetKeyspaceServedFromRequest. + * Constructs a new SetWritableRequest. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceServedFromRequest. - * @implements ISetKeyspaceServedFromRequest + * @classdesc Represents a SetWritableRequest. + * @implements ISetWritableRequest * @constructor - * @param {vtctldata.ISetKeyspaceServedFromRequest=} [properties] Properties to set + * @param {vtctldata.ISetWritableRequest=} [properties] Properties to set */ - function SetKeyspaceServedFromRequest(properties) { - this.cells = []; + function SetWritableRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -131613,134 +138483,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceServedFromRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @instance - */ - SetKeyspaceServedFromRequest.prototype.keyspace = ""; - - /** - * SetKeyspaceServedFromRequest tablet_type. - * @member {topodata.TabletType} tablet_type - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @instance - */ - SetKeyspaceServedFromRequest.prototype.tablet_type = 0; - - /** - * SetKeyspaceServedFromRequest cells. - * @member {Array.} cells - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @instance - */ - SetKeyspaceServedFromRequest.prototype.cells = $util.emptyArray; - - /** - * SetKeyspaceServedFromRequest remove. - * @member {boolean} remove - * @memberof vtctldata.SetKeyspaceServedFromRequest + * SetWritableRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.SetWritableRequest * @instance */ - SetKeyspaceServedFromRequest.prototype.remove = false; + SetWritableRequest.prototype.tablet_alias = null; /** - * SetKeyspaceServedFromRequest source_keyspace. - * @member {string} source_keyspace - * @memberof vtctldata.SetKeyspaceServedFromRequest + * SetWritableRequest writable. + * @member {boolean} writable + * @memberof vtctldata.SetWritableRequest * @instance */ - SetKeyspaceServedFromRequest.prototype.source_keyspace = ""; + SetWritableRequest.prototype.writable = false; /** - * Creates a new SetKeyspaceServedFromRequest instance using the specified properties. + * Creates a new SetWritableRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static - * @param {vtctldata.ISetKeyspaceServedFromRequest=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest instance + * @param {vtctldata.ISetWritableRequest=} [properties] Properties to set + * @returns {vtctldata.SetWritableRequest} SetWritableRequest instance */ - SetKeyspaceServedFromRequest.create = function create(properties) { - return new SetKeyspaceServedFromRequest(properties); + SetWritableRequest.create = function create(properties) { + return new SetWritableRequest(properties); }; /** - * Encodes the specified SetKeyspaceServedFromRequest message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. + * Encodes the specified SetWritableRequest message. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static - * @param {vtctldata.ISetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest message or plain object to encode + * @param {vtctldata.ISetWritableRequest} message SetWritableRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceServedFromRequest.encode = function encode(message, writer) { + SetWritableRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) - writer.uint32(/* id 2, wireType 0 =*/16).int32(message.tablet_type); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); - if (message.remove != null && Object.hasOwnProperty.call(message, "remove")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.remove); - if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_keyspace); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.writable != null && Object.hasOwnProperty.call(message, "writable")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.writable); return writer; }; /** - * Encodes the specified SetKeyspaceServedFromRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromRequest.verify|verify} messages. + * Encodes the specified SetWritableRequest message, length delimited. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static - * @param {vtctldata.ISetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest message or plain object to encode + * @param {vtctldata.ISetWritableRequest} message SetWritableRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceServedFromRequest.encodeDelimited = function encodeDelimited(message, writer) { + SetWritableRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer. + * Decodes a SetWritableRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest + * @returns {vtctldata.SetWritableRequest} SetWritableRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceServedFromRequest.decode = function decode(reader, length) { + SetWritableRequest.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.SetKeyspaceServedFromRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetWritableRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.tablet_type = reader.int32(); - break; - } - case 3: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); - break; - } - case 4: { - message.remove = reader.bool(); - break; - } - case 5: { - message.source_keyspace = reader.string(); + message.writable = reader.bool(); break; } default: @@ -131748,236 +138573,139 @@ export const vtctldata = $root.vtctldata = (() => { break; } } - return message; - }; - - /** - * Decodes a SetKeyspaceServedFromRequest message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - SetKeyspaceServedFromRequest.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a SetKeyspaceServedFromRequest message. - * @function verify - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - SetKeyspaceServedFromRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - switch (message.tablet_type) { - default: - return "tablet_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } - if (message.remove != null && message.hasOwnProperty("remove")) - if (typeof message.remove !== "boolean") - return "remove: boolean expected"; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - if (!$util.isString(message.source_keyspace)) - return "source_keyspace: string expected"; - return null; - }; - - /** - * Creates a SetKeyspaceServedFromRequest message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.SetKeyspaceServedFromRequest - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceServedFromRequest} SetKeyspaceServedFromRequest - */ - SetKeyspaceServedFromRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceServedFromRequest) - return object; - let message = new $root.vtctldata.SetKeyspaceServedFromRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - switch (object.tablet_type) { - default: - if (typeof object.tablet_type === "number") { - message.tablet_type = object.tablet_type; - break; - } - break; - case "UNKNOWN": - case 0: - message.tablet_type = 0; - break; - case "PRIMARY": - case 1: - message.tablet_type = 1; - break; - case "MASTER": - case 1: - message.tablet_type = 1; - break; - case "REPLICA": - case 2: - message.tablet_type = 2; - break; - case "RDONLY": - case 3: - message.tablet_type = 3; - break; - case "BATCH": - case 3: - message.tablet_type = 3; - break; - case "SPARE": - case 4: - message.tablet_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_type = 5; - break; - case "BACKUP": - case 6: - message.tablet_type = 6; - break; - case "RESTORE": - case 7: - message.tablet_type = 7; - break; - case "DRAINED": - case 8: - message.tablet_type = 8; - break; + return message; + }; + + /** + * Decodes a SetWritableRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.SetWritableRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.SetWritableRequest} SetWritableRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + SetWritableRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a SetWritableRequest message. + * @function verify + * @memberof vtctldata.SetWritableRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + SetWritableRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; } - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.SetKeyspaceServedFromRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); + if (message.writable != null && message.hasOwnProperty("writable")) + if (typeof message.writable !== "boolean") + return "writable: boolean expected"; + return null; + }; + + /** + * Creates a SetWritableRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.SetWritableRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.SetWritableRequest} SetWritableRequest + */ + SetWritableRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetWritableRequest) + return object; + let message = new $root.vtctldata.SetWritableRequest(); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.SetWritableRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.remove != null) - message.remove = Boolean(object.remove); - if (object.source_keyspace != null) - message.source_keyspace = String(object.source_keyspace); + if (object.writable != null) + message.writable = Boolean(object.writable); return message; }; /** - * Creates a plain object from a SetKeyspaceServedFromRequest message. Also converts values to other types if specified. + * Creates a plain object from a SetWritableRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static - * @param {vtctldata.SetKeyspaceServedFromRequest} message SetKeyspaceServedFromRequest + * @param {vtctldata.SetWritableRequest} message SetWritableRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceServedFromRequest.toObject = function toObject(message, options) { + SetWritableRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.cells = []; if (options.defaults) { - object.keyspace = ""; - object.tablet_type = options.enums === String ? "UNKNOWN" : 0; - object.remove = false; - object.source_keyspace = ""; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; + object.tablet_alias = null; + object.writable = false; } - if (message.remove != null && message.hasOwnProperty("remove")) - object.remove = message.remove; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - object.source_keyspace = message.source_keyspace; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.writable != null && message.hasOwnProperty("writable")) + object.writable = message.writable; return object; }; /** - * Converts this SetKeyspaceServedFromRequest to JSON. + * Converts this SetWritableRequest to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @instance * @returns {Object.} JSON object */ - SetKeyspaceServedFromRequest.prototype.toJSON = function toJSON() { + SetWritableRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceServedFromRequest + * Gets the default type url for SetWritableRequest * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceServedFromRequest + * @memberof vtctldata.SetWritableRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceServedFromRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetWritableRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceServedFromRequest"; + return typeUrlPrefix + "/vtctldata.SetWritableRequest"; }; - return SetKeyspaceServedFromRequest; + return SetWritableRequest; })(); - vtctldata.SetKeyspaceServedFromResponse = (function() { + vtctldata.SetWritableResponse = (function() { /** - * Properties of a SetKeyspaceServedFromResponse. + * Properties of a SetWritableResponse. * @memberof vtctldata - * @interface ISetKeyspaceServedFromResponse - * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceServedFromResponse keyspace + * @interface ISetWritableResponse */ /** - * Constructs a new SetKeyspaceServedFromResponse. + * Constructs a new SetWritableResponse. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceServedFromResponse. - * @implements ISetKeyspaceServedFromResponse + * @classdesc Represents a SetWritableResponse. + * @implements ISetWritableResponse * @constructor - * @param {vtctldata.ISetKeyspaceServedFromResponse=} [properties] Properties to set + * @param {vtctldata.ISetWritableResponse=} [properties] Properties to set */ - function SetKeyspaceServedFromResponse(properties) { + function SetWritableResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -131985,77 +138713,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceServedFromResponse keyspace. - * @member {topodata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.SetKeyspaceServedFromResponse - * @instance - */ - SetKeyspaceServedFromResponse.prototype.keyspace = null; - - /** - * Creates a new SetKeyspaceServedFromResponse instance using the specified properties. + * Creates a new SetWritableResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static - * @param {vtctldata.ISetKeyspaceServedFromResponse=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse instance + * @param {vtctldata.ISetWritableResponse=} [properties] Properties to set + * @returns {vtctldata.SetWritableResponse} SetWritableResponse instance */ - SetKeyspaceServedFromResponse.create = function create(properties) { - return new SetKeyspaceServedFromResponse(properties); + SetWritableResponse.create = function create(properties) { + return new SetWritableResponse(properties); }; /** - * Encodes the specified SetKeyspaceServedFromResponse message. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. + * Encodes the specified SetWritableResponse message. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static - * @param {vtctldata.ISetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse message or plain object to encode + * @param {vtctldata.ISetWritableResponse} message SetWritableResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceServedFromResponse.encode = function encode(message, writer) { + SetWritableResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetKeyspaceServedFromResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceServedFromResponse.verify|verify} messages. + * Encodes the specified SetWritableResponse message, length delimited. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static - * @param {vtctldata.ISetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse message or plain object to encode + * @param {vtctldata.ISetWritableResponse} message SetWritableResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceServedFromResponse.encodeDelimited = function encodeDelimited(message, writer) { + SetWritableResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer. + * Decodes a SetWritableResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse + * @returns {vtctldata.SetWritableResponse} SetWritableResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceServedFromResponse.decode = function decode(reader, length) { + SetWritableResponse.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.SetKeyspaceServedFromResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SetWritableResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -132065,128 +138779,111 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetKeyspaceServedFromResponse message from the specified reader or buffer, length delimited. + * Decodes a SetWritableResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse + * @returns {vtctldata.SetWritableResponse} SetWritableResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceServedFromResponse.decodeDelimited = function decodeDelimited(reader) { + SetWritableResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetKeyspaceServedFromResponse message. + * Verifies a SetWritableResponse message. * @function verify - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetKeyspaceServedFromResponse.verify = function verify(message) { + SetWritableResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.topodata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; - } return null; }; /** - * Creates a SetKeyspaceServedFromResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SetWritableResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceServedFromResponse} SetKeyspaceServedFromResponse + * @returns {vtctldata.SetWritableResponse} SetWritableResponse */ - SetKeyspaceServedFromResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceServedFromResponse) + SetWritableResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SetWritableResponse) return object; - let message = new $root.vtctldata.SetKeyspaceServedFromResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.SetKeyspaceServedFromResponse.keyspace: object expected"); - message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); - } - return message; + return new $root.vtctldata.SetWritableResponse(); }; /** - * Creates a plain object from a SetKeyspaceServedFromResponse message. Also converts values to other types if specified. + * Creates a plain object from a SetWritableResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static - * @param {vtctldata.SetKeyspaceServedFromResponse} message SetKeyspaceServedFromResponse + * @param {vtctldata.SetWritableResponse} message SetWritableResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceServedFromResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.keyspace = null; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); - return object; + SetWritableResponse.toObject = function toObject() { + return {}; }; /** - * Converts this SetKeyspaceServedFromResponse to JSON. + * Converts this SetWritableResponse to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @instance * @returns {Object.} JSON object */ - SetKeyspaceServedFromResponse.prototype.toJSON = function toJSON() { + SetWritableResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceServedFromResponse + * Gets the default type url for SetWritableResponse * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceServedFromResponse + * @memberof vtctldata.SetWritableResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceServedFromResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SetWritableResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceServedFromResponse"; + return typeUrlPrefix + "/vtctldata.SetWritableResponse"; }; - return SetKeyspaceServedFromResponse; + return SetWritableResponse; })(); - vtctldata.SetKeyspaceShardingInfoRequest = (function() { + vtctldata.ShardReplicationAddRequest = (function() { /** - * Properties of a SetKeyspaceShardingInfoRequest. + * Properties of a ShardReplicationAddRequest. * @memberof vtctldata - * @interface ISetKeyspaceShardingInfoRequest - * @property {string|null} [keyspace] SetKeyspaceShardingInfoRequest keyspace - * @property {boolean|null} [force] SetKeyspaceShardingInfoRequest force + * @interface IShardReplicationAddRequest + * @property {string|null} [keyspace] ShardReplicationAddRequest keyspace + * @property {string|null} [shard] ShardReplicationAddRequest shard + * @property {topodata.ITabletAlias|null} [tablet_alias] ShardReplicationAddRequest tablet_alias */ /** - * Constructs a new SetKeyspaceShardingInfoRequest. + * Constructs a new ShardReplicationAddRequest. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceShardingInfoRequest. - * @implements ISetKeyspaceShardingInfoRequest + * @classdesc Represents a ShardReplicationAddRequest. + * @implements IShardReplicationAddRequest * @constructor - * @param {vtctldata.ISetKeyspaceShardingInfoRequest=} [properties] Properties to set + * @param {vtctldata.IShardReplicationAddRequest=} [properties] Properties to set */ - function SetKeyspaceShardingInfoRequest(properties) { + function ShardReplicationAddRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -132194,80 +138891,90 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceShardingInfoRequest keyspace. + * ShardReplicationAddRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @instance */ - SetKeyspaceShardingInfoRequest.prototype.keyspace = ""; + ShardReplicationAddRequest.prototype.keyspace = ""; /** - * SetKeyspaceShardingInfoRequest force. - * @member {boolean} force - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * ShardReplicationAddRequest shard. + * @member {string} shard + * @memberof vtctldata.ShardReplicationAddRequest * @instance */ - SetKeyspaceShardingInfoRequest.prototype.force = false; + ShardReplicationAddRequest.prototype.shard = ""; /** - * Creates a new SetKeyspaceShardingInfoRequest instance using the specified properties. + * ShardReplicationAddRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ShardReplicationAddRequest + * @instance + */ + ShardReplicationAddRequest.prototype.tablet_alias = null; + + /** + * Creates a new ShardReplicationAddRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static - * @param {vtctldata.ISetKeyspaceShardingInfoRequest=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest instance + * @param {vtctldata.IShardReplicationAddRequest=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest instance */ - SetKeyspaceShardingInfoRequest.create = function create(properties) { - return new SetKeyspaceShardingInfoRequest(properties); + ShardReplicationAddRequest.create = function create(properties) { + return new ShardReplicationAddRequest(properties); }; /** - * Encodes the specified SetKeyspaceShardingInfoRequest message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. + * Encodes the specified ShardReplicationAddRequest message. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static - * @param {vtctldata.ISetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest message or plain object to encode + * @param {vtctldata.IShardReplicationAddRequest} message ShardReplicationAddRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceShardingInfoRequest.encode = function encode(message, writer) { + ShardReplicationAddRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.force != null && Object.hasOwnProperty.call(message, "force")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.force); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetKeyspaceShardingInfoRequest message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoRequest.verify|verify} messages. + * Encodes the specified ShardReplicationAddRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static - * @param {vtctldata.ISetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest message or plain object to encode + * @param {vtctldata.IShardReplicationAddRequest} message ShardReplicationAddRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceShardingInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationAddRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer. + * Decodes a ShardReplicationAddRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest + * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceShardingInfoRequest.decode = function decode(reader, length) { + ShardReplicationAddRequest.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.SetKeyspaceShardingInfoRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationAddRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -132275,8 +138982,12 @@ export const vtctldata = $root.vtctldata = (() => { message.keyspace = reader.string(); break; } - case 4: { - message.force = reader.bool(); + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -132288,131 +138999,143 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetKeyspaceShardingInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationAddRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest + * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceShardingInfoRequest.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationAddRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetKeyspaceShardingInfoRequest message. + * Verifies a ShardReplicationAddRequest message. * @function verify - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetKeyspaceShardingInfoRequest.verify = function verify(message) { + ShardReplicationAddRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.force != null && message.hasOwnProperty("force")) - if (typeof message.force !== "boolean") - return "force: boolean expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { + let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (error) + return "tablet_alias." + error; + } return null; }; /** - * Creates a SetKeyspaceShardingInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationAddRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceShardingInfoRequest} SetKeyspaceShardingInfoRequest + * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest */ - SetKeyspaceShardingInfoRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceShardingInfoRequest) + ShardReplicationAddRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationAddRequest) return object; - let message = new $root.vtctldata.SetKeyspaceShardingInfoRequest(); + let message = new $root.vtctldata.ShardReplicationAddRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.force != null) - message.force = Boolean(object.force); + if (object.shard != null) + message.shard = String(object.shard); + if (object.tablet_alias != null) { + if (typeof object.tablet_alias !== "object") + throw TypeError(".vtctldata.ShardReplicationAddRequest.tablet_alias: object expected"); + message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + } return message; }; /** - * Creates a plain object from a SetKeyspaceShardingInfoRequest message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationAddRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static - * @param {vtctldata.SetKeyspaceShardingInfoRequest} message SetKeyspaceShardingInfoRequest + * @param {vtctldata.ShardReplicationAddRequest} message ShardReplicationAddRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceShardingInfoRequest.toObject = function toObject(message, options) { + ShardReplicationAddRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; - object.force = false; + object.shard = ""; + object.tablet_alias = null; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.force != null && message.hasOwnProperty("force")) - object.force = message.force; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) + object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this SetKeyspaceShardingInfoRequest to JSON. + * Converts this ShardReplicationAddRequest to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @instance * @returns {Object.} JSON object */ - SetKeyspaceShardingInfoRequest.prototype.toJSON = function toJSON() { + ShardReplicationAddRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceShardingInfoRequest + * Gets the default type url for ShardReplicationAddRequest * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceShardingInfoRequest + * @memberof vtctldata.ShardReplicationAddRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceShardingInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationAddRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceShardingInfoRequest"; + return typeUrlPrefix + "/vtctldata.ShardReplicationAddRequest"; }; - return SetKeyspaceShardingInfoRequest; + return ShardReplicationAddRequest; })(); - vtctldata.SetKeyspaceShardingInfoResponse = (function() { + vtctldata.ShardReplicationAddResponse = (function() { /** - * Properties of a SetKeyspaceShardingInfoResponse. + * Properties of a ShardReplicationAddResponse. * @memberof vtctldata - * @interface ISetKeyspaceShardingInfoResponse - * @property {topodata.IKeyspace|null} [keyspace] SetKeyspaceShardingInfoResponse keyspace + * @interface IShardReplicationAddResponse */ /** - * Constructs a new SetKeyspaceShardingInfoResponse. + * Constructs a new ShardReplicationAddResponse. * @memberof vtctldata - * @classdesc Represents a SetKeyspaceShardingInfoResponse. - * @implements ISetKeyspaceShardingInfoResponse + * @classdesc Represents a ShardReplicationAddResponse. + * @implements IShardReplicationAddResponse * @constructor - * @param {vtctldata.ISetKeyspaceShardingInfoResponse=} [properties] Properties to set + * @param {vtctldata.IShardReplicationAddResponse=} [properties] Properties to set */ - function SetKeyspaceShardingInfoResponse(properties) { + function ShardReplicationAddResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -132420,77 +139143,63 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetKeyspaceShardingInfoResponse keyspace. - * @member {topodata.IKeyspace|null|undefined} keyspace - * @memberof vtctldata.SetKeyspaceShardingInfoResponse - * @instance - */ - SetKeyspaceShardingInfoResponse.prototype.keyspace = null; - - /** - * Creates a new SetKeyspaceShardingInfoResponse instance using the specified properties. + * Creates a new ShardReplicationAddResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static - * @param {vtctldata.ISetKeyspaceShardingInfoResponse=} [properties] Properties to set - * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse instance + * @param {vtctldata.IShardReplicationAddResponse=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse instance */ - SetKeyspaceShardingInfoResponse.create = function create(properties) { - return new SetKeyspaceShardingInfoResponse(properties); + ShardReplicationAddResponse.create = function create(properties) { + return new ShardReplicationAddResponse(properties); }; /** - * Encodes the specified SetKeyspaceShardingInfoResponse message. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. + * Encodes the specified ShardReplicationAddResponse message. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static - * @param {vtctldata.ISetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse message or plain object to encode + * @param {vtctldata.IShardReplicationAddResponse} message ShardReplicationAddResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceShardingInfoResponse.encode = function encode(message, writer) { + ShardReplicationAddResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - $root.topodata.Keyspace.encode(message.keyspace, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetKeyspaceShardingInfoResponse message, length delimited. Does not implicitly {@link vtctldata.SetKeyspaceShardingInfoResponse.verify|verify} messages. + * Encodes the specified ShardReplicationAddResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static - * @param {vtctldata.ISetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse message or plain object to encode + * @param {vtctldata.IShardReplicationAddResponse} message ShardReplicationAddResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetKeyspaceShardingInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationAddResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer. + * Decodes a ShardReplicationAddResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse + * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceShardingInfoResponse.decode = function decode(reader, length) { + ShardReplicationAddResponse.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.SetKeyspaceShardingInfoResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationAddResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.keyspace = $root.topodata.Keyspace.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -132500,129 +139209,111 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetKeyspaceShardingInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationAddResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse + * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetKeyspaceShardingInfoResponse.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationAddResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetKeyspaceShardingInfoResponse message. + * Verifies a ShardReplicationAddResponse message. * @function verify - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetKeyspaceShardingInfoResponse.verify = function verify(message) { + ShardReplicationAddResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) { - let error = $root.topodata.Keyspace.verify(message.keyspace); - if (error) - return "keyspace." + error; - } return null; }; /** - * Creates a SetKeyspaceShardingInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationAddResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetKeyspaceShardingInfoResponse} SetKeyspaceShardingInfoResponse + * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse */ - SetKeyspaceShardingInfoResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetKeyspaceShardingInfoResponse) + ShardReplicationAddResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationAddResponse) return object; - let message = new $root.vtctldata.SetKeyspaceShardingInfoResponse(); - if (object.keyspace != null) { - if (typeof object.keyspace !== "object") - throw TypeError(".vtctldata.SetKeyspaceShardingInfoResponse.keyspace: object expected"); - message.keyspace = $root.topodata.Keyspace.fromObject(object.keyspace); - } - return message; + return new $root.vtctldata.ShardReplicationAddResponse(); }; /** - * Creates a plain object from a SetKeyspaceShardingInfoResponse message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationAddResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static - * @param {vtctldata.SetKeyspaceShardingInfoResponse} message SetKeyspaceShardingInfoResponse + * @param {vtctldata.ShardReplicationAddResponse} message ShardReplicationAddResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetKeyspaceShardingInfoResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.keyspace = null; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = $root.topodata.Keyspace.toObject(message.keyspace, options); - return object; + ShardReplicationAddResponse.toObject = function toObject() { + return {}; }; /** - * Converts this SetKeyspaceShardingInfoResponse to JSON. + * Converts this ShardReplicationAddResponse to JSON. * @function toJSON - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @instance * @returns {Object.} JSON object */ - SetKeyspaceShardingInfoResponse.prototype.toJSON = function toJSON() { + ShardReplicationAddResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetKeyspaceShardingInfoResponse + * Gets the default type url for ShardReplicationAddResponse * @function getTypeUrl - * @memberof vtctldata.SetKeyspaceShardingInfoResponse + * @memberof vtctldata.ShardReplicationAddResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetKeyspaceShardingInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationAddResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetKeyspaceShardingInfoResponse"; + return typeUrlPrefix + "/vtctldata.ShardReplicationAddResponse"; }; - return SetKeyspaceShardingInfoResponse; + return ShardReplicationAddResponse; })(); - vtctldata.SetShardIsPrimaryServingRequest = (function() { + vtctldata.ShardReplicationFixRequest = (function() { /** - * Properties of a SetShardIsPrimaryServingRequest. + * Properties of a ShardReplicationFixRequest. * @memberof vtctldata - * @interface ISetShardIsPrimaryServingRequest - * @property {string|null} [keyspace] SetShardIsPrimaryServingRequest keyspace - * @property {string|null} [shard] SetShardIsPrimaryServingRequest shard - * @property {boolean|null} [is_serving] SetShardIsPrimaryServingRequest is_serving + * @interface IShardReplicationFixRequest + * @property {string|null} [keyspace] ShardReplicationFixRequest keyspace + * @property {string|null} [shard] ShardReplicationFixRequest shard + * @property {string|null} [cell] ShardReplicationFixRequest cell */ /** - * Constructs a new SetShardIsPrimaryServingRequest. + * Constructs a new ShardReplicationFixRequest. * @memberof vtctldata - * @classdesc Represents a SetShardIsPrimaryServingRequest. - * @implements ISetShardIsPrimaryServingRequest + * @classdesc Represents a ShardReplicationFixRequest. + * @implements IShardReplicationFixRequest * @constructor - * @param {vtctldata.ISetShardIsPrimaryServingRequest=} [properties] Properties to set + * @param {vtctldata.IShardReplicationFixRequest=} [properties] Properties to set */ - function SetShardIsPrimaryServingRequest(properties) { + function ShardReplicationFixRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -132630,90 +139321,90 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetShardIsPrimaryServingRequest keyspace. + * ShardReplicationFixRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @instance */ - SetShardIsPrimaryServingRequest.prototype.keyspace = ""; + ShardReplicationFixRequest.prototype.keyspace = ""; /** - * SetShardIsPrimaryServingRequest shard. + * ShardReplicationFixRequest shard. * @member {string} shard - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @instance */ - SetShardIsPrimaryServingRequest.prototype.shard = ""; + ShardReplicationFixRequest.prototype.shard = ""; /** - * SetShardIsPrimaryServingRequest is_serving. - * @member {boolean} is_serving - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * ShardReplicationFixRequest cell. + * @member {string} cell + * @memberof vtctldata.ShardReplicationFixRequest * @instance */ - SetShardIsPrimaryServingRequest.prototype.is_serving = false; + ShardReplicationFixRequest.prototype.cell = ""; /** - * Creates a new SetShardIsPrimaryServingRequest instance using the specified properties. + * Creates a new ShardReplicationFixRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static - * @param {vtctldata.ISetShardIsPrimaryServingRequest=} [properties] Properties to set - * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest instance + * @param {vtctldata.IShardReplicationFixRequest=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest instance */ - SetShardIsPrimaryServingRequest.create = function create(properties) { - return new SetShardIsPrimaryServingRequest(properties); + ShardReplicationFixRequest.create = function create(properties) { + return new ShardReplicationFixRequest(properties); }; /** - * Encodes the specified SetShardIsPrimaryServingRequest message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. + * Encodes the specified ShardReplicationFixRequest message. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static - * @param {vtctldata.ISetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest message or plain object to encode + * @param {vtctldata.IShardReplicationFixRequest} message ShardReplicationFixRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardIsPrimaryServingRequest.encode = function encode(message, writer) { + ShardReplicationFixRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.is_serving != null && Object.hasOwnProperty.call(message, "is_serving")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.is_serving); + if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cell); return writer; }; /** - * Encodes the specified SetShardIsPrimaryServingRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingRequest.verify|verify} messages. + * Encodes the specified ShardReplicationFixRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static - * @param {vtctldata.ISetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest message or plain object to encode + * @param {vtctldata.IShardReplicationFixRequest} message ShardReplicationFixRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardIsPrimaryServingRequest.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationFixRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer. + * Decodes a ShardReplicationFixRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest + * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardIsPrimaryServingRequest.decode = function decode(reader, length) { + ShardReplicationFixRequest.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.SetShardIsPrimaryServingRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationFixRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -132726,7 +139417,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 3: { - message.is_serving = reader.bool(); + message.cell = reader.string(); break; } default: @@ -132738,30 +139429,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetShardIsPrimaryServingRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationFixRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest + * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardIsPrimaryServingRequest.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationFixRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetShardIsPrimaryServingRequest message. + * Verifies a ShardReplicationFixRequest message. * @function verify - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetShardIsPrimaryServingRequest.verify = function verify(message) { + ShardReplicationFixRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -132770,107 +139461,107 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.is_serving != null && message.hasOwnProperty("is_serving")) - if (typeof message.is_serving !== "boolean") - return "is_serving: boolean expected"; + if (message.cell != null && message.hasOwnProperty("cell")) + if (!$util.isString(message.cell)) + return "cell: string expected"; return null; }; /** - * Creates a SetShardIsPrimaryServingRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationFixRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetShardIsPrimaryServingRequest} SetShardIsPrimaryServingRequest + * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest */ - SetShardIsPrimaryServingRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetShardIsPrimaryServingRequest) + ShardReplicationFixRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationFixRequest) return object; - let message = new $root.vtctldata.SetShardIsPrimaryServingRequest(); + let message = new $root.vtctldata.ShardReplicationFixRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - if (object.is_serving != null) - message.is_serving = Boolean(object.is_serving); + if (object.cell != null) + message.cell = String(object.cell); return message; }; /** - * Creates a plain object from a SetShardIsPrimaryServingRequest message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationFixRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static - * @param {vtctldata.SetShardIsPrimaryServingRequest} message SetShardIsPrimaryServingRequest + * @param {vtctldata.ShardReplicationFixRequest} message ShardReplicationFixRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetShardIsPrimaryServingRequest.toObject = function toObject(message, options) { + ShardReplicationFixRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; object.shard = ""; - object.is_serving = false; + object.cell = ""; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.is_serving != null && message.hasOwnProperty("is_serving")) - object.is_serving = message.is_serving; + if (message.cell != null && message.hasOwnProperty("cell")) + object.cell = message.cell; return object; }; /** - * Converts this SetShardIsPrimaryServingRequest to JSON. + * Converts this ShardReplicationFixRequest to JSON. * @function toJSON - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @instance * @returns {Object.} JSON object */ - SetShardIsPrimaryServingRequest.prototype.toJSON = function toJSON() { + ShardReplicationFixRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetShardIsPrimaryServingRequest + * Gets the default type url for ShardReplicationFixRequest * @function getTypeUrl - * @memberof vtctldata.SetShardIsPrimaryServingRequest + * @memberof vtctldata.ShardReplicationFixRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetShardIsPrimaryServingRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationFixRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetShardIsPrimaryServingRequest"; + return typeUrlPrefix + "/vtctldata.ShardReplicationFixRequest"; }; - return SetShardIsPrimaryServingRequest; + return ShardReplicationFixRequest; })(); - vtctldata.SetShardIsPrimaryServingResponse = (function() { + vtctldata.ShardReplicationFixResponse = (function() { /** - * Properties of a SetShardIsPrimaryServingResponse. + * Properties of a ShardReplicationFixResponse. * @memberof vtctldata - * @interface ISetShardIsPrimaryServingResponse - * @property {topodata.IShard|null} [shard] SetShardIsPrimaryServingResponse shard + * @interface IShardReplicationFixResponse + * @property {topodata.IShardReplicationError|null} [error] ShardReplicationFixResponse error */ /** - * Constructs a new SetShardIsPrimaryServingResponse. + * Constructs a new ShardReplicationFixResponse. * @memberof vtctldata - * @classdesc Represents a SetShardIsPrimaryServingResponse. - * @implements ISetShardIsPrimaryServingResponse + * @classdesc Represents a ShardReplicationFixResponse. + * @implements IShardReplicationFixResponse * @constructor - * @param {vtctldata.ISetShardIsPrimaryServingResponse=} [properties] Properties to set + * @param {vtctldata.IShardReplicationFixResponse=} [properties] Properties to set */ - function SetShardIsPrimaryServingResponse(properties) { + function ShardReplicationFixResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -132878,75 +139569,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetShardIsPrimaryServingResponse shard. - * @member {topodata.IShard|null|undefined} shard - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * ShardReplicationFixResponse error. + * @member {topodata.IShardReplicationError|null|undefined} error + * @memberof vtctldata.ShardReplicationFixResponse * @instance */ - SetShardIsPrimaryServingResponse.prototype.shard = null; + ShardReplicationFixResponse.prototype.error = null; /** - * Creates a new SetShardIsPrimaryServingResponse instance using the specified properties. + * Creates a new ShardReplicationFixResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static - * @param {vtctldata.ISetShardIsPrimaryServingResponse=} [properties] Properties to set - * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse instance + * @param {vtctldata.IShardReplicationFixResponse=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse instance */ - SetShardIsPrimaryServingResponse.create = function create(properties) { - return new SetShardIsPrimaryServingResponse(properties); + ShardReplicationFixResponse.create = function create(properties) { + return new ShardReplicationFixResponse(properties); }; /** - * Encodes the specified SetShardIsPrimaryServingResponse message. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. + * Encodes the specified ShardReplicationFixResponse message. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static - * @param {vtctldata.ISetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse message or plain object to encode + * @param {vtctldata.IShardReplicationFixResponse} message ShardReplicationFixResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardIsPrimaryServingResponse.encode = function encode(message, writer) { + ShardReplicationFixResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.error != null && Object.hasOwnProperty.call(message, "error")) + $root.topodata.ShardReplicationError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetShardIsPrimaryServingResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardIsPrimaryServingResponse.verify|verify} messages. + * Encodes the specified ShardReplicationFixResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static - * @param {vtctldata.ISetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse message or plain object to encode + * @param {vtctldata.IShardReplicationFixResponse} message ShardReplicationFixResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardIsPrimaryServingResponse.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationFixResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer. + * Decodes a ShardReplicationFixResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse + * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardIsPrimaryServingResponse.decode = function decode(reader, length) { + ShardReplicationFixResponse.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.SetShardIsPrimaryServingResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationFixResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + message.error = $root.topodata.ShardReplicationError.decode(reader, reader.uint32()); break; } default: @@ -132958,135 +139649,128 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetShardIsPrimaryServingResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationFixResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse + * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardIsPrimaryServingResponse.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationFixResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetShardIsPrimaryServingResponse message. + * Verifies a ShardReplicationFixResponse message. * @function verify - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetShardIsPrimaryServingResponse.verify = function verify(message) { + ShardReplicationFixResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.topodata.Shard.verify(message.shard); + if (message.error != null && message.hasOwnProperty("error")) { + let error = $root.topodata.ShardReplicationError.verify(message.error); if (error) - return "shard." + error; + return "error." + error; } return null; }; /** - * Creates a SetShardIsPrimaryServingResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationFixResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetShardIsPrimaryServingResponse} SetShardIsPrimaryServingResponse + * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse */ - SetShardIsPrimaryServingResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetShardIsPrimaryServingResponse) + ShardReplicationFixResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationFixResponse) return object; - let message = new $root.vtctldata.SetShardIsPrimaryServingResponse(); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.SetShardIsPrimaryServingResponse.shard: object expected"); - message.shard = $root.topodata.Shard.fromObject(object.shard); + let message = new $root.vtctldata.ShardReplicationFixResponse(); + if (object.error != null) { + if (typeof object.error !== "object") + throw TypeError(".vtctldata.ShardReplicationFixResponse.error: object expected"); + message.error = $root.topodata.ShardReplicationError.fromObject(object.error); } return message; }; /** - * Creates a plain object from a SetShardIsPrimaryServingResponse message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationFixResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static - * @param {vtctldata.SetShardIsPrimaryServingResponse} message SetShardIsPrimaryServingResponse + * @param {vtctldata.ShardReplicationFixResponse} message ShardReplicationFixResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetShardIsPrimaryServingResponse.toObject = function toObject(message, options) { + ShardReplicationFixResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.shard = null; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.topodata.Shard.toObject(message.shard, options); + object.error = null; + if (message.error != null && message.hasOwnProperty("error")) + object.error = $root.topodata.ShardReplicationError.toObject(message.error, options); return object; }; /** - * Converts this SetShardIsPrimaryServingResponse to JSON. + * Converts this ShardReplicationFixResponse to JSON. * @function toJSON - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @instance * @returns {Object.} JSON object */ - SetShardIsPrimaryServingResponse.prototype.toJSON = function toJSON() { + ShardReplicationFixResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetShardIsPrimaryServingResponse + * Gets the default type url for ShardReplicationFixResponse * @function getTypeUrl - * @memberof vtctldata.SetShardIsPrimaryServingResponse + * @memberof vtctldata.ShardReplicationFixResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetShardIsPrimaryServingResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationFixResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetShardIsPrimaryServingResponse"; + return typeUrlPrefix + "/vtctldata.ShardReplicationFixResponse"; }; - return SetShardIsPrimaryServingResponse; + return ShardReplicationFixResponse; })(); - vtctldata.SetShardTabletControlRequest = (function() { + vtctldata.ShardReplicationPositionsRequest = (function() { /** - * Properties of a SetShardTabletControlRequest. + * Properties of a ShardReplicationPositionsRequest. * @memberof vtctldata - * @interface ISetShardTabletControlRequest - * @property {string|null} [keyspace] SetShardTabletControlRequest keyspace - * @property {string|null} [shard] SetShardTabletControlRequest shard - * @property {topodata.TabletType|null} [tablet_type] SetShardTabletControlRequest tablet_type - * @property {Array.|null} [cells] SetShardTabletControlRequest cells - * @property {Array.|null} [denied_tables] SetShardTabletControlRequest denied_tables - * @property {boolean|null} [disable_query_service] SetShardTabletControlRequest disable_query_service - * @property {boolean|null} [remove] SetShardTabletControlRequest remove + * @interface IShardReplicationPositionsRequest + * @property {string|null} [keyspace] ShardReplicationPositionsRequest keyspace + * @property {string|null} [shard] ShardReplicationPositionsRequest shard */ /** - * Constructs a new SetShardTabletControlRequest. + * Constructs a new ShardReplicationPositionsRequest. * @memberof vtctldata - * @classdesc Represents a SetShardTabletControlRequest. - * @implements ISetShardTabletControlRequest + * @classdesc Represents a ShardReplicationPositionsRequest. + * @implements IShardReplicationPositionsRequest * @constructor - * @param {vtctldata.ISetShardTabletControlRequest=} [properties] Properties to set + * @param {vtctldata.IShardReplicationPositionsRequest=} [properties] Properties to set */ - function SetShardTabletControlRequest(properties) { - this.cells = []; - this.denied_tables = []; + function ShardReplicationPositionsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -133094,165 +139778,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetShardTabletControlRequest keyspace. + * ShardReplicationPositionsRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @instance */ - SetShardTabletControlRequest.prototype.keyspace = ""; + ShardReplicationPositionsRequest.prototype.keyspace = ""; /** - * SetShardTabletControlRequest shard. + * ShardReplicationPositionsRequest shard. * @member {string} shard - * @memberof vtctldata.SetShardTabletControlRequest - * @instance - */ - SetShardTabletControlRequest.prototype.shard = ""; - - /** - * SetShardTabletControlRequest tablet_type. - * @member {topodata.TabletType} tablet_type - * @memberof vtctldata.SetShardTabletControlRequest - * @instance - */ - SetShardTabletControlRequest.prototype.tablet_type = 0; - - /** - * SetShardTabletControlRequest cells. - * @member {Array.} cells - * @memberof vtctldata.SetShardTabletControlRequest - * @instance - */ - SetShardTabletControlRequest.prototype.cells = $util.emptyArray; - - /** - * SetShardTabletControlRequest denied_tables. - * @member {Array.} denied_tables - * @memberof vtctldata.SetShardTabletControlRequest - * @instance - */ - SetShardTabletControlRequest.prototype.denied_tables = $util.emptyArray; - - /** - * SetShardTabletControlRequest disable_query_service. - * @member {boolean} disable_query_service - * @memberof vtctldata.SetShardTabletControlRequest - * @instance - */ - SetShardTabletControlRequest.prototype.disable_query_service = false; - - /** - * SetShardTabletControlRequest remove. - * @member {boolean} remove - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @instance */ - SetShardTabletControlRequest.prototype.remove = false; + ShardReplicationPositionsRequest.prototype.shard = ""; /** - * Creates a new SetShardTabletControlRequest instance using the specified properties. + * Creates a new ShardReplicationPositionsRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static - * @param {vtctldata.ISetShardTabletControlRequest=} [properties] Properties to set - * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest instance + * @param {vtctldata.IShardReplicationPositionsRequest=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest instance */ - SetShardTabletControlRequest.create = function create(properties) { - return new SetShardTabletControlRequest(properties); + ShardReplicationPositionsRequest.create = function create(properties) { + return new ShardReplicationPositionsRequest(properties); }; /** - * Encodes the specified SetShardTabletControlRequest message. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. + * Encodes the specified ShardReplicationPositionsRequest message. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static - * @param {vtctldata.ISetShardTabletControlRequest} message SetShardTabletControlRequest message or plain object to encode + * @param {vtctldata.IShardReplicationPositionsRequest} message ShardReplicationPositionsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardTabletControlRequest.encode = function encode(message, writer) { + ShardReplicationPositionsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.tablet_type != null && Object.hasOwnProperty.call(message, "tablet_type")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.tablet_type); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.cells[i]); - if (message.denied_tables != null && message.denied_tables.length) - for (let i = 0; i < message.denied_tables.length; ++i) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.denied_tables[i]); - if (message.disable_query_service != null && Object.hasOwnProperty.call(message, "disable_query_service")) - writer.uint32(/* id 6, wireType 0 =*/48).bool(message.disable_query_service); - if (message.remove != null && Object.hasOwnProperty.call(message, "remove")) - writer.uint32(/* id 7, wireType 0 =*/56).bool(message.remove); return writer; }; /** - * Encodes the specified SetShardTabletControlRequest message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlRequest.verify|verify} messages. + * Encodes the specified ShardReplicationPositionsRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static - * @param {vtctldata.ISetShardTabletControlRequest} message SetShardTabletControlRequest message or plain object to encode + * @param {vtctldata.IShardReplicationPositionsRequest} message ShardReplicationPositionsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardTabletControlRequest.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationPositionsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetShardTabletControlRequest message from the specified reader or buffer. + * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetShardTabletControlRequest - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - SetShardTabletControlRequest.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.SetShardTabletControlRequest(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.tablet_type = reader.int32(); - break; - } - case 4: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); - break; - } - case 5: { - if (!(message.denied_tables && message.denied_tables.length)) - message.denied_tables = []; - message.denied_tables.push(reader.string()); - break; - } - case 6: { - message.disable_query_service = reader.bool(); + * @memberof vtctldata.ShardReplicationPositionsRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardReplicationPositionsRequest.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.ShardReplicationPositionsRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); break; } - case 7: { - message.remove = reader.bool(); + case 2: { + message.shard = reader.string(); break; } default: @@ -133264,30 +139872,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetShardTabletControlRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest + * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardTabletControlRequest.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationPositionsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetShardTabletControlRequest message. + * Verifies a ShardReplicationPositionsRequest message. * @function verify - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetShardTabletControlRequest.verify = function verify(message) { + ShardReplicationPositionsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -133296,229 +139904,102 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - switch (message.tablet_type) { - default: - return "tablet_type: enum value expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } - if (message.denied_tables != null && message.hasOwnProperty("denied_tables")) { - if (!Array.isArray(message.denied_tables)) - return "denied_tables: array expected"; - for (let i = 0; i < message.denied_tables.length; ++i) - if (!$util.isString(message.denied_tables[i])) - return "denied_tables: string[] expected"; - } - if (message.disable_query_service != null && message.hasOwnProperty("disable_query_service")) - if (typeof message.disable_query_service !== "boolean") - return "disable_query_service: boolean expected"; - if (message.remove != null && message.hasOwnProperty("remove")) - if (typeof message.remove !== "boolean") - return "remove: boolean expected"; return null; }; /** - * Creates a SetShardTabletControlRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationPositionsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetShardTabletControlRequest} SetShardTabletControlRequest + * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest */ - SetShardTabletControlRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetShardTabletControlRequest) + ShardReplicationPositionsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationPositionsRequest) return object; - let message = new $root.vtctldata.SetShardTabletControlRequest(); + let message = new $root.vtctldata.ShardReplicationPositionsRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - switch (object.tablet_type) { - default: - if (typeof object.tablet_type === "number") { - message.tablet_type = object.tablet_type; - break; - } - break; - case "UNKNOWN": - case 0: - message.tablet_type = 0; - break; - case "PRIMARY": - case 1: - message.tablet_type = 1; - break; - case "MASTER": - case 1: - message.tablet_type = 1; - break; - case "REPLICA": - case 2: - message.tablet_type = 2; - break; - case "RDONLY": - case 3: - message.tablet_type = 3; - break; - case "BATCH": - case 3: - message.tablet_type = 3; - break; - case "SPARE": - case 4: - message.tablet_type = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_type = 5; - break; - case "BACKUP": - case 6: - message.tablet_type = 6; - break; - case "RESTORE": - case 7: - message.tablet_type = 7; - break; - case "DRAINED": - case 8: - message.tablet_type = 8; - break; - } - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".vtctldata.SetShardTabletControlRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } - if (object.denied_tables) { - if (!Array.isArray(object.denied_tables)) - throw TypeError(".vtctldata.SetShardTabletControlRequest.denied_tables: array expected"); - message.denied_tables = []; - for (let i = 0; i < object.denied_tables.length; ++i) - message.denied_tables[i] = String(object.denied_tables[i]); - } - if (object.disable_query_service != null) - message.disable_query_service = Boolean(object.disable_query_service); - if (object.remove != null) - message.remove = Boolean(object.remove); return message; }; /** - * Creates a plain object from a SetShardTabletControlRequest message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationPositionsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static - * @param {vtctldata.SetShardTabletControlRequest} message SetShardTabletControlRequest + * @param {vtctldata.ShardReplicationPositionsRequest} message ShardReplicationPositionsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetShardTabletControlRequest.toObject = function toObject(message, options) { + ShardReplicationPositionsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.cells = []; - object.denied_tables = []; - } if (options.defaults) { object.keyspace = ""; object.shard = ""; - object.tablet_type = options.enums === String ? "UNKNOWN" : 0; - object.disable_query_service = false; - object.remove = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.tablet_type != null && message.hasOwnProperty("tablet_type")) - object.tablet_type = options.enums === String ? $root.topodata.TabletType[message.tablet_type] === undefined ? message.tablet_type : $root.topodata.TabletType[message.tablet_type] : message.tablet_type; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; - } - if (message.denied_tables && message.denied_tables.length) { - object.denied_tables = []; - for (let j = 0; j < message.denied_tables.length; ++j) - object.denied_tables[j] = message.denied_tables[j]; - } - if (message.disable_query_service != null && message.hasOwnProperty("disable_query_service")) - object.disable_query_service = message.disable_query_service; - if (message.remove != null && message.hasOwnProperty("remove")) - object.remove = message.remove; return object; }; /** - * Converts this SetShardTabletControlRequest to JSON. + * Converts this ShardReplicationPositionsRequest to JSON. * @function toJSON - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @instance * @returns {Object.} JSON object */ - SetShardTabletControlRequest.prototype.toJSON = function toJSON() { + ShardReplicationPositionsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetShardTabletControlRequest + * Gets the default type url for ShardReplicationPositionsRequest * @function getTypeUrl - * @memberof vtctldata.SetShardTabletControlRequest + * @memberof vtctldata.ShardReplicationPositionsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetShardTabletControlRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationPositionsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetShardTabletControlRequest"; + return typeUrlPrefix + "/vtctldata.ShardReplicationPositionsRequest"; }; - return SetShardTabletControlRequest; + return ShardReplicationPositionsRequest; })(); - vtctldata.SetShardTabletControlResponse = (function() { + vtctldata.ShardReplicationPositionsResponse = (function() { /** - * Properties of a SetShardTabletControlResponse. + * Properties of a ShardReplicationPositionsResponse. * @memberof vtctldata - * @interface ISetShardTabletControlResponse - * @property {topodata.IShard|null} [shard] SetShardTabletControlResponse shard + * @interface IShardReplicationPositionsResponse + * @property {Object.|null} [replication_statuses] ShardReplicationPositionsResponse replication_statuses + * @property {Object.|null} [tablet_map] ShardReplicationPositionsResponse tablet_map */ /** - * Constructs a new SetShardTabletControlResponse. + * Constructs a new ShardReplicationPositionsResponse. * @memberof vtctldata - * @classdesc Represents a SetShardTabletControlResponse. - * @implements ISetShardTabletControlResponse + * @classdesc Represents a ShardReplicationPositionsResponse. + * @implements IShardReplicationPositionsResponse * @constructor - * @param {vtctldata.ISetShardTabletControlResponse=} [properties] Properties to set + * @param {vtctldata.IShardReplicationPositionsResponse=} [properties] Properties to set */ - function SetShardTabletControlResponse(properties) { + function ShardReplicationPositionsResponse(properties) { + this.replication_statuses = {}; + this.tablet_map = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -133526,75 +140007,133 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetShardTabletControlResponse shard. - * @member {topodata.IShard|null|undefined} shard - * @memberof vtctldata.SetShardTabletControlResponse + * ShardReplicationPositionsResponse replication_statuses. + * @member {Object.} replication_statuses + * @memberof vtctldata.ShardReplicationPositionsResponse * @instance */ - SetShardTabletControlResponse.prototype.shard = null; + ShardReplicationPositionsResponse.prototype.replication_statuses = $util.emptyObject; /** - * Creates a new SetShardTabletControlResponse instance using the specified properties. + * ShardReplicationPositionsResponse tablet_map. + * @member {Object.} tablet_map + * @memberof vtctldata.ShardReplicationPositionsResponse + * @instance + */ + ShardReplicationPositionsResponse.prototype.tablet_map = $util.emptyObject; + + /** + * Creates a new ShardReplicationPositionsResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static - * @param {vtctldata.ISetShardTabletControlResponse=} [properties] Properties to set - * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse instance + * @param {vtctldata.IShardReplicationPositionsResponse=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse instance */ - SetShardTabletControlResponse.create = function create(properties) { - return new SetShardTabletControlResponse(properties); + ShardReplicationPositionsResponse.create = function create(properties) { + return new ShardReplicationPositionsResponse(properties); }; /** - * Encodes the specified SetShardTabletControlResponse message. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. + * Encodes the specified ShardReplicationPositionsResponse message. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static - * @param {vtctldata.ISetShardTabletControlResponse} message SetShardTabletControlResponse message or plain object to encode + * @param {vtctldata.IShardReplicationPositionsResponse} message ShardReplicationPositionsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardTabletControlResponse.encode = function encode(message, writer) { + ShardReplicationPositionsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.replication_statuses != null && Object.hasOwnProperty.call(message, "replication_statuses")) + for (let keys = Object.keys(message.replication_statuses), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.replicationdata.Status.encode(message.replication_statuses[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.tablet_map != null && Object.hasOwnProperty.call(message, "tablet_map")) + for (let keys = Object.keys(message.tablet_map), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.topodata.Tablet.encode(message.tablet_map[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified SetShardTabletControlResponse message, length delimited. Does not implicitly {@link vtctldata.SetShardTabletControlResponse.verify|verify} messages. + * Encodes the specified ShardReplicationPositionsResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static - * @param {vtctldata.ISetShardTabletControlResponse} message SetShardTabletControlResponse message or plain object to encode + * @param {vtctldata.IShardReplicationPositionsResponse} message ShardReplicationPositionsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetShardTabletControlResponse.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationPositionsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetShardTabletControlResponse message from the specified reader or buffer. + * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse + * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardTabletControlResponse.decode = function decode(reader, length) { + ShardReplicationPositionsResponse.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.SetShardTabletControlResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationPositionsResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + if (message.replication_statuses === $util.emptyObject) + message.replication_statuses = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.replicationdata.Status.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.replication_statuses[key] = value; + break; + } + case 2: { + if (message.tablet_map === $util.emptyObject) + message.tablet_map = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.topodata.Tablet.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.tablet_map[key] = value; break; } default: @@ -133606,128 +140145,170 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetShardTabletControlResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse + * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetShardTabletControlResponse.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationPositionsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetShardTabletControlResponse message. + * Verifies a ShardReplicationPositionsResponse message. * @function verify - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetShardTabletControlResponse.verify = function verify(message) { + ShardReplicationPositionsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.topodata.Shard.verify(message.shard); - if (error) - return "shard." + error; + if (message.replication_statuses != null && message.hasOwnProperty("replication_statuses")) { + if (!$util.isObject(message.replication_statuses)) + return "replication_statuses: object expected"; + let key = Object.keys(message.replication_statuses); + for (let i = 0; i < key.length; ++i) { + let error = $root.replicationdata.Status.verify(message.replication_statuses[key[i]]); + if (error) + return "replication_statuses." + error; + } + } + if (message.tablet_map != null && message.hasOwnProperty("tablet_map")) { + if (!$util.isObject(message.tablet_map)) + return "tablet_map: object expected"; + let key = Object.keys(message.tablet_map); + for (let i = 0; i < key.length; ++i) { + let error = $root.topodata.Tablet.verify(message.tablet_map[key[i]]); + if (error) + return "tablet_map." + error; + } } return null; }; /** - * Creates a SetShardTabletControlResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationPositionsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetShardTabletControlResponse} SetShardTabletControlResponse + * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse */ - SetShardTabletControlResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetShardTabletControlResponse) + ShardReplicationPositionsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationPositionsResponse) return object; - let message = new $root.vtctldata.SetShardTabletControlResponse(); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.SetShardTabletControlResponse.shard: object expected"); - message.shard = $root.topodata.Shard.fromObject(object.shard); + let message = new $root.vtctldata.ShardReplicationPositionsResponse(); + if (object.replication_statuses) { + if (typeof object.replication_statuses !== "object") + throw TypeError(".vtctldata.ShardReplicationPositionsResponse.replication_statuses: object expected"); + message.replication_statuses = {}; + for (let keys = Object.keys(object.replication_statuses), i = 0; i < keys.length; ++i) { + if (typeof object.replication_statuses[keys[i]] !== "object") + throw TypeError(".vtctldata.ShardReplicationPositionsResponse.replication_statuses: object expected"); + message.replication_statuses[keys[i]] = $root.replicationdata.Status.fromObject(object.replication_statuses[keys[i]]); + } + } + if (object.tablet_map) { + if (typeof object.tablet_map !== "object") + throw TypeError(".vtctldata.ShardReplicationPositionsResponse.tablet_map: object expected"); + message.tablet_map = {}; + for (let keys = Object.keys(object.tablet_map), i = 0; i < keys.length; ++i) { + if (typeof object.tablet_map[keys[i]] !== "object") + throw TypeError(".vtctldata.ShardReplicationPositionsResponse.tablet_map: object expected"); + message.tablet_map[keys[i]] = $root.topodata.Tablet.fromObject(object.tablet_map[keys[i]]); + } } return message; }; /** - * Creates a plain object from a SetShardTabletControlResponse message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationPositionsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static - * @param {vtctldata.SetShardTabletControlResponse} message SetShardTabletControlResponse + * @param {vtctldata.ShardReplicationPositionsResponse} message ShardReplicationPositionsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetShardTabletControlResponse.toObject = function toObject(message, options) { + ShardReplicationPositionsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.shard = null; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.topodata.Shard.toObject(message.shard, options); + if (options.objects || options.defaults) { + object.replication_statuses = {}; + object.tablet_map = {}; + } + let keys2; + if (message.replication_statuses && (keys2 = Object.keys(message.replication_statuses)).length) { + object.replication_statuses = {}; + for (let j = 0; j < keys2.length; ++j) + object.replication_statuses[keys2[j]] = $root.replicationdata.Status.toObject(message.replication_statuses[keys2[j]], options); + } + if (message.tablet_map && (keys2 = Object.keys(message.tablet_map)).length) { + object.tablet_map = {}; + for (let j = 0; j < keys2.length; ++j) + object.tablet_map[keys2[j]] = $root.topodata.Tablet.toObject(message.tablet_map[keys2[j]], options); + } return object; }; /** - * Converts this SetShardTabletControlResponse to JSON. + * Converts this ShardReplicationPositionsResponse to JSON. * @function toJSON - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @instance * @returns {Object.} JSON object */ - SetShardTabletControlResponse.prototype.toJSON = function toJSON() { + ShardReplicationPositionsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetShardTabletControlResponse + * Gets the default type url for ShardReplicationPositionsResponse * @function getTypeUrl - * @memberof vtctldata.SetShardTabletControlResponse + * @memberof vtctldata.ShardReplicationPositionsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetShardTabletControlResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationPositionsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetShardTabletControlResponse"; + return typeUrlPrefix + "/vtctldata.ShardReplicationPositionsResponse"; }; - return SetShardTabletControlResponse; + return ShardReplicationPositionsResponse; })(); - vtctldata.SetWritableRequest = (function() { + vtctldata.ShardReplicationRemoveRequest = (function() { /** - * Properties of a SetWritableRequest. + * Properties of a ShardReplicationRemoveRequest. * @memberof vtctldata - * @interface ISetWritableRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] SetWritableRequest tablet_alias - * @property {boolean|null} [writable] SetWritableRequest writable + * @interface IShardReplicationRemoveRequest + * @property {string|null} [keyspace] ShardReplicationRemoveRequest keyspace + * @property {string|null} [shard] ShardReplicationRemoveRequest shard + * @property {topodata.ITabletAlias|null} [tablet_alias] ShardReplicationRemoveRequest tablet_alias */ /** - * Constructs a new SetWritableRequest. + * Constructs a new ShardReplicationRemoveRequest. * @memberof vtctldata - * @classdesc Represents a SetWritableRequest. - * @implements ISetWritableRequest + * @classdesc Represents a ShardReplicationRemoveRequest. + * @implements IShardReplicationRemoveRequest * @constructor - * @param {vtctldata.ISetWritableRequest=} [properties] Properties to set + * @param {vtctldata.IShardReplicationRemoveRequest=} [properties] Properties to set */ - function SetWritableRequest(properties) { + function ShardReplicationRemoveRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -133735,89 +140316,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SetWritableRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.SetWritableRequest + * ShardReplicationRemoveRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ShardReplicationRemoveRequest * @instance */ - SetWritableRequest.prototype.tablet_alias = null; + ShardReplicationRemoveRequest.prototype.keyspace = ""; /** - * SetWritableRequest writable. - * @member {boolean} writable - * @memberof vtctldata.SetWritableRequest + * ShardReplicationRemoveRequest shard. + * @member {string} shard + * @memberof vtctldata.ShardReplicationRemoveRequest * @instance */ - SetWritableRequest.prototype.writable = false; + ShardReplicationRemoveRequest.prototype.shard = ""; /** - * Creates a new SetWritableRequest instance using the specified properties. + * ShardReplicationRemoveRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.ShardReplicationRemoveRequest + * @instance + */ + ShardReplicationRemoveRequest.prototype.tablet_alias = null; + + /** + * Creates a new ShardReplicationRemoveRequest instance using the specified properties. * @function create - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static - * @param {vtctldata.ISetWritableRequest=} [properties] Properties to set - * @returns {vtctldata.SetWritableRequest} SetWritableRequest instance + * @param {vtctldata.IShardReplicationRemoveRequest=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest instance */ - SetWritableRequest.create = function create(properties) { - return new SetWritableRequest(properties); + ShardReplicationRemoveRequest.create = function create(properties) { + return new ShardReplicationRemoveRequest(properties); }; /** - * Encodes the specified SetWritableRequest message. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. + * Encodes the specified ShardReplicationRemoveRequest message. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static - * @param {vtctldata.ISetWritableRequest} message SetWritableRequest message or plain object to encode + * @param {vtctldata.IShardReplicationRemoveRequest} message ShardReplicationRemoveRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetWritableRequest.encode = function encode(message, writer) { + ShardReplicationRemoveRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.writable != null && Object.hasOwnProperty.call(message, "writable")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.writable); + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified SetWritableRequest message, length delimited. Does not implicitly {@link vtctldata.SetWritableRequest.verify|verify} messages. + * Encodes the specified ShardReplicationRemoveRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static - * @param {vtctldata.ISetWritableRequest} message SetWritableRequest message or plain object to encode + * @param {vtctldata.IShardReplicationRemoveRequest} message ShardReplicationRemoveRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetWritableRequest.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationRemoveRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetWritableRequest message from the specified reader or buffer. + * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetWritableRequest} SetWritableRequest + * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetWritableRequest.decode = function decode(reader, length) { + ShardReplicationRemoveRequest.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.SetWritableRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationRemoveRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); break; } case 2: { - message.writable = reader.bool(); + message.shard = reader.string(); + break; + } + case 3: { + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -133829,135 +140424,143 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetWritableRequest message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetWritableRequest} SetWritableRequest + * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetWritableRequest.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationRemoveRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetWritableRequest message. + * Verifies a ShardReplicationRemoveRequest message. * @function verify - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetWritableRequest.verify = function verify(message) { + ShardReplicationRemoveRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) return "tablet_alias." + error; } - if (message.writable != null && message.hasOwnProperty("writable")) - if (typeof message.writable !== "boolean") - return "writable: boolean expected"; return null; }; /** - * Creates a SetWritableRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationRemoveRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetWritableRequest} SetWritableRequest + * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest */ - SetWritableRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetWritableRequest) + ShardReplicationRemoveRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationRemoveRequest) return object; - let message = new $root.vtctldata.SetWritableRequest(); + let message = new $root.vtctldata.ShardReplicationRemoveRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.SetWritableRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.ShardReplicationRemoveRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.writable != null) - message.writable = Boolean(object.writable); return message; }; /** - * Creates a plain object from a SetWritableRequest message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationRemoveRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static - * @param {vtctldata.SetWritableRequest} message SetWritableRequest + * @param {vtctldata.ShardReplicationRemoveRequest} message ShardReplicationRemoveRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetWritableRequest.toObject = function toObject(message, options) { + ShardReplicationRemoveRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { + object.keyspace = ""; + object.shard = ""; object.tablet_alias = null; - object.writable = false; } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.writable != null && message.hasOwnProperty("writable")) - object.writable = message.writable; return object; }; /** - * Converts this SetWritableRequest to JSON. + * Converts this ShardReplicationRemoveRequest to JSON. * @function toJSON - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @instance * @returns {Object.} JSON object */ - SetWritableRequest.prototype.toJSON = function toJSON() { + ShardReplicationRemoveRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetWritableRequest + * Gets the default type url for ShardReplicationRemoveRequest * @function getTypeUrl - * @memberof vtctldata.SetWritableRequest + * @memberof vtctldata.ShardReplicationRemoveRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetWritableRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationRemoveRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetWritableRequest"; + return typeUrlPrefix + "/vtctldata.ShardReplicationRemoveRequest"; }; - return SetWritableRequest; + return ShardReplicationRemoveRequest; })(); - vtctldata.SetWritableResponse = (function() { + vtctldata.ShardReplicationRemoveResponse = (function() { /** - * Properties of a SetWritableResponse. + * Properties of a ShardReplicationRemoveResponse. * @memberof vtctldata - * @interface ISetWritableResponse + * @interface IShardReplicationRemoveResponse */ /** - * Constructs a new SetWritableResponse. + * Constructs a new ShardReplicationRemoveResponse. * @memberof vtctldata - * @classdesc Represents a SetWritableResponse. - * @implements ISetWritableResponse + * @classdesc Represents a ShardReplicationRemoveResponse. + * @implements IShardReplicationRemoveResponse * @constructor - * @param {vtctldata.ISetWritableResponse=} [properties] Properties to set + * @param {vtctldata.IShardReplicationRemoveResponse=} [properties] Properties to set */ - function SetWritableResponse(properties) { + function ShardReplicationRemoveResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -133965,60 +140568,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new SetWritableResponse instance using the specified properties. + * Creates a new ShardReplicationRemoveResponse instance using the specified properties. * @function create - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static - * @param {vtctldata.ISetWritableResponse=} [properties] Properties to set - * @returns {vtctldata.SetWritableResponse} SetWritableResponse instance + * @param {vtctldata.IShardReplicationRemoveResponse=} [properties] Properties to set + * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse instance */ - SetWritableResponse.create = function create(properties) { - return new SetWritableResponse(properties); + ShardReplicationRemoveResponse.create = function create(properties) { + return new ShardReplicationRemoveResponse(properties); }; /** - * Encodes the specified SetWritableResponse message. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. + * Encodes the specified ShardReplicationRemoveResponse message. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static - * @param {vtctldata.ISetWritableResponse} message SetWritableResponse message or plain object to encode + * @param {vtctldata.IShardReplicationRemoveResponse} message ShardReplicationRemoveResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetWritableResponse.encode = function encode(message, writer) { + ShardReplicationRemoveResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified SetWritableResponse message, length delimited. Does not implicitly {@link vtctldata.SetWritableResponse.verify|verify} messages. + * Encodes the specified ShardReplicationRemoveResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static - * @param {vtctldata.ISetWritableResponse} message SetWritableResponse message or plain object to encode + * @param {vtctldata.IShardReplicationRemoveResponse} message ShardReplicationRemoveResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SetWritableResponse.encodeDelimited = function encodeDelimited(message, writer) { + ShardReplicationRemoveResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SetWritableResponse message from the specified reader or buffer. + * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SetWritableResponse} SetWritableResponse + * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetWritableResponse.decode = function decode(reader, length) { + ShardReplicationRemoveResponse.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.SetWritableResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ShardReplicationRemoveResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -134031,111 +140634,110 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SetWritableResponse message from the specified reader or buffer, length delimited. + * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SetWritableResponse} SetWritableResponse + * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SetWritableResponse.decodeDelimited = function decodeDelimited(reader) { + ShardReplicationRemoveResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SetWritableResponse message. + * Verifies a ShardReplicationRemoveResponse message. * @function verify - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SetWritableResponse.verify = function verify(message) { + ShardReplicationRemoveResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a SetWritableResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ShardReplicationRemoveResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SetWritableResponse} SetWritableResponse + * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse */ - SetWritableResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SetWritableResponse) + ShardReplicationRemoveResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ShardReplicationRemoveResponse) return object; - return new $root.vtctldata.SetWritableResponse(); + return new $root.vtctldata.ShardReplicationRemoveResponse(); }; /** - * Creates a plain object from a SetWritableResponse message. Also converts values to other types if specified. + * Creates a plain object from a ShardReplicationRemoveResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static - * @param {vtctldata.SetWritableResponse} message SetWritableResponse + * @param {vtctldata.ShardReplicationRemoveResponse} message ShardReplicationRemoveResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SetWritableResponse.toObject = function toObject() { + ShardReplicationRemoveResponse.toObject = function toObject() { return {}; }; /** - * Converts this SetWritableResponse to JSON. + * Converts this ShardReplicationRemoveResponse to JSON. * @function toJSON - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @instance * @returns {Object.} JSON object */ - SetWritableResponse.prototype.toJSON = function toJSON() { + ShardReplicationRemoveResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SetWritableResponse + * Gets the default type url for ShardReplicationRemoveResponse * @function getTypeUrl - * @memberof vtctldata.SetWritableResponse + * @memberof vtctldata.ShardReplicationRemoveResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SetWritableResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ShardReplicationRemoveResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SetWritableResponse"; + return typeUrlPrefix + "/vtctldata.ShardReplicationRemoveResponse"; }; - return SetWritableResponse; + return ShardReplicationRemoveResponse; })(); - vtctldata.ShardReplicationAddRequest = (function() { + vtctldata.SleepTabletRequest = (function() { /** - * Properties of a ShardReplicationAddRequest. + * Properties of a SleepTabletRequest. * @memberof vtctldata - * @interface IShardReplicationAddRequest - * @property {string|null} [keyspace] ShardReplicationAddRequest keyspace - * @property {string|null} [shard] ShardReplicationAddRequest shard - * @property {topodata.ITabletAlias|null} [tablet_alias] ShardReplicationAddRequest tablet_alias + * @interface ISleepTabletRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] SleepTabletRequest tablet_alias + * @property {vttime.IDuration|null} [duration] SleepTabletRequest duration */ /** - * Constructs a new ShardReplicationAddRequest. + * Constructs a new SleepTabletRequest. * @memberof vtctldata - * @classdesc Represents a ShardReplicationAddRequest. - * @implements IShardReplicationAddRequest + * @classdesc Represents a SleepTabletRequest. + * @implements ISleepTabletRequest * @constructor - * @param {vtctldata.IShardReplicationAddRequest=} [properties] Properties to set + * @param {vtctldata.ISleepTabletRequest=} [properties] Properties to set */ - function ShardReplicationAddRequest(properties) { + function SleepTabletRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -134143,103 +140745,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationAddRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ShardReplicationAddRequest - * @instance - */ - ShardReplicationAddRequest.prototype.keyspace = ""; - - /** - * ShardReplicationAddRequest shard. - * @member {string} shard - * @memberof vtctldata.ShardReplicationAddRequest + * SleepTabletRequest tablet_alias. + * @member {topodata.ITabletAlias|null|undefined} tablet_alias + * @memberof vtctldata.SleepTabletRequest * @instance */ - ShardReplicationAddRequest.prototype.shard = ""; + SleepTabletRequest.prototype.tablet_alias = null; /** - * ShardReplicationAddRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ShardReplicationAddRequest + * SleepTabletRequest duration. + * @member {vttime.IDuration|null|undefined} duration + * @memberof vtctldata.SleepTabletRequest * @instance */ - ShardReplicationAddRequest.prototype.tablet_alias = null; + SleepTabletRequest.prototype.duration = null; /** - * Creates a new ShardReplicationAddRequest instance using the specified properties. + * Creates a new SleepTabletRequest instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static - * @param {vtctldata.IShardReplicationAddRequest=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest instance + * @param {vtctldata.ISleepTabletRequest=} [properties] Properties to set + * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest instance */ - ShardReplicationAddRequest.create = function create(properties) { - return new ShardReplicationAddRequest(properties); + SleepTabletRequest.create = function create(properties) { + return new SleepTabletRequest(properties); }; /** - * Encodes the specified ShardReplicationAddRequest message. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. + * Encodes the specified SleepTabletRequest message. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static - * @param {vtctldata.IShardReplicationAddRequest} message ShardReplicationAddRequest message or plain object to encode + * @param {vtctldata.ISleepTabletRequest} message SleepTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationAddRequest.encode = function encode(message, writer) { + SleepTabletRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.duration != null && Object.hasOwnProperty.call(message, "duration")) + $root.vttime.Duration.encode(message.duration, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified ShardReplicationAddRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddRequest.verify|verify} messages. + * Encodes the specified SleepTabletRequest message, length delimited. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static - * @param {vtctldata.IShardReplicationAddRequest} message ShardReplicationAddRequest message or plain object to encode + * @param {vtctldata.ISleepTabletRequest} message SleepTabletRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationAddRequest.encodeDelimited = function encodeDelimited(message, writer) { + SleepTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationAddRequest message from the specified reader or buffer. + * Decodes a SleepTabletRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest + * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationAddRequest.decode = function decode(reader, length) { + SleepTabletRequest.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.ShardReplicationAddRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SleepTabletRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.duration = $root.vttime.Duration.decode(reader, reader.uint32()); break; } default: @@ -134251,143 +140839,140 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationAddRequest message from the specified reader or buffer, length delimited. + * Decodes a SleepTabletRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest + * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationAddRequest.decodeDelimited = function decodeDelimited(reader) { + SleepTabletRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationAddRequest message. + * Verifies a SleepTabletRequest message. * @function verify - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationAddRequest.verify = function verify(message) { + SleepTabletRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) return "tablet_alias." + error; } + if (message.duration != null && message.hasOwnProperty("duration")) { + let error = $root.vttime.Duration.verify(message.duration); + if (error) + return "duration." + error; + } return null; }; /** - * Creates a ShardReplicationAddRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SleepTabletRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationAddRequest} ShardReplicationAddRequest + * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest */ - ShardReplicationAddRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationAddRequest) + SleepTabletRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SleepTabletRequest) return object; - let message = new $root.vtctldata.ShardReplicationAddRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); + let message = new $root.vtctldata.SleepTabletRequest(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ShardReplicationAddRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.SleepTabletRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } + if (object.duration != null) { + if (typeof object.duration !== "object") + throw TypeError(".vtctldata.SleepTabletRequest.duration: object expected"); + message.duration = $root.vttime.Duration.fromObject(object.duration); + } return message; }; /** - * Creates a plain object from a ShardReplicationAddRequest message. Also converts values to other types if specified. + * Creates a plain object from a SleepTabletRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static - * @param {vtctldata.ShardReplicationAddRequest} message ShardReplicationAddRequest + * @param {vtctldata.SleepTabletRequest} message SleepTabletRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationAddRequest.toObject = function toObject(message, options) { + SleepTabletRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; object.tablet_alias = null; + object.duration = null; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (message.duration != null && message.hasOwnProperty("duration")) + object.duration = $root.vttime.Duration.toObject(message.duration, options); return object; }; /** - * Converts this ShardReplicationAddRequest to JSON. + * Converts this SleepTabletRequest to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @instance * @returns {Object.} JSON object */ - ShardReplicationAddRequest.prototype.toJSON = function toJSON() { + SleepTabletRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationAddRequest + * Gets the default type url for SleepTabletRequest * @function getTypeUrl - * @memberof vtctldata.ShardReplicationAddRequest + * @memberof vtctldata.SleepTabletRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationAddRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SleepTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationAddRequest"; + return typeUrlPrefix + "/vtctldata.SleepTabletRequest"; }; - return ShardReplicationAddRequest; + return SleepTabletRequest; })(); - vtctldata.ShardReplicationAddResponse = (function() { + vtctldata.SleepTabletResponse = (function() { /** - * Properties of a ShardReplicationAddResponse. + * Properties of a SleepTabletResponse. * @memberof vtctldata - * @interface IShardReplicationAddResponse + * @interface ISleepTabletResponse */ /** - * Constructs a new ShardReplicationAddResponse. + * Constructs a new SleepTabletResponse. * @memberof vtctldata - * @classdesc Represents a ShardReplicationAddResponse. - * @implements IShardReplicationAddResponse + * @classdesc Represents a SleepTabletResponse. + * @implements ISleepTabletResponse * @constructor - * @param {vtctldata.IShardReplicationAddResponse=} [properties] Properties to set + * @param {vtctldata.ISleepTabletResponse=} [properties] Properties to set */ - function ShardReplicationAddResponse(properties) { + function SleepTabletResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -134395,60 +140980,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new ShardReplicationAddResponse instance using the specified properties. + * Creates a new SleepTabletResponse instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static - * @param {vtctldata.IShardReplicationAddResponse=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse instance + * @param {vtctldata.ISleepTabletResponse=} [properties] Properties to set + * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse instance */ - ShardReplicationAddResponse.create = function create(properties) { - return new ShardReplicationAddResponse(properties); + SleepTabletResponse.create = function create(properties) { + return new SleepTabletResponse(properties); }; /** - * Encodes the specified ShardReplicationAddResponse message. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. + * Encodes the specified SleepTabletResponse message. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static - * @param {vtctldata.IShardReplicationAddResponse} message ShardReplicationAddResponse message or plain object to encode + * @param {vtctldata.ISleepTabletResponse} message SleepTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationAddResponse.encode = function encode(message, writer) { + SleepTabletResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified ShardReplicationAddResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationAddResponse.verify|verify} messages. + * Encodes the specified SleepTabletResponse message, length delimited. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static - * @param {vtctldata.IShardReplicationAddResponse} message ShardReplicationAddResponse message or plain object to encode + * @param {vtctldata.ISleepTabletResponse} message SleepTabletResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationAddResponse.encodeDelimited = function encodeDelimited(message, writer) { + SleepTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationAddResponse message from the specified reader or buffer. + * Decodes a SleepTabletResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse + * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationAddResponse.decode = function decode(reader, length) { + SleepTabletResponse.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.ShardReplicationAddResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SleepTabletResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -134461,111 +141046,116 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationAddResponse message from the specified reader or buffer, length delimited. + * Decodes a SleepTabletResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse + * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationAddResponse.decodeDelimited = function decodeDelimited(reader) { + SleepTabletResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationAddResponse message. + * Verifies a SleepTabletResponse message. * @function verify - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationAddResponse.verify = function verify(message) { + SleepTabletResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a ShardReplicationAddResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SleepTabletResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationAddResponse} ShardReplicationAddResponse + * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse */ - ShardReplicationAddResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationAddResponse) + SleepTabletResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SleepTabletResponse) return object; - return new $root.vtctldata.ShardReplicationAddResponse(); + return new $root.vtctldata.SleepTabletResponse(); }; /** - * Creates a plain object from a ShardReplicationAddResponse message. Also converts values to other types if specified. + * Creates a plain object from a SleepTabletResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static - * @param {vtctldata.ShardReplicationAddResponse} message ShardReplicationAddResponse + * @param {vtctldata.SleepTabletResponse} message SleepTabletResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationAddResponse.toObject = function toObject() { + SleepTabletResponse.toObject = function toObject() { return {}; }; /** - * Converts this ShardReplicationAddResponse to JSON. + * Converts this SleepTabletResponse to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @instance * @returns {Object.} JSON object */ - ShardReplicationAddResponse.prototype.toJSON = function toJSON() { + SleepTabletResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationAddResponse + * Gets the default type url for SleepTabletResponse * @function getTypeUrl - * @memberof vtctldata.ShardReplicationAddResponse + * @memberof vtctldata.SleepTabletResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationAddResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SleepTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationAddResponse"; + return typeUrlPrefix + "/vtctldata.SleepTabletResponse"; }; - return ShardReplicationAddResponse; + return SleepTabletResponse; })(); - vtctldata.ShardReplicationFixRequest = (function() { + vtctldata.SourceShardAddRequest = (function() { /** - * Properties of a ShardReplicationFixRequest. + * Properties of a SourceShardAddRequest. * @memberof vtctldata - * @interface IShardReplicationFixRequest - * @property {string|null} [keyspace] ShardReplicationFixRequest keyspace - * @property {string|null} [shard] ShardReplicationFixRequest shard - * @property {string|null} [cell] ShardReplicationFixRequest cell + * @interface ISourceShardAddRequest + * @property {string|null} [keyspace] SourceShardAddRequest keyspace + * @property {string|null} [shard] SourceShardAddRequest shard + * @property {number|null} [uid] SourceShardAddRequest uid + * @property {string|null} [source_keyspace] SourceShardAddRequest source_keyspace + * @property {string|null} [source_shard] SourceShardAddRequest source_shard + * @property {topodata.IKeyRange|null} [key_range] SourceShardAddRequest key_range + * @property {Array.|null} [tables] SourceShardAddRequest tables */ /** - * Constructs a new ShardReplicationFixRequest. + * Constructs a new SourceShardAddRequest. * @memberof vtctldata - * @classdesc Represents a ShardReplicationFixRequest. - * @implements IShardReplicationFixRequest + * @classdesc Represents a SourceShardAddRequest. + * @implements ISourceShardAddRequest * @constructor - * @param {vtctldata.IShardReplicationFixRequest=} [properties] Properties to set + * @param {vtctldata.ISourceShardAddRequest=} [properties] Properties to set */ - function ShardReplicationFixRequest(properties) { + function SourceShardAddRequest(properties) { + this.tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -134573,90 +141163,131 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationFixRequest keyspace. + * SourceShardAddRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @instance */ - ShardReplicationFixRequest.prototype.keyspace = ""; + SourceShardAddRequest.prototype.keyspace = ""; /** - * ShardReplicationFixRequest shard. + * SourceShardAddRequest shard. * @member {string} shard - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @instance */ - ShardReplicationFixRequest.prototype.shard = ""; + SourceShardAddRequest.prototype.shard = ""; /** - * ShardReplicationFixRequest cell. - * @member {string} cell - * @memberof vtctldata.ShardReplicationFixRequest + * SourceShardAddRequest uid. + * @member {number} uid + * @memberof vtctldata.SourceShardAddRequest * @instance */ - ShardReplicationFixRequest.prototype.cell = ""; + SourceShardAddRequest.prototype.uid = 0; /** - * Creates a new ShardReplicationFixRequest instance using the specified properties. + * SourceShardAddRequest source_keyspace. + * @member {string} source_keyspace + * @memberof vtctldata.SourceShardAddRequest + * @instance + */ + SourceShardAddRequest.prototype.source_keyspace = ""; + + /** + * SourceShardAddRequest source_shard. + * @member {string} source_shard + * @memberof vtctldata.SourceShardAddRequest + * @instance + */ + SourceShardAddRequest.prototype.source_shard = ""; + + /** + * SourceShardAddRequest key_range. + * @member {topodata.IKeyRange|null|undefined} key_range + * @memberof vtctldata.SourceShardAddRequest + * @instance + */ + SourceShardAddRequest.prototype.key_range = null; + + /** + * SourceShardAddRequest tables. + * @member {Array.} tables + * @memberof vtctldata.SourceShardAddRequest + * @instance + */ + SourceShardAddRequest.prototype.tables = $util.emptyArray; + + /** + * Creates a new SourceShardAddRequest instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static - * @param {vtctldata.IShardReplicationFixRequest=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest instance + * @param {vtctldata.ISourceShardAddRequest=} [properties] Properties to set + * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest instance */ - ShardReplicationFixRequest.create = function create(properties) { - return new ShardReplicationFixRequest(properties); + SourceShardAddRequest.create = function create(properties) { + return new SourceShardAddRequest(properties); }; /** - * Encodes the specified ShardReplicationFixRequest message. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. + * Encodes the specified SourceShardAddRequest message. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static - * @param {vtctldata.IShardReplicationFixRequest} message ShardReplicationFixRequest message or plain object to encode + * @param {vtctldata.ISourceShardAddRequest} message SourceShardAddRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationFixRequest.encode = function encode(message, writer) { + SourceShardAddRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.cell != null && Object.hasOwnProperty.call(message, "cell")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cell); + if (message.uid != null && Object.hasOwnProperty.call(message, "uid")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.uid); + if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.source_keyspace); + if (message.source_shard != null && Object.hasOwnProperty.call(message, "source_shard")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_shard); + if (message.key_range != null && Object.hasOwnProperty.call(message, "key_range")) + $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.tables != null && message.tables.length) + for (let i = 0; i < message.tables.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.tables[i]); return writer; }; /** - * Encodes the specified ShardReplicationFixRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixRequest.verify|verify} messages. + * Encodes the specified SourceShardAddRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static - * @param {vtctldata.IShardReplicationFixRequest} message ShardReplicationFixRequest message or plain object to encode + * @param {vtctldata.ISourceShardAddRequest} message SourceShardAddRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationFixRequest.encodeDelimited = function encodeDelimited(message, writer) { + SourceShardAddRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationFixRequest message from the specified reader or buffer. + * Decodes a SourceShardAddRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest + * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationFixRequest.decode = function decode(reader, length) { + SourceShardAddRequest.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.ShardReplicationFixRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SourceShardAddRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -134669,7 +141300,25 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 3: { - message.cell = reader.string(); + message.uid = reader.int32(); + break; + } + case 4: { + message.source_keyspace = reader.string(); + break; + } + case 5: { + message.source_shard = reader.string(); + break; + } + case 6: { + message.key_range = $root.topodata.KeyRange.decode(reader, reader.uint32()); + break; + } + case 7: { + if (!(message.tables && message.tables.length)) + message.tables = []; + message.tables.push(reader.string()); break; } default: @@ -134681,30 +141330,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationFixRequest message from the specified reader or buffer, length delimited. + * Decodes a SourceShardAddRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest + * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationFixRequest.decodeDelimited = function decodeDelimited(reader) { + SourceShardAddRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationFixRequest message. + * Verifies a SourceShardAddRequest message. * @function verify - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationFixRequest.verify = function verify(message) { + SourceShardAddRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -134713,107 +141362,157 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; - if (message.cell != null && message.hasOwnProperty("cell")) - if (!$util.isString(message.cell)) - return "cell: string expected"; + if (message.uid != null && message.hasOwnProperty("uid")) + if (!$util.isInteger(message.uid)) + return "uid: integer expected"; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + if (!$util.isString(message.source_keyspace)) + return "source_keyspace: string expected"; + if (message.source_shard != null && message.hasOwnProperty("source_shard")) + if (!$util.isString(message.source_shard)) + return "source_shard: string expected"; + if (message.key_range != null && message.hasOwnProperty("key_range")) { + let error = $root.topodata.KeyRange.verify(message.key_range); + if (error) + return "key_range." + error; + } + if (message.tables != null && message.hasOwnProperty("tables")) { + if (!Array.isArray(message.tables)) + return "tables: array expected"; + for (let i = 0; i < message.tables.length; ++i) + if (!$util.isString(message.tables[i])) + return "tables: string[] expected"; + } return null; }; /** - * Creates a ShardReplicationFixRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardAddRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationFixRequest} ShardReplicationFixRequest + * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest */ - ShardReplicationFixRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationFixRequest) + SourceShardAddRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SourceShardAddRequest) return object; - let message = new $root.vtctldata.ShardReplicationFixRequest(); + let message = new $root.vtctldata.SourceShardAddRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); - if (object.cell != null) - message.cell = String(object.cell); + if (object.uid != null) + message.uid = object.uid | 0; + if (object.source_keyspace != null) + message.source_keyspace = String(object.source_keyspace); + if (object.source_shard != null) + message.source_shard = String(object.source_shard); + if (object.key_range != null) { + if (typeof object.key_range !== "object") + throw TypeError(".vtctldata.SourceShardAddRequest.key_range: object expected"); + message.key_range = $root.topodata.KeyRange.fromObject(object.key_range); + } + if (object.tables) { + if (!Array.isArray(object.tables)) + throw TypeError(".vtctldata.SourceShardAddRequest.tables: array expected"); + message.tables = []; + for (let i = 0; i < object.tables.length; ++i) + message.tables[i] = String(object.tables[i]); + } return message; }; /** - * Creates a plain object from a ShardReplicationFixRequest message. Also converts values to other types if specified. + * Creates a plain object from a SourceShardAddRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static - * @param {vtctldata.ShardReplicationFixRequest} message ShardReplicationFixRequest + * @param {vtctldata.SourceShardAddRequest} message SourceShardAddRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationFixRequest.toObject = function toObject(message, options) { + SourceShardAddRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.tables = []; if (options.defaults) { object.keyspace = ""; object.shard = ""; - object.cell = ""; + object.uid = 0; + object.source_keyspace = ""; + object.source_shard = ""; + object.key_range = null; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; - if (message.cell != null && message.hasOwnProperty("cell")) - object.cell = message.cell; + if (message.uid != null && message.hasOwnProperty("uid")) + object.uid = message.uid; + if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) + object.source_keyspace = message.source_keyspace; + if (message.source_shard != null && message.hasOwnProperty("source_shard")) + object.source_shard = message.source_shard; + if (message.key_range != null && message.hasOwnProperty("key_range")) + object.key_range = $root.topodata.KeyRange.toObject(message.key_range, options); + if (message.tables && message.tables.length) { + object.tables = []; + for (let j = 0; j < message.tables.length; ++j) + object.tables[j] = message.tables[j]; + } return object; }; /** - * Converts this ShardReplicationFixRequest to JSON. + * Converts this SourceShardAddRequest to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @instance * @returns {Object.} JSON object */ - ShardReplicationFixRequest.prototype.toJSON = function toJSON() { + SourceShardAddRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationFixRequest + * Gets the default type url for SourceShardAddRequest * @function getTypeUrl - * @memberof vtctldata.ShardReplicationFixRequest + * @memberof vtctldata.SourceShardAddRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationFixRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SourceShardAddRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationFixRequest"; + return typeUrlPrefix + "/vtctldata.SourceShardAddRequest"; }; - return ShardReplicationFixRequest; + return SourceShardAddRequest; })(); - vtctldata.ShardReplicationFixResponse = (function() { + vtctldata.SourceShardAddResponse = (function() { /** - * Properties of a ShardReplicationFixResponse. + * Properties of a SourceShardAddResponse. * @memberof vtctldata - * @interface IShardReplicationFixResponse - * @property {topodata.IShardReplicationError|null} [error] ShardReplicationFixResponse error + * @interface ISourceShardAddResponse + * @property {topodata.IShard|null} [shard] SourceShardAddResponse shard */ /** - * Constructs a new ShardReplicationFixResponse. + * Constructs a new SourceShardAddResponse. * @memberof vtctldata - * @classdesc Represents a ShardReplicationFixResponse. - * @implements IShardReplicationFixResponse + * @classdesc Represents a SourceShardAddResponse. + * @implements ISourceShardAddResponse * @constructor - * @param {vtctldata.IShardReplicationFixResponse=} [properties] Properties to set + * @param {vtctldata.ISourceShardAddResponse=} [properties] Properties to set */ - function ShardReplicationFixResponse(properties) { + function SourceShardAddResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -134821,75 +141520,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationFixResponse error. - * @member {topodata.IShardReplicationError|null|undefined} error - * @memberof vtctldata.ShardReplicationFixResponse + * SourceShardAddResponse shard. + * @member {topodata.IShard|null|undefined} shard + * @memberof vtctldata.SourceShardAddResponse * @instance */ - ShardReplicationFixResponse.prototype.error = null; + SourceShardAddResponse.prototype.shard = null; /** - * Creates a new ShardReplicationFixResponse instance using the specified properties. + * Creates a new SourceShardAddResponse instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static - * @param {vtctldata.IShardReplicationFixResponse=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse instance + * @param {vtctldata.ISourceShardAddResponse=} [properties] Properties to set + * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse instance */ - ShardReplicationFixResponse.create = function create(properties) { - return new ShardReplicationFixResponse(properties); + SourceShardAddResponse.create = function create(properties) { + return new SourceShardAddResponse(properties); }; /** - * Encodes the specified ShardReplicationFixResponse message. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. + * Encodes the specified SourceShardAddResponse message. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static - * @param {vtctldata.IShardReplicationFixResponse} message ShardReplicationFixResponse message or plain object to encode + * @param {vtctldata.ISourceShardAddResponse} message SourceShardAddResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationFixResponse.encode = function encode(message, writer) { + SourceShardAddResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.error != null && Object.hasOwnProperty.call(message, "error")) - $root.topodata.ShardReplicationError.encode(message.error, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ShardReplicationFixResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationFixResponse.verify|verify} messages. + * Encodes the specified SourceShardAddResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static - * @param {vtctldata.IShardReplicationFixResponse} message ShardReplicationFixResponse message or plain object to encode + * @param {vtctldata.ISourceShardAddResponse} message SourceShardAddResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationFixResponse.encodeDelimited = function encodeDelimited(message, writer) { + SourceShardAddResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationFixResponse message from the specified reader or buffer. + * Decodes a SourceShardAddResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse + * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationFixResponse.decode = function decode(reader, length) { + SourceShardAddResponse.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.ShardReplicationFixResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SourceShardAddResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.error = $root.topodata.ShardReplicationError.decode(reader, reader.uint32()); + message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); break; } default: @@ -134901,128 +141600,129 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationFixResponse message from the specified reader or buffer, length delimited. + * Decodes a SourceShardAddResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse + * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationFixResponse.decodeDelimited = function decodeDelimited(reader) { + SourceShardAddResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationFixResponse message. + * Verifies a SourceShardAddResponse message. * @function verify - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationFixResponse.verify = function verify(message) { + SourceShardAddResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.error != null && message.hasOwnProperty("error")) { - let error = $root.topodata.ShardReplicationError.verify(message.error); + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.topodata.Shard.verify(message.shard); if (error) - return "error." + error; + return "shard." + error; } return null; }; /** - * Creates a ShardReplicationFixResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardAddResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationFixResponse} ShardReplicationFixResponse + * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse */ - ShardReplicationFixResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationFixResponse) + SourceShardAddResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SourceShardAddResponse) return object; - let message = new $root.vtctldata.ShardReplicationFixResponse(); - if (object.error != null) { - if (typeof object.error !== "object") - throw TypeError(".vtctldata.ShardReplicationFixResponse.error: object expected"); - message.error = $root.topodata.ShardReplicationError.fromObject(object.error); + let message = new $root.vtctldata.SourceShardAddResponse(); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.SourceShardAddResponse.shard: object expected"); + message.shard = $root.topodata.Shard.fromObject(object.shard); } return message; }; /** - * Creates a plain object from a ShardReplicationFixResponse message. Also converts values to other types if specified. + * Creates a plain object from a SourceShardAddResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static - * @param {vtctldata.ShardReplicationFixResponse} message ShardReplicationFixResponse + * @param {vtctldata.SourceShardAddResponse} message SourceShardAddResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationFixResponse.toObject = function toObject(message, options) { + SourceShardAddResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.error = null; - if (message.error != null && message.hasOwnProperty("error")) - object.error = $root.topodata.ShardReplicationError.toObject(message.error, options); + object.shard = null; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.topodata.Shard.toObject(message.shard, options); return object; }; /** - * Converts this ShardReplicationFixResponse to JSON. + * Converts this SourceShardAddResponse to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @instance * @returns {Object.} JSON object */ - ShardReplicationFixResponse.prototype.toJSON = function toJSON() { + SourceShardAddResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationFixResponse + * Gets the default type url for SourceShardAddResponse * @function getTypeUrl - * @memberof vtctldata.ShardReplicationFixResponse + * @memberof vtctldata.SourceShardAddResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationFixResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SourceShardAddResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationFixResponse"; + return typeUrlPrefix + "/vtctldata.SourceShardAddResponse"; }; - return ShardReplicationFixResponse; + return SourceShardAddResponse; })(); - vtctldata.ShardReplicationPositionsRequest = (function() { + vtctldata.SourceShardDeleteRequest = (function() { /** - * Properties of a ShardReplicationPositionsRequest. + * Properties of a SourceShardDeleteRequest. * @memberof vtctldata - * @interface IShardReplicationPositionsRequest - * @property {string|null} [keyspace] ShardReplicationPositionsRequest keyspace - * @property {string|null} [shard] ShardReplicationPositionsRequest shard + * @interface ISourceShardDeleteRequest + * @property {string|null} [keyspace] SourceShardDeleteRequest keyspace + * @property {string|null} [shard] SourceShardDeleteRequest shard + * @property {number|null} [uid] SourceShardDeleteRequest uid */ /** - * Constructs a new ShardReplicationPositionsRequest. + * Constructs a new SourceShardDeleteRequest. * @memberof vtctldata - * @classdesc Represents a ShardReplicationPositionsRequest. - * @implements IShardReplicationPositionsRequest + * @classdesc Represents a SourceShardDeleteRequest. + * @implements ISourceShardDeleteRequest * @constructor - * @param {vtctldata.IShardReplicationPositionsRequest=} [properties] Properties to set + * @param {vtctldata.ISourceShardDeleteRequest=} [properties] Properties to set */ - function ShardReplicationPositionsRequest(properties) { + function SourceShardDeleteRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -135030,80 +141730,90 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationPositionsRequest keyspace. + * SourceShardDeleteRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @instance */ - ShardReplicationPositionsRequest.prototype.keyspace = ""; + SourceShardDeleteRequest.prototype.keyspace = ""; /** - * ShardReplicationPositionsRequest shard. + * SourceShardDeleteRequest shard. * @member {string} shard - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @instance */ - ShardReplicationPositionsRequest.prototype.shard = ""; + SourceShardDeleteRequest.prototype.shard = ""; /** - * Creates a new ShardReplicationPositionsRequest instance using the specified properties. + * SourceShardDeleteRequest uid. + * @member {number} uid + * @memberof vtctldata.SourceShardDeleteRequest + * @instance + */ + SourceShardDeleteRequest.prototype.uid = 0; + + /** + * Creates a new SourceShardDeleteRequest instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static - * @param {vtctldata.IShardReplicationPositionsRequest=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest instance + * @param {vtctldata.ISourceShardDeleteRequest=} [properties] Properties to set + * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest instance */ - ShardReplicationPositionsRequest.create = function create(properties) { - return new ShardReplicationPositionsRequest(properties); + SourceShardDeleteRequest.create = function create(properties) { + return new SourceShardDeleteRequest(properties); }; /** - * Encodes the specified ShardReplicationPositionsRequest message. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. + * Encodes the specified SourceShardDeleteRequest message. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static - * @param {vtctldata.IShardReplicationPositionsRequest} message ShardReplicationPositionsRequest message or plain object to encode + * @param {vtctldata.ISourceShardDeleteRequest} message SourceShardDeleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationPositionsRequest.encode = function encode(message, writer) { + SourceShardDeleteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.uid != null && Object.hasOwnProperty.call(message, "uid")) + writer.uint32(/* id 3, wireType 0 =*/24).int32(message.uid); return writer; }; /** - * Encodes the specified ShardReplicationPositionsRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsRequest.verify|verify} messages. + * Encodes the specified SourceShardDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static - * @param {vtctldata.IShardReplicationPositionsRequest} message ShardReplicationPositionsRequest message or plain object to encode + * @param {vtctldata.ISourceShardDeleteRequest} message SourceShardDeleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationPositionsRequest.encodeDelimited = function encodeDelimited(message, writer) { + SourceShardDeleteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer. + * Decodes a SourceShardDeleteRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest + * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationPositionsRequest.decode = function decode(reader, length) { + SourceShardDeleteRequest.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.ShardReplicationPositionsRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SourceShardDeleteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -135115,6 +141825,10 @@ export const vtctldata = $root.vtctldata = (() => { message.shard = reader.string(); break; } + case 3: { + message.uid = reader.int32(); + break; + } default: reader.skipType(tag & 7); break; @@ -135124,30 +141838,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationPositionsRequest message from the specified reader or buffer, length delimited. + * Decodes a SourceShardDeleteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest + * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationPositionsRequest.decodeDelimited = function decodeDelimited(reader) { + SourceShardDeleteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationPositionsRequest message. + * Verifies a SourceShardDeleteRequest message. * @function verify - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationPositionsRequest.verify = function verify(message) { + SourceShardDeleteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) @@ -135156,102 +141870,107 @@ export const vtctldata = $root.vtctldata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; + if (message.uid != null && message.hasOwnProperty("uid")) + if (!$util.isInteger(message.uid)) + return "uid: integer expected"; return null; }; /** - * Creates a ShardReplicationPositionsRequest message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardDeleteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationPositionsRequest} ShardReplicationPositionsRequest + * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest */ - ShardReplicationPositionsRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationPositionsRequest) + SourceShardDeleteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SourceShardDeleteRequest) return object; - let message = new $root.vtctldata.ShardReplicationPositionsRequest(); + let message = new $root.vtctldata.SourceShardDeleteRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); + if (object.uid != null) + message.uid = object.uid | 0; return message; }; /** - * Creates a plain object from a ShardReplicationPositionsRequest message. Also converts values to other types if specified. + * Creates a plain object from a SourceShardDeleteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static - * @param {vtctldata.ShardReplicationPositionsRequest} message ShardReplicationPositionsRequest + * @param {vtctldata.SourceShardDeleteRequest} message SourceShardDeleteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationPositionsRequest.toObject = function toObject(message, options) { + SourceShardDeleteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; object.shard = ""; + object.uid = 0; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; + if (message.uid != null && message.hasOwnProperty("uid")) + object.uid = message.uid; return object; }; /** - * Converts this ShardReplicationPositionsRequest to JSON. + * Converts this SourceShardDeleteRequest to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @instance * @returns {Object.} JSON object */ - ShardReplicationPositionsRequest.prototype.toJSON = function toJSON() { + SourceShardDeleteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationPositionsRequest + * Gets the default type url for SourceShardDeleteRequest * @function getTypeUrl - * @memberof vtctldata.ShardReplicationPositionsRequest + * @memberof vtctldata.SourceShardDeleteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationPositionsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SourceShardDeleteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationPositionsRequest"; + return typeUrlPrefix + "/vtctldata.SourceShardDeleteRequest"; }; - return ShardReplicationPositionsRequest; + return SourceShardDeleteRequest; })(); - vtctldata.ShardReplicationPositionsResponse = (function() { + vtctldata.SourceShardDeleteResponse = (function() { /** - * Properties of a ShardReplicationPositionsResponse. + * Properties of a SourceShardDeleteResponse. * @memberof vtctldata - * @interface IShardReplicationPositionsResponse - * @property {Object.|null} [replication_statuses] ShardReplicationPositionsResponse replication_statuses - * @property {Object.|null} [tablet_map] ShardReplicationPositionsResponse tablet_map + * @interface ISourceShardDeleteResponse + * @property {topodata.IShard|null} [shard] SourceShardDeleteResponse shard */ /** - * Constructs a new ShardReplicationPositionsResponse. + * Constructs a new SourceShardDeleteResponse. * @memberof vtctldata - * @classdesc Represents a ShardReplicationPositionsResponse. - * @implements IShardReplicationPositionsResponse + * @classdesc Represents a SourceShardDeleteResponse. + * @implements ISourceShardDeleteResponse * @constructor - * @param {vtctldata.IShardReplicationPositionsResponse=} [properties] Properties to set + * @param {vtctldata.ISourceShardDeleteResponse=} [properties] Properties to set */ - function ShardReplicationPositionsResponse(properties) { - this.replication_statuses = {}; - this.tablet_map = {}; + function SourceShardDeleteResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -135259,133 +141978,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationPositionsResponse replication_statuses. - * @member {Object.} replication_statuses - * @memberof vtctldata.ShardReplicationPositionsResponse - * @instance - */ - ShardReplicationPositionsResponse.prototype.replication_statuses = $util.emptyObject; - - /** - * ShardReplicationPositionsResponse tablet_map. - * @member {Object.} tablet_map - * @memberof vtctldata.ShardReplicationPositionsResponse + * SourceShardDeleteResponse shard. + * @member {topodata.IShard|null|undefined} shard + * @memberof vtctldata.SourceShardDeleteResponse * @instance */ - ShardReplicationPositionsResponse.prototype.tablet_map = $util.emptyObject; + SourceShardDeleteResponse.prototype.shard = null; /** - * Creates a new ShardReplicationPositionsResponse instance using the specified properties. + * Creates a new SourceShardDeleteResponse instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static - * @param {vtctldata.IShardReplicationPositionsResponse=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse instance + * @param {vtctldata.ISourceShardDeleteResponse=} [properties] Properties to set + * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse instance */ - ShardReplicationPositionsResponse.create = function create(properties) { - return new ShardReplicationPositionsResponse(properties); + SourceShardDeleteResponse.create = function create(properties) { + return new SourceShardDeleteResponse(properties); }; /** - * Encodes the specified ShardReplicationPositionsResponse message. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. + * Encodes the specified SourceShardDeleteResponse message. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static - * @param {vtctldata.IShardReplicationPositionsResponse} message ShardReplicationPositionsResponse message or plain object to encode + * @param {vtctldata.ISourceShardDeleteResponse} message SourceShardDeleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationPositionsResponse.encode = function encode(message, writer) { + SourceShardDeleteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.replication_statuses != null && Object.hasOwnProperty.call(message, "replication_statuses")) - for (let keys = Object.keys(message.replication_statuses), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.replicationdata.Status.encode(message.replication_statuses[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } - if (message.tablet_map != null && Object.hasOwnProperty.call(message, "tablet_map")) - for (let keys = Object.keys(message.tablet_map), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.topodata.Tablet.encode(message.tablet_map[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ShardReplicationPositionsResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationPositionsResponse.verify|verify} messages. + * Encodes the specified SourceShardDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static - * @param {vtctldata.IShardReplicationPositionsResponse} message ShardReplicationPositionsResponse message or plain object to encode + * @param {vtctldata.ISourceShardDeleteResponse} message SourceShardDeleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationPositionsResponse.encodeDelimited = function encodeDelimited(message, writer) { + SourceShardDeleteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer. + * Decodes a SourceShardDeleteResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse + * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationPositionsResponse.decode = function decode(reader, length) { + SourceShardDeleteResponse.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.ShardReplicationPositionsResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.SourceShardDeleteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (message.replication_statuses === $util.emptyObject) - message.replication_statuses = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.replicationdata.Status.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.replication_statuses[key] = value; - break; - } - case 2: { - if (message.tablet_map === $util.emptyObject) - message.tablet_map = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.topodata.Tablet.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.tablet_map[key] = value; + message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); break; } default: @@ -135397,170 +142058,127 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationPositionsResponse message from the specified reader or buffer, length delimited. + * Decodes a SourceShardDeleteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse + * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationPositionsResponse.decodeDelimited = function decodeDelimited(reader) { + SourceShardDeleteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationPositionsResponse message. + * Verifies a SourceShardDeleteResponse message. * @function verify - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationPositionsResponse.verify = function verify(message) { + SourceShardDeleteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.replication_statuses != null && message.hasOwnProperty("replication_statuses")) { - if (!$util.isObject(message.replication_statuses)) - return "replication_statuses: object expected"; - let key = Object.keys(message.replication_statuses); - for (let i = 0; i < key.length; ++i) { - let error = $root.replicationdata.Status.verify(message.replication_statuses[key[i]]); - if (error) - return "replication_statuses." + error; - } - } - if (message.tablet_map != null && message.hasOwnProperty("tablet_map")) { - if (!$util.isObject(message.tablet_map)) - return "tablet_map: object expected"; - let key = Object.keys(message.tablet_map); - for (let i = 0; i < key.length; ++i) { - let error = $root.topodata.Tablet.verify(message.tablet_map[key[i]]); - if (error) - return "tablet_map." + error; - } + if (message.shard != null && message.hasOwnProperty("shard")) { + let error = $root.topodata.Shard.verify(message.shard); + if (error) + return "shard." + error; } return null; }; /** - * Creates a ShardReplicationPositionsResponse message from a plain object. Also converts values to their respective internal types. + * Creates a SourceShardDeleteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationPositionsResponse} ShardReplicationPositionsResponse + * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse */ - ShardReplicationPositionsResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationPositionsResponse) + SourceShardDeleteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.SourceShardDeleteResponse) return object; - let message = new $root.vtctldata.ShardReplicationPositionsResponse(); - if (object.replication_statuses) { - if (typeof object.replication_statuses !== "object") - throw TypeError(".vtctldata.ShardReplicationPositionsResponse.replication_statuses: object expected"); - message.replication_statuses = {}; - for (let keys = Object.keys(object.replication_statuses), i = 0; i < keys.length; ++i) { - if (typeof object.replication_statuses[keys[i]] !== "object") - throw TypeError(".vtctldata.ShardReplicationPositionsResponse.replication_statuses: object expected"); - message.replication_statuses[keys[i]] = $root.replicationdata.Status.fromObject(object.replication_statuses[keys[i]]); - } - } - if (object.tablet_map) { - if (typeof object.tablet_map !== "object") - throw TypeError(".vtctldata.ShardReplicationPositionsResponse.tablet_map: object expected"); - message.tablet_map = {}; - for (let keys = Object.keys(object.tablet_map), i = 0; i < keys.length; ++i) { - if (typeof object.tablet_map[keys[i]] !== "object") - throw TypeError(".vtctldata.ShardReplicationPositionsResponse.tablet_map: object expected"); - message.tablet_map[keys[i]] = $root.topodata.Tablet.fromObject(object.tablet_map[keys[i]]); - } + let message = new $root.vtctldata.SourceShardDeleteResponse(); + if (object.shard != null) { + if (typeof object.shard !== "object") + throw TypeError(".vtctldata.SourceShardDeleteResponse.shard: object expected"); + message.shard = $root.topodata.Shard.fromObject(object.shard); } return message; }; /** - * Creates a plain object from a ShardReplicationPositionsResponse message. Also converts values to other types if specified. + * Creates a plain object from a SourceShardDeleteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static - * @param {vtctldata.ShardReplicationPositionsResponse} message ShardReplicationPositionsResponse + * @param {vtctldata.SourceShardDeleteResponse} message SourceShardDeleteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationPositionsResponse.toObject = function toObject(message, options) { + SourceShardDeleteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.objects || options.defaults) { - object.replication_statuses = {}; - object.tablet_map = {}; - } - let keys2; - if (message.replication_statuses && (keys2 = Object.keys(message.replication_statuses)).length) { - object.replication_statuses = {}; - for (let j = 0; j < keys2.length; ++j) - object.replication_statuses[keys2[j]] = $root.replicationdata.Status.toObject(message.replication_statuses[keys2[j]], options); - } - if (message.tablet_map && (keys2 = Object.keys(message.tablet_map)).length) { - object.tablet_map = {}; - for (let j = 0; j < keys2.length; ++j) - object.tablet_map[keys2[j]] = $root.topodata.Tablet.toObject(message.tablet_map[keys2[j]], options); - } + if (options.defaults) + object.shard = null; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = $root.topodata.Shard.toObject(message.shard, options); return object; }; /** - * Converts this ShardReplicationPositionsResponse to JSON. + * Converts this SourceShardDeleteResponse to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @instance * @returns {Object.} JSON object */ - ShardReplicationPositionsResponse.prototype.toJSON = function toJSON() { + SourceShardDeleteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationPositionsResponse + * Gets the default type url for SourceShardDeleteResponse * @function getTypeUrl - * @memberof vtctldata.ShardReplicationPositionsResponse + * @memberof vtctldata.SourceShardDeleteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationPositionsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + SourceShardDeleteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationPositionsResponse"; + return typeUrlPrefix + "/vtctldata.SourceShardDeleteResponse"; }; - return ShardReplicationPositionsResponse; + return SourceShardDeleteResponse; })(); - vtctldata.ShardReplicationRemoveRequest = (function() { + vtctldata.StartReplicationRequest = (function() { /** - * Properties of a ShardReplicationRemoveRequest. + * Properties of a StartReplicationRequest. * @memberof vtctldata - * @interface IShardReplicationRemoveRequest - * @property {string|null} [keyspace] ShardReplicationRemoveRequest keyspace - * @property {string|null} [shard] ShardReplicationRemoveRequest shard - * @property {topodata.ITabletAlias|null} [tablet_alias] ShardReplicationRemoveRequest tablet_alias + * @interface IStartReplicationRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] StartReplicationRequest tablet_alias */ /** - * Constructs a new ShardReplicationRemoveRequest. + * Constructs a new StartReplicationRequest. * @memberof vtctldata - * @classdesc Represents a ShardReplicationRemoveRequest. - * @implements IShardReplicationRemoveRequest + * @classdesc Represents a StartReplicationRequest. + * @implements IStartReplicationRequest * @constructor - * @param {vtctldata.IShardReplicationRemoveRequest=} [properties] Properties to set + * @param {vtctldata.IStartReplicationRequest=} [properties] Properties to set */ - function ShardReplicationRemoveRequest(properties) { + function StartReplicationRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -135568,102 +142186,74 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ShardReplicationRemoveRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ShardReplicationRemoveRequest - * @instance - */ - ShardReplicationRemoveRequest.prototype.keyspace = ""; - - /** - * ShardReplicationRemoveRequest shard. - * @member {string} shard - * @memberof vtctldata.ShardReplicationRemoveRequest - * @instance - */ - ShardReplicationRemoveRequest.prototype.shard = ""; - - /** - * ShardReplicationRemoveRequest tablet_alias. + * StartReplicationRequest tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @instance */ - ShardReplicationRemoveRequest.prototype.tablet_alias = null; + StartReplicationRequest.prototype.tablet_alias = null; /** - * Creates a new ShardReplicationRemoveRequest instance using the specified properties. + * Creates a new StartReplicationRequest instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static - * @param {vtctldata.IShardReplicationRemoveRequest=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest instance + * @param {vtctldata.IStartReplicationRequest=} [properties] Properties to set + * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest instance */ - ShardReplicationRemoveRequest.create = function create(properties) { - return new ShardReplicationRemoveRequest(properties); + StartReplicationRequest.create = function create(properties) { + return new StartReplicationRequest(properties); }; /** - * Encodes the specified ShardReplicationRemoveRequest message. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. + * Encodes the specified StartReplicationRequest message. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static - * @param {vtctldata.IShardReplicationRemoveRequest} message ShardReplicationRemoveRequest message or plain object to encode + * @param {vtctldata.IStartReplicationRequest} message StartReplicationRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationRemoveRequest.encode = function encode(message, writer) { + StartReplicationRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified ShardReplicationRemoveRequest message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveRequest.verify|verify} messages. + * Encodes the specified StartReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static - * @param {vtctldata.IShardReplicationRemoveRequest} message ShardReplicationRemoveRequest message or plain object to encode + * @param {vtctldata.IStartReplicationRequest} message StartReplicationRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationRemoveRequest.encodeDelimited = function encodeDelimited(message, writer) { + StartReplicationRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer. + * Decodes a StartReplicationRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest + * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationRemoveRequest.decode = function decode(reader, length) { + StartReplicationRequest.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.ShardReplicationRemoveRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.StartReplicationRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } @@ -135676,38 +142266,32 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationRemoveRequest message from the specified reader or buffer, length delimited. + * Decodes a StartReplicationRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest + * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationRemoveRequest.decodeDelimited = function decodeDelimited(reader) { + StartReplicationRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationRemoveRequest message. + * Verifies a StartReplicationRequest message. * @function verify - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationRemoveRequest.verify = function verify(message) { + StartReplicationRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { let error = $root.topodata.TabletAlias.verify(message.tablet_alias); if (error) @@ -135717,102 +142301,91 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Creates a ShardReplicationRemoveRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StartReplicationRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationRemoveRequest} ShardReplicationRemoveRequest + * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest */ - ShardReplicationRemoveRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationRemoveRequest) + StartReplicationRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.StartReplicationRequest) return object; - let message = new $root.vtctldata.ShardReplicationRemoveRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); + let message = new $root.vtctldata.StartReplicationRequest(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.ShardReplicationRemoveRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.StartReplicationRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } return message; }; /** - * Creates a plain object from a ShardReplicationRemoveRequest message. Also converts values to other types if specified. + * Creates a plain object from a StartReplicationRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static - * @param {vtctldata.ShardReplicationRemoveRequest} message ShardReplicationRemoveRequest + * @param {vtctldata.StartReplicationRequest} message StartReplicationRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationRemoveRequest.toObject = function toObject(message, options) { + StartReplicationRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; + if (options.defaults) object.tablet_alias = null; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); return object; }; /** - * Converts this ShardReplicationRemoveRequest to JSON. + * Converts this StartReplicationRequest to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @instance * @returns {Object.} JSON object */ - ShardReplicationRemoveRequest.prototype.toJSON = function toJSON() { + StartReplicationRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationRemoveRequest + * Gets the default type url for StartReplicationRequest * @function getTypeUrl - * @memberof vtctldata.ShardReplicationRemoveRequest + * @memberof vtctldata.StartReplicationRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationRemoveRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StartReplicationRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationRemoveRequest"; + return typeUrlPrefix + "/vtctldata.StartReplicationRequest"; }; - return ShardReplicationRemoveRequest; + return StartReplicationRequest; })(); - vtctldata.ShardReplicationRemoveResponse = (function() { + vtctldata.StartReplicationResponse = (function() { /** - * Properties of a ShardReplicationRemoveResponse. + * Properties of a StartReplicationResponse. * @memberof vtctldata - * @interface IShardReplicationRemoveResponse + * @interface IStartReplicationResponse */ /** - * Constructs a new ShardReplicationRemoveResponse. + * Constructs a new StartReplicationResponse. * @memberof vtctldata - * @classdesc Represents a ShardReplicationRemoveResponse. - * @implements IShardReplicationRemoveResponse + * @classdesc Represents a StartReplicationResponse. + * @implements IStartReplicationResponse * @constructor - * @param {vtctldata.IShardReplicationRemoveResponse=} [properties] Properties to set + * @param {vtctldata.IStartReplicationResponse=} [properties] Properties to set */ - function ShardReplicationRemoveResponse(properties) { + function StartReplicationResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -135820,60 +142393,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new ShardReplicationRemoveResponse instance using the specified properties. + * Creates a new StartReplicationResponse instance using the specified properties. * @function create - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static - * @param {vtctldata.IShardReplicationRemoveResponse=} [properties] Properties to set - * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse instance + * @param {vtctldata.IStartReplicationResponse=} [properties] Properties to set + * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse instance */ - ShardReplicationRemoveResponse.create = function create(properties) { - return new ShardReplicationRemoveResponse(properties); + StartReplicationResponse.create = function create(properties) { + return new StartReplicationResponse(properties); }; /** - * Encodes the specified ShardReplicationRemoveResponse message. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. + * Encodes the specified StartReplicationResponse message. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static - * @param {vtctldata.IShardReplicationRemoveResponse} message ShardReplicationRemoveResponse message or plain object to encode + * @param {vtctldata.IStartReplicationResponse} message StartReplicationResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationRemoveResponse.encode = function encode(message, writer) { + StartReplicationResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified ShardReplicationRemoveResponse message, length delimited. Does not implicitly {@link vtctldata.ShardReplicationRemoveResponse.verify|verify} messages. + * Encodes the specified StartReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static - * @param {vtctldata.IShardReplicationRemoveResponse} message ShardReplicationRemoveResponse message or plain object to encode + * @param {vtctldata.IStartReplicationResponse} message StartReplicationResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ShardReplicationRemoveResponse.encodeDelimited = function encodeDelimited(message, writer) { + StartReplicationResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer. + * Decodes a StartReplicationResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse + * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationRemoveResponse.decode = function decode(reader, length) { + StartReplicationResponse.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.ShardReplicationRemoveResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.StartReplicationResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -135886,110 +142459,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ShardReplicationRemoveResponse message from the specified reader or buffer, length delimited. + * Decodes a StartReplicationResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse + * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ShardReplicationRemoveResponse.decodeDelimited = function decodeDelimited(reader) { + StartReplicationResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ShardReplicationRemoveResponse message. + * Verifies a StartReplicationResponse message. * @function verify - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ShardReplicationRemoveResponse.verify = function verify(message) { + StartReplicationResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a ShardReplicationRemoveResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StartReplicationResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ShardReplicationRemoveResponse} ShardReplicationRemoveResponse + * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse */ - ShardReplicationRemoveResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ShardReplicationRemoveResponse) + StartReplicationResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.StartReplicationResponse) return object; - return new $root.vtctldata.ShardReplicationRemoveResponse(); + return new $root.vtctldata.StartReplicationResponse(); }; /** - * Creates a plain object from a ShardReplicationRemoveResponse message. Also converts values to other types if specified. + * Creates a plain object from a StartReplicationResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static - * @param {vtctldata.ShardReplicationRemoveResponse} message ShardReplicationRemoveResponse + * @param {vtctldata.StartReplicationResponse} message StartReplicationResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ShardReplicationRemoveResponse.toObject = function toObject() { + StartReplicationResponse.toObject = function toObject() { return {}; }; /** - * Converts this ShardReplicationRemoveResponse to JSON. + * Converts this StartReplicationResponse to JSON. * @function toJSON - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @instance * @returns {Object.} JSON object */ - ShardReplicationRemoveResponse.prototype.toJSON = function toJSON() { + StartReplicationResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ShardReplicationRemoveResponse + * Gets the default type url for StartReplicationResponse * @function getTypeUrl - * @memberof vtctldata.ShardReplicationRemoveResponse + * @memberof vtctldata.StartReplicationResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ShardReplicationRemoveResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StartReplicationResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ShardReplicationRemoveResponse"; + return typeUrlPrefix + "/vtctldata.StartReplicationResponse"; }; - return ShardReplicationRemoveResponse; + return StartReplicationResponse; })(); - vtctldata.SleepTabletRequest = (function() { + vtctldata.StopReplicationRequest = (function() { /** - * Properties of a SleepTabletRequest. + * Properties of a StopReplicationRequest. * @memberof vtctldata - * @interface ISleepTabletRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] SleepTabletRequest tablet_alias - * @property {vttime.IDuration|null} [duration] SleepTabletRequest duration + * @interface IStopReplicationRequest + * @property {topodata.ITabletAlias|null} [tablet_alias] StopReplicationRequest tablet_alias */ /** - * Constructs a new SleepTabletRequest. + * Constructs a new StopReplicationRequest. * @memberof vtctldata - * @classdesc Represents a SleepTabletRequest. - * @implements ISleepTabletRequest + * @classdesc Represents a StopReplicationRequest. + * @implements IStopReplicationRequest * @constructor - * @param {vtctldata.ISleepTabletRequest=} [properties] Properties to set + * @param {vtctldata.IStopReplicationRequest=} [properties] Properties to set */ - function SleepTabletRequest(properties) { + function StopReplicationRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -135997,80 +142569,70 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SleepTabletRequest tablet_alias. + * StopReplicationRequest tablet_alias. * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.SleepTabletRequest - * @instance - */ - SleepTabletRequest.prototype.tablet_alias = null; - - /** - * SleepTabletRequest duration. - * @member {vttime.IDuration|null|undefined} duration - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @instance */ - SleepTabletRequest.prototype.duration = null; + StopReplicationRequest.prototype.tablet_alias = null; /** - * Creates a new SleepTabletRequest instance using the specified properties. + * Creates a new StopReplicationRequest instance using the specified properties. * @function create - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static - * @param {vtctldata.ISleepTabletRequest=} [properties] Properties to set - * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest instance + * @param {vtctldata.IStopReplicationRequest=} [properties] Properties to set + * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest instance */ - SleepTabletRequest.create = function create(properties) { - return new SleepTabletRequest(properties); + StopReplicationRequest.create = function create(properties) { + return new StopReplicationRequest(properties); }; /** - * Encodes the specified SleepTabletRequest message. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. + * Encodes the specified StopReplicationRequest message. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static - * @param {vtctldata.ISleepTabletRequest} message SleepTabletRequest message or plain object to encode + * @param {vtctldata.IStopReplicationRequest} message StopReplicationRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SleepTabletRequest.encode = function encode(message, writer) { + StopReplicationRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.duration != null && Object.hasOwnProperty.call(message, "duration")) - $root.vttime.Duration.encode(message.duration, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified SleepTabletRequest message, length delimited. Does not implicitly {@link vtctldata.SleepTabletRequest.verify|verify} messages. + * Encodes the specified StopReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static - * @param {vtctldata.ISleepTabletRequest} message SleepTabletRequest message or plain object to encode + * @param {vtctldata.IStopReplicationRequest} message StopReplicationRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SleepTabletRequest.encodeDelimited = function encodeDelimited(message, writer) { + StopReplicationRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SleepTabletRequest message from the specified reader or buffer. + * Decodes a StopReplicationRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest + * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SleepTabletRequest.decode = function decode(reader, length) { + StopReplicationRequest.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.SleepTabletRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.StopReplicationRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -136078,10 +142640,6 @@ export const vtctldata = $root.vtctldata = (() => { message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } - case 2: { - message.duration = $root.vttime.Duration.decode(reader, reader.uint32()); - break; - } default: reader.skipType(tag & 7); break; @@ -136091,30 +142649,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SleepTabletRequest message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest + * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SleepTabletRequest.decodeDelimited = function decodeDelimited(reader) { + StopReplicationRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SleepTabletRequest message. + * Verifies a StopReplicationRequest message. * @function verify - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SleepTabletRequest.verify = function verify(message) { + StopReplicationRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { @@ -136122,109 +142680,95 @@ export const vtctldata = $root.vtctldata = (() => { if (error) return "tablet_alias." + error; } - if (message.duration != null && message.hasOwnProperty("duration")) { - let error = $root.vttime.Duration.verify(message.duration); - if (error) - return "duration." + error; - } return null; }; /** - * Creates a SleepTabletRequest message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SleepTabletRequest} SleepTabletRequest + * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest */ - SleepTabletRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SleepTabletRequest) + StopReplicationRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.StopReplicationRequest) return object; - let message = new $root.vtctldata.SleepTabletRequest(); + let message = new $root.vtctldata.StopReplicationRequest(); if (object.tablet_alias != null) { if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.SleepTabletRequest.tablet_alias: object expected"); + throw TypeError(".vtctldata.StopReplicationRequest.tablet_alias: object expected"); message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); } - if (object.duration != null) { - if (typeof object.duration !== "object") - throw TypeError(".vtctldata.SleepTabletRequest.duration: object expected"); - message.duration = $root.vttime.Duration.fromObject(object.duration); - } return message; }; /** - * Creates a plain object from a SleepTabletRequest message. Also converts values to other types if specified. + * Creates a plain object from a StopReplicationRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static - * @param {vtctldata.SleepTabletRequest} message SleepTabletRequest + * @param {vtctldata.StopReplicationRequest} message StopReplicationRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SleepTabletRequest.toObject = function toObject(message, options) { + StopReplicationRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { + if (options.defaults) object.tablet_alias = null; - object.duration = null; - } if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); - if (message.duration != null && message.hasOwnProperty("duration")) - object.duration = $root.vttime.Duration.toObject(message.duration, options); return object; }; /** - * Converts this SleepTabletRequest to JSON. + * Converts this StopReplicationRequest to JSON. * @function toJSON - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @instance * @returns {Object.} JSON object */ - SleepTabletRequest.prototype.toJSON = function toJSON() { + StopReplicationRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SleepTabletRequest + * Gets the default type url for StopReplicationRequest * @function getTypeUrl - * @memberof vtctldata.SleepTabletRequest + * @memberof vtctldata.StopReplicationRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SleepTabletRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StopReplicationRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SleepTabletRequest"; + return typeUrlPrefix + "/vtctldata.StopReplicationRequest"; }; - return SleepTabletRequest; + return StopReplicationRequest; })(); - vtctldata.SleepTabletResponse = (function() { + vtctldata.StopReplicationResponse = (function() { /** - * Properties of a SleepTabletResponse. + * Properties of a StopReplicationResponse. * @memberof vtctldata - * @interface ISleepTabletResponse + * @interface IStopReplicationResponse */ /** - * Constructs a new SleepTabletResponse. + * Constructs a new StopReplicationResponse. * @memberof vtctldata - * @classdesc Represents a SleepTabletResponse. - * @implements ISleepTabletResponse + * @classdesc Represents a StopReplicationResponse. + * @implements IStopReplicationResponse * @constructor - * @param {vtctldata.ISleepTabletResponse=} [properties] Properties to set + * @param {vtctldata.IStopReplicationResponse=} [properties] Properties to set */ - function SleepTabletResponse(properties) { + function StopReplicationResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -136232,60 +142776,60 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new SleepTabletResponse instance using the specified properties. + * Creates a new StopReplicationResponse instance using the specified properties. * @function create - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static - * @param {vtctldata.ISleepTabletResponse=} [properties] Properties to set - * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse instance + * @param {vtctldata.IStopReplicationResponse=} [properties] Properties to set + * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse instance */ - SleepTabletResponse.create = function create(properties) { - return new SleepTabletResponse(properties); + StopReplicationResponse.create = function create(properties) { + return new StopReplicationResponse(properties); }; /** - * Encodes the specified SleepTabletResponse message. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. + * Encodes the specified StopReplicationResponse message. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static - * @param {vtctldata.ISleepTabletResponse} message SleepTabletResponse message or plain object to encode + * @param {vtctldata.IStopReplicationResponse} message StopReplicationResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SleepTabletResponse.encode = function encode(message, writer) { + StopReplicationResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); return writer; }; /** - * Encodes the specified SleepTabletResponse message, length delimited. Does not implicitly {@link vtctldata.SleepTabletResponse.verify|verify} messages. + * Encodes the specified StopReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static - * @param {vtctldata.ISleepTabletResponse} message SleepTabletResponse message or plain object to encode + * @param {vtctldata.IStopReplicationResponse} message StopReplicationResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SleepTabletResponse.encodeDelimited = function encodeDelimited(message, writer) { + StopReplicationResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SleepTabletResponse message from the specified reader or buffer. + * Decodes a StopReplicationResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse + * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SleepTabletResponse.decode = function decode(reader, length) { + StopReplicationResponse.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.SleepTabletResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.StopReplicationResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -136298,116 +142842,109 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SleepTabletResponse message from the specified reader or buffer, length delimited. + * Decodes a StopReplicationResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse + * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SleepTabletResponse.decodeDelimited = function decodeDelimited(reader) { + StopReplicationResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SleepTabletResponse message. + * Verifies a StopReplicationResponse message. * @function verify - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SleepTabletResponse.verify = function verify(message) { + StopReplicationResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; return null; }; /** - * Creates a SleepTabletResponse message from a plain object. Also converts values to their respective internal types. + * Creates a StopReplicationResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SleepTabletResponse} SleepTabletResponse + * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse */ - SleepTabletResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SleepTabletResponse) + StopReplicationResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.StopReplicationResponse) return object; - return new $root.vtctldata.SleepTabletResponse(); + return new $root.vtctldata.StopReplicationResponse(); }; /** - * Creates a plain object from a SleepTabletResponse message. Also converts values to other types if specified. + * Creates a plain object from a StopReplicationResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static - * @param {vtctldata.SleepTabletResponse} message SleepTabletResponse + * @param {vtctldata.StopReplicationResponse} message StopReplicationResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SleepTabletResponse.toObject = function toObject() { + StopReplicationResponse.toObject = function toObject() { return {}; }; /** - * Converts this SleepTabletResponse to JSON. + * Converts this StopReplicationResponse to JSON. * @function toJSON - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @instance * @returns {Object.} JSON object */ - SleepTabletResponse.prototype.toJSON = function toJSON() { + StopReplicationResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SleepTabletResponse + * Gets the default type url for StopReplicationResponse * @function getTypeUrl - * @memberof vtctldata.SleepTabletResponse + * @memberof vtctldata.StopReplicationResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SleepTabletResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + StopReplicationResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SleepTabletResponse"; + return typeUrlPrefix + "/vtctldata.StopReplicationResponse"; }; - return SleepTabletResponse; + return StopReplicationResponse; })(); - vtctldata.SourceShardAddRequest = (function() { + vtctldata.TabletExternallyReparentedRequest = (function() { /** - * Properties of a SourceShardAddRequest. + * Properties of a TabletExternallyReparentedRequest. * @memberof vtctldata - * @interface ISourceShardAddRequest - * @property {string|null} [keyspace] SourceShardAddRequest keyspace - * @property {string|null} [shard] SourceShardAddRequest shard - * @property {number|null} [uid] SourceShardAddRequest uid - * @property {string|null} [source_keyspace] SourceShardAddRequest source_keyspace - * @property {string|null} [source_shard] SourceShardAddRequest source_shard - * @property {topodata.IKeyRange|null} [key_range] SourceShardAddRequest key_range - * @property {Array.|null} [tables] SourceShardAddRequest tables + * @interface ITabletExternallyReparentedRequest + * @property {topodata.ITabletAlias|null} [tablet] TabletExternallyReparentedRequest tablet */ /** - * Constructs a new SourceShardAddRequest. + * Constructs a new TabletExternallyReparentedRequest. * @memberof vtctldata - * @classdesc Represents a SourceShardAddRequest. - * @implements ISourceShardAddRequest + * @classdesc Represents a TabletExternallyReparentedRequest. + * @implements ITabletExternallyReparentedRequest * @constructor - * @param {vtctldata.ISourceShardAddRequest=} [properties] Properties to set + * @param {vtctldata.ITabletExternallyReparentedRequest=} [properties] Properties to set */ - function SourceShardAddRequest(properties) { - this.tables = []; + function TabletExternallyReparentedRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -136415,162 +142952,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SourceShardAddRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.keyspace = ""; - - /** - * SourceShardAddRequest shard. - * @member {string} shard - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.shard = ""; - - /** - * SourceShardAddRequest uid. - * @member {number} uid - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.uid = 0; - - /** - * SourceShardAddRequest source_keyspace. - * @member {string} source_keyspace - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.source_keyspace = ""; - - /** - * SourceShardAddRequest source_shard. - * @member {string} source_shard - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.source_shard = ""; - - /** - * SourceShardAddRequest key_range. - * @member {topodata.IKeyRange|null|undefined} key_range - * @memberof vtctldata.SourceShardAddRequest - * @instance - */ - SourceShardAddRequest.prototype.key_range = null; - - /** - * SourceShardAddRequest tables. - * @member {Array.} tables - * @memberof vtctldata.SourceShardAddRequest + * TabletExternallyReparentedRequest tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.TabletExternallyReparentedRequest * @instance */ - SourceShardAddRequest.prototype.tables = $util.emptyArray; + TabletExternallyReparentedRequest.prototype.tablet = null; /** - * Creates a new SourceShardAddRequest instance using the specified properties. + * Creates a new TabletExternallyReparentedRequest instance using the specified properties. * @function create - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static - * @param {vtctldata.ISourceShardAddRequest=} [properties] Properties to set - * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest instance + * @param {vtctldata.ITabletExternallyReparentedRequest=} [properties] Properties to set + * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest instance */ - SourceShardAddRequest.create = function create(properties) { - return new SourceShardAddRequest(properties); + TabletExternallyReparentedRequest.create = function create(properties) { + return new TabletExternallyReparentedRequest(properties); }; /** - * Encodes the specified SourceShardAddRequest message. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. + * Encodes the specified TabletExternallyReparentedRequest message. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static - * @param {vtctldata.ISourceShardAddRequest} message SourceShardAddRequest message or plain object to encode + * @param {vtctldata.ITabletExternallyReparentedRequest} message TabletExternallyReparentedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardAddRequest.encode = function encode(message, writer) { + TabletExternallyReparentedRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.uid != null && Object.hasOwnProperty.call(message, "uid")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.uid); - if (message.source_keyspace != null && Object.hasOwnProperty.call(message, "source_keyspace")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.source_keyspace); - if (message.source_shard != null && Object.hasOwnProperty.call(message, "source_shard")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.source_shard); - if (message.key_range != null && Object.hasOwnProperty.call(message, "key_range")) - $root.topodata.KeyRange.encode(message.key_range, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); - if (message.tables != null && message.tables.length) - for (let i = 0; i < message.tables.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.tables[i]); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified SourceShardAddRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddRequest.verify|verify} messages. + * Encodes the specified TabletExternallyReparentedRequest message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static - * @param {vtctldata.ISourceShardAddRequest} message SourceShardAddRequest message or plain object to encode + * @param {vtctldata.ITabletExternallyReparentedRequest} message TabletExternallyReparentedRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardAddRequest.encodeDelimited = function encodeDelimited(message, writer) { + TabletExternallyReparentedRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SourceShardAddRequest message from the specified reader or buffer. + * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest + * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardAddRequest.decode = function decode(reader, length) { + TabletExternallyReparentedRequest.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.SourceShardAddRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.TabletExternallyReparentedRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); - break; - } - case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.uid = reader.int32(); - break; - } - case 4: { - message.source_keyspace = reader.string(); - break; - } - case 5: { - message.source_shard = reader.string(); - break; - } - case 6: { - message.key_range = $root.topodata.KeyRange.decode(reader, reader.uint32()); - break; - } - case 7: { - if (!(message.tables && message.tables.length)) - message.tables = []; - message.tables.push(reader.string()); + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -136582,189 +143032,130 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SourceShardAddRequest message from the specified reader or buffer, length delimited. + * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest + * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardAddRequest.decodeDelimited = function decodeDelimited(reader) { + TabletExternallyReparentedRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SourceShardAddRequest message. + * Verifies a TabletExternallyReparentedRequest message. * @function verify - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SourceShardAddRequest.verify = function verify(message) { + TabletExternallyReparentedRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.uid != null && message.hasOwnProperty("uid")) - if (!$util.isInteger(message.uid)) - return "uid: integer expected"; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - if (!$util.isString(message.source_keyspace)) - return "source_keyspace: string expected"; - if (message.source_shard != null && message.hasOwnProperty("source_shard")) - if (!$util.isString(message.source_shard)) - return "source_shard: string expected"; - if (message.key_range != null && message.hasOwnProperty("key_range")) { - let error = $root.topodata.KeyRange.verify(message.key_range); + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); if (error) - return "key_range." + error; - } - if (message.tables != null && message.hasOwnProperty("tables")) { - if (!Array.isArray(message.tables)) - return "tables: array expected"; - for (let i = 0; i < message.tables.length; ++i) - if (!$util.isString(message.tables[i])) - return "tables: string[] expected"; + return "tablet." + error; } return null; }; /** - * Creates a SourceShardAddRequest message from a plain object. Also converts values to their respective internal types. + * Creates a TabletExternallyReparentedRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SourceShardAddRequest} SourceShardAddRequest + * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest */ - SourceShardAddRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SourceShardAddRequest) - return object; - let message = new $root.vtctldata.SourceShardAddRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.uid != null) - message.uid = object.uid | 0; - if (object.source_keyspace != null) - message.source_keyspace = String(object.source_keyspace); - if (object.source_shard != null) - message.source_shard = String(object.source_shard); - if (object.key_range != null) { - if (typeof object.key_range !== "object") - throw TypeError(".vtctldata.SourceShardAddRequest.key_range: object expected"); - message.key_range = $root.topodata.KeyRange.fromObject(object.key_range); - } - if (object.tables) { - if (!Array.isArray(object.tables)) - throw TypeError(".vtctldata.SourceShardAddRequest.tables: array expected"); - message.tables = []; - for (let i = 0; i < object.tables.length; ++i) - message.tables[i] = String(object.tables[i]); + TabletExternallyReparentedRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.TabletExternallyReparentedRequest) + return object; + let message = new $root.vtctldata.TabletExternallyReparentedRequest(); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.TabletExternallyReparentedRequest.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); } return message; }; /** - * Creates a plain object from a SourceShardAddRequest message. Also converts values to other types if specified. + * Creates a plain object from a TabletExternallyReparentedRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static - * @param {vtctldata.SourceShardAddRequest} message SourceShardAddRequest + * @param {vtctldata.TabletExternallyReparentedRequest} message TabletExternallyReparentedRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SourceShardAddRequest.toObject = function toObject(message, options) { + TabletExternallyReparentedRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.tables = []; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.uid = 0; - object.source_keyspace = ""; - object.source_shard = ""; - object.key_range = null; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.uid != null && message.hasOwnProperty("uid")) - object.uid = message.uid; - if (message.source_keyspace != null && message.hasOwnProperty("source_keyspace")) - object.source_keyspace = message.source_keyspace; - if (message.source_shard != null && message.hasOwnProperty("source_shard")) - object.source_shard = message.source_shard; - if (message.key_range != null && message.hasOwnProperty("key_range")) - object.key_range = $root.topodata.KeyRange.toObject(message.key_range, options); - if (message.tables && message.tables.length) { - object.tables = []; - for (let j = 0; j < message.tables.length; ++j) - object.tables[j] = message.tables[j]; - } + if (options.defaults) + object.tablet = null; + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); return object; }; /** - * Converts this SourceShardAddRequest to JSON. + * Converts this TabletExternallyReparentedRequest to JSON. * @function toJSON - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @instance * @returns {Object.} JSON object */ - SourceShardAddRequest.prototype.toJSON = function toJSON() { + TabletExternallyReparentedRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SourceShardAddRequest + * Gets the default type url for TabletExternallyReparentedRequest * @function getTypeUrl - * @memberof vtctldata.SourceShardAddRequest + * @memberof vtctldata.TabletExternallyReparentedRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SourceShardAddRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + TabletExternallyReparentedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SourceShardAddRequest"; + return typeUrlPrefix + "/vtctldata.TabletExternallyReparentedRequest"; }; - return SourceShardAddRequest; + return TabletExternallyReparentedRequest; })(); - vtctldata.SourceShardAddResponse = (function() { + vtctldata.TabletExternallyReparentedResponse = (function() { /** - * Properties of a SourceShardAddResponse. + * Properties of a TabletExternallyReparentedResponse. * @memberof vtctldata - * @interface ISourceShardAddResponse - * @property {topodata.IShard|null} [shard] SourceShardAddResponse shard + * @interface ITabletExternallyReparentedResponse + * @property {string|null} [keyspace] TabletExternallyReparentedResponse keyspace + * @property {string|null} [shard] TabletExternallyReparentedResponse shard + * @property {topodata.ITabletAlias|null} [new_primary] TabletExternallyReparentedResponse new_primary + * @property {topodata.ITabletAlias|null} [old_primary] TabletExternallyReparentedResponse old_primary */ /** - * Constructs a new SourceShardAddResponse. + * Constructs a new TabletExternallyReparentedResponse. * @memberof vtctldata - * @classdesc Represents a SourceShardAddResponse. - * @implements ISourceShardAddResponse + * @classdesc Represents a TabletExternallyReparentedResponse. + * @implements ITabletExternallyReparentedResponse * @constructor - * @param {vtctldata.ISourceShardAddResponse=} [properties] Properties to set + * @param {vtctldata.ITabletExternallyReparentedResponse=} [properties] Properties to set */ - function SourceShardAddResponse(properties) { + function TabletExternallyReparentedResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -136772,75 +143163,117 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SourceShardAddResponse shard. - * @member {topodata.IShard|null|undefined} shard - * @memberof vtctldata.SourceShardAddResponse + * TabletExternallyReparentedResponse keyspace. + * @member {string} keyspace + * @memberof vtctldata.TabletExternallyReparentedResponse * @instance */ - SourceShardAddResponse.prototype.shard = null; + TabletExternallyReparentedResponse.prototype.keyspace = ""; /** - * Creates a new SourceShardAddResponse instance using the specified properties. + * TabletExternallyReparentedResponse shard. + * @member {string} shard + * @memberof vtctldata.TabletExternallyReparentedResponse + * @instance + */ + TabletExternallyReparentedResponse.prototype.shard = ""; + + /** + * TabletExternallyReparentedResponse new_primary. + * @member {topodata.ITabletAlias|null|undefined} new_primary + * @memberof vtctldata.TabletExternallyReparentedResponse + * @instance + */ + TabletExternallyReparentedResponse.prototype.new_primary = null; + + /** + * TabletExternallyReparentedResponse old_primary. + * @member {topodata.ITabletAlias|null|undefined} old_primary + * @memberof vtctldata.TabletExternallyReparentedResponse + * @instance + */ + TabletExternallyReparentedResponse.prototype.old_primary = null; + + /** + * Creates a new TabletExternallyReparentedResponse instance using the specified properties. * @function create - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static - * @param {vtctldata.ISourceShardAddResponse=} [properties] Properties to set - * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse instance + * @param {vtctldata.ITabletExternallyReparentedResponse=} [properties] Properties to set + * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse instance */ - SourceShardAddResponse.create = function create(properties) { - return new SourceShardAddResponse(properties); + TabletExternallyReparentedResponse.create = function create(properties) { + return new TabletExternallyReparentedResponse(properties); }; /** - * Encodes the specified SourceShardAddResponse message. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. + * Encodes the specified TabletExternallyReparentedResponse message. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static - * @param {vtctldata.ISourceShardAddResponse} message SourceShardAddResponse message or plain object to encode + * @param {vtctldata.ITabletExternallyReparentedResponse} message TabletExternallyReparentedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardAddResponse.encode = function encode(message, writer) { + TabletExternallyReparentedResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) + $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.old_primary != null && Object.hasOwnProperty.call(message, "old_primary")) + $root.topodata.TabletAlias.encode(message.old_primary, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; /** - * Encodes the specified SourceShardAddResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardAddResponse.verify|verify} messages. + * Encodes the specified TabletExternallyReparentedResponse message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static - * @param {vtctldata.ISourceShardAddResponse} message SourceShardAddResponse message or plain object to encode + * @param {vtctldata.ITabletExternallyReparentedResponse} message TabletExternallyReparentedResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardAddResponse.encodeDelimited = function encodeDelimited(message, writer) { + TabletExternallyReparentedResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SourceShardAddResponse message from the specified reader or buffer. + * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse + * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardAddResponse.decode = function decode(reader, length) { + TabletExternallyReparentedResponse.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.SourceShardAddResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.TabletExternallyReparentedResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.shard = reader.string(); + break; + } + case 3: { + message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 4: { + message.old_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } default: @@ -136852,129 +143285,158 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SourceShardAddResponse message from the specified reader or buffer, length delimited. + * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse + * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardAddResponse.decodeDelimited = function decodeDelimited(reader) { + TabletExternallyReparentedResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SourceShardAddResponse message. + * Verifies a TabletExternallyReparentedResponse message. * @function verify - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SourceShardAddResponse.verify = function verify(message) { + TabletExternallyReparentedResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.topodata.Shard.verify(message.shard); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) { + let error = $root.topodata.TabletAlias.verify(message.new_primary); if (error) - return "shard." + error; + return "new_primary." + error; + } + if (message.old_primary != null && message.hasOwnProperty("old_primary")) { + let error = $root.topodata.TabletAlias.verify(message.old_primary); + if (error) + return "old_primary." + error; } return null; }; /** - * Creates a SourceShardAddResponse message from a plain object. Also converts values to their respective internal types. + * Creates a TabletExternallyReparentedResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SourceShardAddResponse} SourceShardAddResponse + * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse */ - SourceShardAddResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SourceShardAddResponse) + TabletExternallyReparentedResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.TabletExternallyReparentedResponse) return object; - let message = new $root.vtctldata.SourceShardAddResponse(); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.SourceShardAddResponse.shard: object expected"); - message.shard = $root.topodata.Shard.fromObject(object.shard); + let message = new $root.vtctldata.TabletExternallyReparentedResponse(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.new_primary != null) { + if (typeof object.new_primary !== "object") + throw TypeError(".vtctldata.TabletExternallyReparentedResponse.new_primary: object expected"); + message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); + } + if (object.old_primary != null) { + if (typeof object.old_primary !== "object") + throw TypeError(".vtctldata.TabletExternallyReparentedResponse.old_primary: object expected"); + message.old_primary = $root.topodata.TabletAlias.fromObject(object.old_primary); } return message; }; /** - * Creates a plain object from a SourceShardAddResponse message. Also converts values to other types if specified. + * Creates a plain object from a TabletExternallyReparentedResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static - * @param {vtctldata.SourceShardAddResponse} message SourceShardAddResponse + * @param {vtctldata.TabletExternallyReparentedResponse} message TabletExternallyReparentedResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SourceShardAddResponse.toObject = function toObject(message, options) { + TabletExternallyReparentedResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.shard = null; + if (options.defaults) { + object.keyspace = ""; + object.shard = ""; + object.new_primary = null; + object.old_primary = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.topodata.Shard.toObject(message.shard, options); + object.shard = message.shard; + if (message.new_primary != null && message.hasOwnProperty("new_primary")) + object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); + if (message.old_primary != null && message.hasOwnProperty("old_primary")) + object.old_primary = $root.topodata.TabletAlias.toObject(message.old_primary, options); return object; }; /** - * Converts this SourceShardAddResponse to JSON. + * Converts this TabletExternallyReparentedResponse to JSON. * @function toJSON - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @instance * @returns {Object.} JSON object */ - SourceShardAddResponse.prototype.toJSON = function toJSON() { + TabletExternallyReparentedResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SourceShardAddResponse + * Gets the default type url for TabletExternallyReparentedResponse * @function getTypeUrl - * @memberof vtctldata.SourceShardAddResponse + * @memberof vtctldata.TabletExternallyReparentedResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SourceShardAddResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + TabletExternallyReparentedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SourceShardAddResponse"; + return typeUrlPrefix + "/vtctldata.TabletExternallyReparentedResponse"; }; - return SourceShardAddResponse; + return TabletExternallyReparentedResponse; })(); - vtctldata.SourceShardDeleteRequest = (function() { + vtctldata.UpdateCellInfoRequest = (function() { /** - * Properties of a SourceShardDeleteRequest. + * Properties of an UpdateCellInfoRequest. * @memberof vtctldata - * @interface ISourceShardDeleteRequest - * @property {string|null} [keyspace] SourceShardDeleteRequest keyspace - * @property {string|null} [shard] SourceShardDeleteRequest shard - * @property {number|null} [uid] SourceShardDeleteRequest uid + * @interface IUpdateCellInfoRequest + * @property {string|null} [name] UpdateCellInfoRequest name + * @property {topodata.ICellInfo|null} [cell_info] UpdateCellInfoRequest cell_info */ /** - * Constructs a new SourceShardDeleteRequest. + * Constructs a new UpdateCellInfoRequest. * @memberof vtctldata - * @classdesc Represents a SourceShardDeleteRequest. - * @implements ISourceShardDeleteRequest + * @classdesc Represents an UpdateCellInfoRequest. + * @implements IUpdateCellInfoRequest * @constructor - * @param {vtctldata.ISourceShardDeleteRequest=} [properties] Properties to set + * @param {vtctldata.IUpdateCellInfoRequest=} [properties] Properties to set */ - function SourceShardDeleteRequest(properties) { + function UpdateCellInfoRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -136982,103 +143444,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SourceShardDeleteRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.SourceShardDeleteRequest - * @instance - */ - SourceShardDeleteRequest.prototype.keyspace = ""; - - /** - * SourceShardDeleteRequest shard. - * @member {string} shard - * @memberof vtctldata.SourceShardDeleteRequest + * UpdateCellInfoRequest name. + * @member {string} name + * @memberof vtctldata.UpdateCellInfoRequest * @instance */ - SourceShardDeleteRequest.prototype.shard = ""; + UpdateCellInfoRequest.prototype.name = ""; /** - * SourceShardDeleteRequest uid. - * @member {number} uid - * @memberof vtctldata.SourceShardDeleteRequest + * UpdateCellInfoRequest cell_info. + * @member {topodata.ICellInfo|null|undefined} cell_info + * @memberof vtctldata.UpdateCellInfoRequest * @instance */ - SourceShardDeleteRequest.prototype.uid = 0; + UpdateCellInfoRequest.prototype.cell_info = null; /** - * Creates a new SourceShardDeleteRequest instance using the specified properties. + * Creates a new UpdateCellInfoRequest instance using the specified properties. * @function create - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static - * @param {vtctldata.ISourceShardDeleteRequest=} [properties] Properties to set - * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest instance + * @param {vtctldata.IUpdateCellInfoRequest=} [properties] Properties to set + * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest instance */ - SourceShardDeleteRequest.create = function create(properties) { - return new SourceShardDeleteRequest(properties); + UpdateCellInfoRequest.create = function create(properties) { + return new UpdateCellInfoRequest(properties); }; /** - * Encodes the specified SourceShardDeleteRequest message. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. + * Encodes the specified UpdateCellInfoRequest message. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static - * @param {vtctldata.ISourceShardDeleteRequest} message SourceShardDeleteRequest message or plain object to encode + * @param {vtctldata.IUpdateCellInfoRequest} message UpdateCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardDeleteRequest.encode = function encode(message, writer) { + UpdateCellInfoRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.uid != null && Object.hasOwnProperty.call(message, "uid")) - writer.uint32(/* id 3, wireType 0 =*/24).int32(message.uid); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) + $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified SourceShardDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteRequest.verify|verify} messages. + * Encodes the specified UpdateCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static - * @param {vtctldata.ISourceShardDeleteRequest} message SourceShardDeleteRequest message or plain object to encode + * @param {vtctldata.IUpdateCellInfoRequest} message UpdateCellInfoRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardDeleteRequest.encodeDelimited = function encodeDelimited(message, writer) { + UpdateCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SourceShardDeleteRequest message from the specified reader or buffer. + * Decodes an UpdateCellInfoRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest + * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardDeleteRequest.decode = function decode(reader, length) { + UpdateCellInfoRequest.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.SourceShardDeleteRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateCellInfoRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.name = reader.string(); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.uid = reader.int32(); + message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); break; } default: @@ -137090,139 +143538,137 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SourceShardDeleteRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellInfoRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest + * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardDeleteRequest.decodeDelimited = function decodeDelimited(reader) { + UpdateCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SourceShardDeleteRequest message. + * Verifies an UpdateCellInfoRequest message. * @function verify - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SourceShardDeleteRequest.verify = function verify(message) { + UpdateCellInfoRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.uid != null && message.hasOwnProperty("uid")) - if (!$util.isInteger(message.uid)) - return "uid: integer expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) { + let error = $root.topodata.CellInfo.verify(message.cell_info); + if (error) + return "cell_info." + error; + } return null; }; /** - * Creates a SourceShardDeleteRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellInfoRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.SourceShardDeleteRequest} SourceShardDeleteRequest + * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest */ - SourceShardDeleteRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SourceShardDeleteRequest) + UpdateCellInfoRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateCellInfoRequest) return object; - let message = new $root.vtctldata.SourceShardDeleteRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.uid != null) - message.uid = object.uid | 0; + let message = new $root.vtctldata.UpdateCellInfoRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.cell_info != null) { + if (typeof object.cell_info !== "object") + throw TypeError(".vtctldata.UpdateCellInfoRequest.cell_info: object expected"); + message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + } return message; }; /** - * Creates a plain object from a SourceShardDeleteRequest message. Also converts values to other types if specified. + * Creates a plain object from an UpdateCellInfoRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static - * @param {vtctldata.SourceShardDeleteRequest} message SourceShardDeleteRequest + * @param {vtctldata.UpdateCellInfoRequest} message UpdateCellInfoRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SourceShardDeleteRequest.toObject = function toObject(message, options) { + UpdateCellInfoRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.uid = 0; + object.name = ""; + object.cell_info = null; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.uid != null && message.hasOwnProperty("uid")) - object.uid = message.uid; + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) + object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); return object; }; /** - * Converts this SourceShardDeleteRequest to JSON. + * Converts this UpdateCellInfoRequest to JSON. * @function toJSON - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @instance * @returns {Object.} JSON object */ - SourceShardDeleteRequest.prototype.toJSON = function toJSON() { + UpdateCellInfoRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SourceShardDeleteRequest + * Gets the default type url for UpdateCellInfoRequest * @function getTypeUrl - * @memberof vtctldata.SourceShardDeleteRequest + * @memberof vtctldata.UpdateCellInfoRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SourceShardDeleteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SourceShardDeleteRequest"; + return typeUrlPrefix + "/vtctldata.UpdateCellInfoRequest"; }; - return SourceShardDeleteRequest; + return UpdateCellInfoRequest; })(); - vtctldata.SourceShardDeleteResponse = (function() { + vtctldata.UpdateCellInfoResponse = (function() { /** - * Properties of a SourceShardDeleteResponse. + * Properties of an UpdateCellInfoResponse. * @memberof vtctldata - * @interface ISourceShardDeleteResponse - * @property {topodata.IShard|null} [shard] SourceShardDeleteResponse shard + * @interface IUpdateCellInfoResponse + * @property {string|null} [name] UpdateCellInfoResponse name + * @property {topodata.ICellInfo|null} [cell_info] UpdateCellInfoResponse cell_info */ /** - * Constructs a new SourceShardDeleteResponse. + * Constructs a new UpdateCellInfoResponse. * @memberof vtctldata - * @classdesc Represents a SourceShardDeleteResponse. - * @implements ISourceShardDeleteResponse + * @classdesc Represents an UpdateCellInfoResponse. + * @implements IUpdateCellInfoResponse * @constructor - * @param {vtctldata.ISourceShardDeleteResponse=} [properties] Properties to set + * @param {vtctldata.IUpdateCellInfoResponse=} [properties] Properties to set */ - function SourceShardDeleteResponse(properties) { + function UpdateCellInfoResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -137230,75 +143676,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * SourceShardDeleteResponse shard. - * @member {topodata.IShard|null|undefined} shard - * @memberof vtctldata.SourceShardDeleteResponse + * UpdateCellInfoResponse name. + * @member {string} name + * @memberof vtctldata.UpdateCellInfoResponse * @instance */ - SourceShardDeleteResponse.prototype.shard = null; + UpdateCellInfoResponse.prototype.name = ""; /** - * Creates a new SourceShardDeleteResponse instance using the specified properties. + * UpdateCellInfoResponse cell_info. + * @member {topodata.ICellInfo|null|undefined} cell_info + * @memberof vtctldata.UpdateCellInfoResponse + * @instance + */ + UpdateCellInfoResponse.prototype.cell_info = null; + + /** + * Creates a new UpdateCellInfoResponse instance using the specified properties. * @function create - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static - * @param {vtctldata.ISourceShardDeleteResponse=} [properties] Properties to set - * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse instance + * @param {vtctldata.IUpdateCellInfoResponse=} [properties] Properties to set + * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse instance */ - SourceShardDeleteResponse.create = function create(properties) { - return new SourceShardDeleteResponse(properties); + UpdateCellInfoResponse.create = function create(properties) { + return new UpdateCellInfoResponse(properties); }; /** - * Encodes the specified SourceShardDeleteResponse message. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. + * Encodes the specified UpdateCellInfoResponse message. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static - * @param {vtctldata.ISourceShardDeleteResponse} message SourceShardDeleteResponse message or plain object to encode + * @param {vtctldata.IUpdateCellInfoResponse} message UpdateCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardDeleteResponse.encode = function encode(message, writer) { + UpdateCellInfoResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - $root.topodata.Shard.encode(message.shard, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) + $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified SourceShardDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.SourceShardDeleteResponse.verify|verify} messages. + * Encodes the specified UpdateCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static - * @param {vtctldata.ISourceShardDeleteResponse} message SourceShardDeleteResponse message or plain object to encode + * @param {vtctldata.IUpdateCellInfoResponse} message UpdateCellInfoResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - SourceShardDeleteResponse.encodeDelimited = function encodeDelimited(message, writer) { + UpdateCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a SourceShardDeleteResponse message from the specified reader or buffer. + * Decodes an UpdateCellInfoResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse + * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardDeleteResponse.decode = function decode(reader, length) { + UpdateCellInfoResponse.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.SourceShardDeleteResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateCellInfoResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.shard = $root.topodata.Shard.decode(reader, reader.uint32()); + message.name = reader.string(); + break; + } + case 2: { + message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); break; } default: @@ -137310,127 +143770,137 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a SourceShardDeleteResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellInfoResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse + * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - SourceShardDeleteResponse.decodeDelimited = function decodeDelimited(reader) { + UpdateCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a SourceShardDeleteResponse message. + * Verifies an UpdateCellInfoResponse message. * @function verify - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - SourceShardDeleteResponse.verify = function verify(message) { + UpdateCellInfoResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.shard != null && message.hasOwnProperty("shard")) { - let error = $root.topodata.Shard.verify(message.shard); + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) { + let error = $root.topodata.CellInfo.verify(message.cell_info); if (error) - return "shard." + error; + return "cell_info." + error; } return null; }; /** - * Creates a SourceShardDeleteResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellInfoResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.SourceShardDeleteResponse} SourceShardDeleteResponse + * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse */ - SourceShardDeleteResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.SourceShardDeleteResponse) + UpdateCellInfoResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateCellInfoResponse) return object; - let message = new $root.vtctldata.SourceShardDeleteResponse(); - if (object.shard != null) { - if (typeof object.shard !== "object") - throw TypeError(".vtctldata.SourceShardDeleteResponse.shard: object expected"); - message.shard = $root.topodata.Shard.fromObject(object.shard); + let message = new $root.vtctldata.UpdateCellInfoResponse(); + if (object.name != null) + message.name = String(object.name); + if (object.cell_info != null) { + if (typeof object.cell_info !== "object") + throw TypeError(".vtctldata.UpdateCellInfoResponse.cell_info: object expected"); + message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); } return message; }; /** - * Creates a plain object from a SourceShardDeleteResponse message. Also converts values to other types if specified. + * Creates a plain object from an UpdateCellInfoResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static - * @param {vtctldata.SourceShardDeleteResponse} message SourceShardDeleteResponse + * @param {vtctldata.UpdateCellInfoResponse} message UpdateCellInfoResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - SourceShardDeleteResponse.toObject = function toObject(message, options) { + UpdateCellInfoResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.shard = null; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = $root.topodata.Shard.toObject(message.shard, options); + if (options.defaults) { + object.name = ""; + object.cell_info = null; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cell_info != null && message.hasOwnProperty("cell_info")) + object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); return object; }; /** - * Converts this SourceShardDeleteResponse to JSON. + * Converts this UpdateCellInfoResponse to JSON. * @function toJSON - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @instance * @returns {Object.} JSON object */ - SourceShardDeleteResponse.prototype.toJSON = function toJSON() { + UpdateCellInfoResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for SourceShardDeleteResponse + * Gets the default type url for UpdateCellInfoResponse * @function getTypeUrl - * @memberof vtctldata.SourceShardDeleteResponse + * @memberof vtctldata.UpdateCellInfoResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - SourceShardDeleteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.SourceShardDeleteResponse"; + return typeUrlPrefix + "/vtctldata.UpdateCellInfoResponse"; }; - return SourceShardDeleteResponse; + return UpdateCellInfoResponse; })(); - vtctldata.StartReplicationRequest = (function() { + vtctldata.UpdateCellsAliasRequest = (function() { /** - * Properties of a StartReplicationRequest. + * Properties of an UpdateCellsAliasRequest. * @memberof vtctldata - * @interface IStartReplicationRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] StartReplicationRequest tablet_alias + * @interface IUpdateCellsAliasRequest + * @property {string|null} [name] UpdateCellsAliasRequest name + * @property {topodata.ICellsAlias|null} [cells_alias] UpdateCellsAliasRequest cells_alias */ /** - * Constructs a new StartReplicationRequest. + * Constructs a new UpdateCellsAliasRequest. * @memberof vtctldata - * @classdesc Represents a StartReplicationRequest. - * @implements IStartReplicationRequest + * @classdesc Represents an UpdateCellsAliasRequest. + * @implements IUpdateCellsAliasRequest * @constructor - * @param {vtctldata.IStartReplicationRequest=} [properties] Properties to set + * @param {vtctldata.IUpdateCellsAliasRequest=} [properties] Properties to set */ - function StartReplicationRequest(properties) { + function UpdateCellsAliasRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -137438,75 +143908,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * StartReplicationRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.StartReplicationRequest + * UpdateCellsAliasRequest name. + * @member {string} name + * @memberof vtctldata.UpdateCellsAliasRequest * @instance */ - StartReplicationRequest.prototype.tablet_alias = null; + UpdateCellsAliasRequest.prototype.name = ""; /** - * Creates a new StartReplicationRequest instance using the specified properties. + * UpdateCellsAliasRequest cells_alias. + * @member {topodata.ICellsAlias|null|undefined} cells_alias + * @memberof vtctldata.UpdateCellsAliasRequest + * @instance + */ + UpdateCellsAliasRequest.prototype.cells_alias = null; + + /** + * Creates a new UpdateCellsAliasRequest instance using the specified properties. * @function create - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static - * @param {vtctldata.IStartReplicationRequest=} [properties] Properties to set - * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest instance + * @param {vtctldata.IUpdateCellsAliasRequest=} [properties] Properties to set + * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest instance */ - StartReplicationRequest.create = function create(properties) { - return new StartReplicationRequest(properties); + UpdateCellsAliasRequest.create = function create(properties) { + return new UpdateCellsAliasRequest(properties); }; /** - * Encodes the specified StartReplicationRequest message. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. + * Encodes the specified UpdateCellsAliasRequest message. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static - * @param {vtctldata.IStartReplicationRequest} message StartReplicationRequest message or plain object to encode + * @param {vtctldata.IUpdateCellsAliasRequest} message UpdateCellsAliasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartReplicationRequest.encode = function encode(message, writer) { + UpdateCellsAliasRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cells_alias != null && Object.hasOwnProperty.call(message, "cells_alias")) + $root.topodata.CellsAlias.encode(message.cells_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified StartReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StartReplicationRequest.verify|verify} messages. + * Encodes the specified UpdateCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static - * @param {vtctldata.IStartReplicationRequest} message StartReplicationRequest message or plain object to encode + * @param {vtctldata.IUpdateCellsAliasRequest} message UpdateCellsAliasRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartReplicationRequest.encodeDelimited = function encodeDelimited(message, writer) { + UpdateCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StartReplicationRequest message from the specified reader or buffer. + * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest + * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartReplicationRequest.decode = function decode(reader, length) { + UpdateCellsAliasRequest.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.StartReplicationRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateCellsAliasRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.name = reader.string(); + break; + } + case 2: { + message.cells_alias = $root.topodata.CellsAlias.decode(reader, reader.uint32()); break; } default: @@ -137518,126 +144002,137 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a StartReplicationRequest message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest + * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartReplicationRequest.decodeDelimited = function decodeDelimited(reader) { + UpdateCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StartReplicationRequest message. + * Verifies an UpdateCellsAliasRequest message. * @function verify - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StartReplicationRequest.verify = function verify(message) { + UpdateCellsAliasRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) { + let error = $root.topodata.CellsAlias.verify(message.cells_alias); if (error) - return "tablet_alias." + error; + return "cells_alias." + error; } return null; }; /** - * Creates a StartReplicationRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellsAliasRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.StartReplicationRequest} StartReplicationRequest + * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest */ - StartReplicationRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.StartReplicationRequest) + UpdateCellsAliasRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateCellsAliasRequest) return object; - let message = new $root.vtctldata.StartReplicationRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.StartReplicationRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); + let message = new $root.vtctldata.UpdateCellsAliasRequest(); + if (object.name != null) + message.name = String(object.name); + if (object.cells_alias != null) { + if (typeof object.cells_alias !== "object") + throw TypeError(".vtctldata.UpdateCellsAliasRequest.cells_alias: object expected"); + message.cells_alias = $root.topodata.CellsAlias.fromObject(object.cells_alias); } return message; }; /** - * Creates a plain object from a StartReplicationRequest message. Also converts values to other types if specified. + * Creates a plain object from an UpdateCellsAliasRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static - * @param {vtctldata.StartReplicationRequest} message StartReplicationRequest + * @param {vtctldata.UpdateCellsAliasRequest} message UpdateCellsAliasRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StartReplicationRequest.toObject = function toObject(message, options) { + UpdateCellsAliasRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + if (options.defaults) { + object.name = ""; + object.cells_alias = null; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) + object.cells_alias = $root.topodata.CellsAlias.toObject(message.cells_alias, options); return object; }; /** - * Converts this StartReplicationRequest to JSON. + * Converts this UpdateCellsAliasRequest to JSON. * @function toJSON - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @instance * @returns {Object.} JSON object */ - StartReplicationRequest.prototype.toJSON = function toJSON() { + UpdateCellsAliasRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StartReplicationRequest + * Gets the default type url for UpdateCellsAliasRequest * @function getTypeUrl - * @memberof vtctldata.StartReplicationRequest + * @memberof vtctldata.UpdateCellsAliasRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StartReplicationRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.StartReplicationRequest"; + return typeUrlPrefix + "/vtctldata.UpdateCellsAliasRequest"; }; - return StartReplicationRequest; + return UpdateCellsAliasRequest; })(); - vtctldata.StartReplicationResponse = (function() { + vtctldata.UpdateCellsAliasResponse = (function() { /** - * Properties of a StartReplicationResponse. + * Properties of an UpdateCellsAliasResponse. * @memberof vtctldata - * @interface IStartReplicationResponse + * @interface IUpdateCellsAliasResponse + * @property {string|null} [name] UpdateCellsAliasResponse name + * @property {topodata.ICellsAlias|null} [cells_alias] UpdateCellsAliasResponse cells_alias */ /** - * Constructs a new StartReplicationResponse. + * Constructs a new UpdateCellsAliasResponse. * @memberof vtctldata - * @classdesc Represents a StartReplicationResponse. - * @implements IStartReplicationResponse + * @classdesc Represents an UpdateCellsAliasResponse. + * @implements IUpdateCellsAliasResponse * @constructor - * @param {vtctldata.IStartReplicationResponse=} [properties] Properties to set + * @param {vtctldata.IUpdateCellsAliasResponse=} [properties] Properties to set */ - function StartReplicationResponse(properties) { + function UpdateCellsAliasResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -137645,63 +144140,91 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new StartReplicationResponse instance using the specified properties. + * UpdateCellsAliasResponse name. + * @member {string} name + * @memberof vtctldata.UpdateCellsAliasResponse + * @instance + */ + UpdateCellsAliasResponse.prototype.name = ""; + + /** + * UpdateCellsAliasResponse cells_alias. + * @member {topodata.ICellsAlias|null|undefined} cells_alias + * @memberof vtctldata.UpdateCellsAliasResponse + * @instance + */ + UpdateCellsAliasResponse.prototype.cells_alias = null; + + /** + * Creates a new UpdateCellsAliasResponse instance using the specified properties. * @function create - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static - * @param {vtctldata.IStartReplicationResponse=} [properties] Properties to set - * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse instance + * @param {vtctldata.IUpdateCellsAliasResponse=} [properties] Properties to set + * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse instance */ - StartReplicationResponse.create = function create(properties) { - return new StartReplicationResponse(properties); + UpdateCellsAliasResponse.create = function create(properties) { + return new UpdateCellsAliasResponse(properties); }; /** - * Encodes the specified StartReplicationResponse message. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. + * Encodes the specified UpdateCellsAliasResponse message. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static - * @param {vtctldata.IStartReplicationResponse} message StartReplicationResponse message or plain object to encode + * @param {vtctldata.IUpdateCellsAliasResponse} message UpdateCellsAliasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartReplicationResponse.encode = function encode(message, writer) { + UpdateCellsAliasResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.name != null && Object.hasOwnProperty.call(message, "name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); + if (message.cells_alias != null && Object.hasOwnProperty.call(message, "cells_alias")) + $root.topodata.CellsAlias.encode(message.cells_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified StartReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StartReplicationResponse.verify|verify} messages. + * Encodes the specified UpdateCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static - * @param {vtctldata.IStartReplicationResponse} message StartReplicationResponse message or plain object to encode + * @param {vtctldata.IUpdateCellsAliasResponse} message UpdateCellsAliasResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StartReplicationResponse.encodeDelimited = function encodeDelimited(message, writer) { + UpdateCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StartReplicationResponse message from the specified reader or buffer. + * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse + * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartReplicationResponse.decode = function decode(reader, length) { + UpdateCellsAliasResponse.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.StartReplicationResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.UpdateCellsAliasResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + message.name = reader.string(); + break; + } + case 2: { + message.cells_alias = $root.topodata.CellsAlias.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -137711,109 +144234,136 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a StartReplicationResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse + * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StartReplicationResponse.decodeDelimited = function decodeDelimited(reader) { + UpdateCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StartReplicationResponse message. + * Verifies an UpdateCellsAliasResponse message. * @function verify - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StartReplicationResponse.verify = function verify(message) { + UpdateCellsAliasResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.name != null && message.hasOwnProperty("name")) + if (!$util.isString(message.name)) + return "name: string expected"; + if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) { + let error = $root.topodata.CellsAlias.verify(message.cells_alias); + if (error) + return "cells_alias." + error; + } return null; }; /** - * Creates a StartReplicationResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateCellsAliasResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.StartReplicationResponse} StartReplicationResponse + * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse */ - StartReplicationResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.StartReplicationResponse) + UpdateCellsAliasResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.UpdateCellsAliasResponse) return object; - return new $root.vtctldata.StartReplicationResponse(); + let message = new $root.vtctldata.UpdateCellsAliasResponse(); + if (object.name != null) + message.name = String(object.name); + if (object.cells_alias != null) { + if (typeof object.cells_alias !== "object") + throw TypeError(".vtctldata.UpdateCellsAliasResponse.cells_alias: object expected"); + message.cells_alias = $root.topodata.CellsAlias.fromObject(object.cells_alias); + } + return message; }; /** - * Creates a plain object from a StartReplicationResponse message. Also converts values to other types if specified. + * Creates a plain object from an UpdateCellsAliasResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static - * @param {vtctldata.StartReplicationResponse} message StartReplicationResponse + * @param {vtctldata.UpdateCellsAliasResponse} message UpdateCellsAliasResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StartReplicationResponse.toObject = function toObject() { - return {}; + UpdateCellsAliasResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.name = ""; + object.cells_alias = null; + } + if (message.name != null && message.hasOwnProperty("name")) + object.name = message.name; + if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) + object.cells_alias = $root.topodata.CellsAlias.toObject(message.cells_alias, options); + return object; }; /** - * Converts this StartReplicationResponse to JSON. + * Converts this UpdateCellsAliasResponse to JSON. * @function toJSON - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @instance * @returns {Object.} JSON object */ - StartReplicationResponse.prototype.toJSON = function toJSON() { + UpdateCellsAliasResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StartReplicationResponse + * Gets the default type url for UpdateCellsAliasResponse * @function getTypeUrl - * @memberof vtctldata.StartReplicationResponse + * @memberof vtctldata.UpdateCellsAliasResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StartReplicationResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.StartReplicationResponse"; + return typeUrlPrefix + "/vtctldata.UpdateCellsAliasResponse"; }; - return StartReplicationResponse; + return UpdateCellsAliasResponse; })(); - vtctldata.StopReplicationRequest = (function() { + vtctldata.ValidateRequest = (function() { /** - * Properties of a StopReplicationRequest. + * Properties of a ValidateRequest. * @memberof vtctldata - * @interface IStopReplicationRequest - * @property {topodata.ITabletAlias|null} [tablet_alias] StopReplicationRequest tablet_alias + * @interface IValidateRequest + * @property {boolean|null} [ping_tablets] ValidateRequest ping_tablets */ /** - * Constructs a new StopReplicationRequest. + * Constructs a new ValidateRequest. * @memberof vtctldata - * @classdesc Represents a StopReplicationRequest. - * @implements IStopReplicationRequest + * @classdesc Represents a ValidateRequest. + * @implements IValidateRequest * @constructor - * @param {vtctldata.IStopReplicationRequest=} [properties] Properties to set + * @param {vtctldata.IValidateRequest=} [properties] Properties to set */ - function StopReplicationRequest(properties) { + function ValidateRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -137821,75 +144371,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * StopReplicationRequest tablet_alias. - * @member {topodata.ITabletAlias|null|undefined} tablet_alias - * @memberof vtctldata.StopReplicationRequest + * ValidateRequest ping_tablets. + * @member {boolean} ping_tablets + * @memberof vtctldata.ValidateRequest * @instance */ - StopReplicationRequest.prototype.tablet_alias = null; + ValidateRequest.prototype.ping_tablets = false; /** - * Creates a new StopReplicationRequest instance using the specified properties. + * Creates a new ValidateRequest instance using the specified properties. * @function create - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static - * @param {vtctldata.IStopReplicationRequest=} [properties] Properties to set - * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest instance + * @param {vtctldata.IValidateRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateRequest} ValidateRequest instance */ - StopReplicationRequest.create = function create(properties) { - return new StopReplicationRequest(properties); + ValidateRequest.create = function create(properties) { + return new ValidateRequest(properties); }; /** - * Encodes the specified StopReplicationRequest message. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. + * Encodes the specified ValidateRequest message. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static - * @param {vtctldata.IStopReplicationRequest} message StopReplicationRequest message or plain object to encode + * @param {vtctldata.IValidateRequest} message ValidateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationRequest.encode = function encode(message, writer) { + ValidateRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_alias != null && Object.hasOwnProperty.call(message, "tablet_alias")) - $root.topodata.TabletAlias.encode(message.tablet_alias, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.ping_tablets); return writer; }; /** - * Encodes the specified StopReplicationRequest message, length delimited. Does not implicitly {@link vtctldata.StopReplicationRequest.verify|verify} messages. + * Encodes the specified ValidateRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static - * @param {vtctldata.IStopReplicationRequest} message StopReplicationRequest message or plain object to encode + * @param {vtctldata.IValidateRequest} message ValidateRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StopReplicationRequest message from the specified reader or buffer. + * Decodes a ValidateRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest + * @returns {vtctldata.ValidateRequest} ValidateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationRequest.decode = function decode(reader, length) { + ValidateRequest.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.StopReplicationRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_alias = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.ping_tablets = reader.bool(); break; } default: @@ -137901,126 +144451,125 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a StopReplicationRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest + * @returns {vtctldata.ValidateRequest} ValidateRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StopReplicationRequest message. + * Verifies a ValidateRequest message. * @function verify - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StopReplicationRequest.verify = function verify(message) { + ValidateRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) { - let error = $root.topodata.TabletAlias.verify(message.tablet_alias); - if (error) - return "tablet_alias." + error; - } + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + if (typeof message.ping_tablets !== "boolean") + return "ping_tablets: boolean expected"; return null; }; /** - * Creates a StopReplicationRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.StopReplicationRequest} StopReplicationRequest + * @returns {vtctldata.ValidateRequest} ValidateRequest */ - StopReplicationRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.StopReplicationRequest) + ValidateRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateRequest) return object; - let message = new $root.vtctldata.StopReplicationRequest(); - if (object.tablet_alias != null) { - if (typeof object.tablet_alias !== "object") - throw TypeError(".vtctldata.StopReplicationRequest.tablet_alias: object expected"); - message.tablet_alias = $root.topodata.TabletAlias.fromObject(object.tablet_alias); - } + let message = new $root.vtctldata.ValidateRequest(); + if (object.ping_tablets != null) + message.ping_tablets = Boolean(object.ping_tablets); return message; }; /** - * Creates a plain object from a StopReplicationRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static - * @param {vtctldata.StopReplicationRequest} message StopReplicationRequest + * @param {vtctldata.ValidateRequest} message ValidateRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StopReplicationRequest.toObject = function toObject(message, options) { + ValidateRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.tablet_alias = null; - if (message.tablet_alias != null && message.hasOwnProperty("tablet_alias")) - object.tablet_alias = $root.topodata.TabletAlias.toObject(message.tablet_alias, options); + object.ping_tablets = false; + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + object.ping_tablets = message.ping_tablets; return object; }; /** - * Converts this StopReplicationRequest to JSON. + * Converts this ValidateRequest to JSON. * @function toJSON - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @instance * @returns {Object.} JSON object */ - StopReplicationRequest.prototype.toJSON = function toJSON() { + ValidateRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StopReplicationRequest + * Gets the default type url for ValidateRequest * @function getTypeUrl - * @memberof vtctldata.StopReplicationRequest + * @memberof vtctldata.ValidateRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StopReplicationRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.StopReplicationRequest"; + return typeUrlPrefix + "/vtctldata.ValidateRequest"; }; - return StopReplicationRequest; + return ValidateRequest; })(); - vtctldata.StopReplicationResponse = (function() { + vtctldata.ValidateResponse = (function() { /** - * Properties of a StopReplicationResponse. + * Properties of a ValidateResponse. * @memberof vtctldata - * @interface IStopReplicationResponse + * @interface IValidateResponse + * @property {Array.|null} [results] ValidateResponse results + * @property {Object.|null} [results_by_keyspace] ValidateResponse results_by_keyspace */ /** - * Constructs a new StopReplicationResponse. + * Constructs a new ValidateResponse. * @memberof vtctldata - * @classdesc Represents a StopReplicationResponse. - * @implements IStopReplicationResponse + * @classdesc Represents a ValidateResponse. + * @implements IValidateResponse * @constructor - * @param {vtctldata.IStopReplicationResponse=} [properties] Properties to set + * @param {vtctldata.IValidateResponse=} [properties] Properties to set */ - function StopReplicationResponse(properties) { + function ValidateResponse(properties) { + this.results = []; + this.results_by_keyspace = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -138028,63 +144577,116 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * Creates a new StopReplicationResponse instance using the specified properties. + * ValidateResponse results. + * @member {Array.} results + * @memberof vtctldata.ValidateResponse + * @instance + */ + ValidateResponse.prototype.results = $util.emptyArray; + + /** + * ValidateResponse results_by_keyspace. + * @member {Object.} results_by_keyspace + * @memberof vtctldata.ValidateResponse + * @instance + */ + ValidateResponse.prototype.results_by_keyspace = $util.emptyObject; + + /** + * Creates a new ValidateResponse instance using the specified properties. * @function create - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static - * @param {vtctldata.IStopReplicationResponse=} [properties] Properties to set - * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse instance + * @param {vtctldata.IValidateResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateResponse} ValidateResponse instance */ - StopReplicationResponse.create = function create(properties) { - return new StopReplicationResponse(properties); + ValidateResponse.create = function create(properties) { + return new ValidateResponse(properties); }; /** - * Encodes the specified StopReplicationResponse message. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. + * Encodes the specified ValidateResponse message. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static - * @param {vtctldata.IStopReplicationResponse} message StopReplicationResponse message or plain object to encode + * @param {vtctldata.IValidateResponse} message ValidateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationResponse.encode = function encode(message, writer) { + ValidateResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); + if (message.results != null && message.results.length) + for (let i = 0; i < message.results.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); + if (message.results_by_keyspace != null && Object.hasOwnProperty.call(message, "results_by_keyspace")) + for (let keys = Object.keys(message.results_by_keyspace), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.ValidateKeyspaceResponse.encode(message.results_by_keyspace[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified StopReplicationResponse message, length delimited. Does not implicitly {@link vtctldata.StopReplicationResponse.verify|verify} messages. + * Encodes the specified ValidateResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static - * @param {vtctldata.IStopReplicationResponse} message StopReplicationResponse message or plain object to encode + * @param {vtctldata.IValidateResponse} message ValidateResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - StopReplicationResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a StopReplicationResponse message from the specified reader or buffer. + * Decodes a ValidateResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse + * @returns {vtctldata.ValidateResponse} ValidateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationResponse.decode = function decode(reader, length) { + ValidateResponse.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.StopReplicationResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { + case 1: { + if (!(message.results && message.results.length)) + message.results = []; + message.results.push(reader.string()); + break; + } + case 2: { + if (message.results_by_keyspace === $util.emptyObject) + message.results_by_keyspace = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.ValidateKeyspaceResponse.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.results_by_keyspace[key] = value; + break; + } default: reader.skipType(tag & 7); break; @@ -138094,109 +144696,163 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a StopReplicationResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse + * @returns {vtctldata.ValidateResponse} ValidateResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - StopReplicationResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a StopReplicationResponse message. + * Verifies a ValidateResponse message. * @function verify - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - StopReplicationResponse.verify = function verify(message) { + ValidateResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; + if (message.results != null && message.hasOwnProperty("results")) { + if (!Array.isArray(message.results)) + return "results: array expected"; + for (let i = 0; i < message.results.length; ++i) + if (!$util.isString(message.results[i])) + return "results: string[] expected"; + } + if (message.results_by_keyspace != null && message.hasOwnProperty("results_by_keyspace")) { + if (!$util.isObject(message.results_by_keyspace)) + return "results_by_keyspace: object expected"; + let key = Object.keys(message.results_by_keyspace); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.ValidateKeyspaceResponse.verify(message.results_by_keyspace[key[i]]); + if (error) + return "results_by_keyspace." + error; + } + } return null; }; /** - * Creates a StopReplicationResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.StopReplicationResponse} StopReplicationResponse + * @returns {vtctldata.ValidateResponse} ValidateResponse */ - StopReplicationResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.StopReplicationResponse) + ValidateResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateResponse) return object; - return new $root.vtctldata.StopReplicationResponse(); + let message = new $root.vtctldata.ValidateResponse(); + if (object.results) { + if (!Array.isArray(object.results)) + throw TypeError(".vtctldata.ValidateResponse.results: array expected"); + message.results = []; + for (let i = 0; i < object.results.length; ++i) + message.results[i] = String(object.results[i]); + } + if (object.results_by_keyspace) { + if (typeof object.results_by_keyspace !== "object") + throw TypeError(".vtctldata.ValidateResponse.results_by_keyspace: object expected"); + message.results_by_keyspace = {}; + for (let keys = Object.keys(object.results_by_keyspace), i = 0; i < keys.length; ++i) { + if (typeof object.results_by_keyspace[keys[i]] !== "object") + throw TypeError(".vtctldata.ValidateResponse.results_by_keyspace: object expected"); + message.results_by_keyspace[keys[i]] = $root.vtctldata.ValidateKeyspaceResponse.fromObject(object.results_by_keyspace[keys[i]]); + } + } + return message; }; /** - * Creates a plain object from a StopReplicationResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static - * @param {vtctldata.StopReplicationResponse} message StopReplicationResponse + * @param {vtctldata.ValidateResponse} message ValidateResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - StopReplicationResponse.toObject = function toObject() { - return {}; + ValidateResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.results = []; + if (options.objects || options.defaults) + object.results_by_keyspace = {}; + if (message.results && message.results.length) { + object.results = []; + for (let j = 0; j < message.results.length; ++j) + object.results[j] = message.results[j]; + } + let keys2; + if (message.results_by_keyspace && (keys2 = Object.keys(message.results_by_keyspace)).length) { + object.results_by_keyspace = {}; + for (let j = 0; j < keys2.length; ++j) + object.results_by_keyspace[keys2[j]] = $root.vtctldata.ValidateKeyspaceResponse.toObject(message.results_by_keyspace[keys2[j]], options); + } + return object; }; /** - * Converts this StopReplicationResponse to JSON. + * Converts this ValidateResponse to JSON. * @function toJSON - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @instance * @returns {Object.} JSON object */ - StopReplicationResponse.prototype.toJSON = function toJSON() { + ValidateResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for StopReplicationResponse + * Gets the default type url for ValidateResponse * @function getTypeUrl - * @memberof vtctldata.StopReplicationResponse + * @memberof vtctldata.ValidateResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - StopReplicationResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.StopReplicationResponse"; + return typeUrlPrefix + "/vtctldata.ValidateResponse"; }; - return StopReplicationResponse; + return ValidateResponse; })(); - vtctldata.TabletExternallyReparentedRequest = (function() { + vtctldata.ValidateKeyspaceRequest = (function() { /** - * Properties of a TabletExternallyReparentedRequest. + * Properties of a ValidateKeyspaceRequest. * @memberof vtctldata - * @interface ITabletExternallyReparentedRequest - * @property {topodata.ITabletAlias|null} [tablet] TabletExternallyReparentedRequest tablet + * @interface IValidateKeyspaceRequest + * @property {string|null} [keyspace] ValidateKeyspaceRequest keyspace + * @property {boolean|null} [ping_tablets] ValidateKeyspaceRequest ping_tablets */ /** - * Constructs a new TabletExternallyReparentedRequest. + * Constructs a new ValidateKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a TabletExternallyReparentedRequest. - * @implements ITabletExternallyReparentedRequest + * @classdesc Represents a ValidateKeyspaceRequest. + * @implements IValidateKeyspaceRequest * @constructor - * @param {vtctldata.ITabletExternallyReparentedRequest=} [properties] Properties to set + * @param {vtctldata.IValidateKeyspaceRequest=} [properties] Properties to set */ - function TabletExternallyReparentedRequest(properties) { + function ValidateKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -138204,75 +144860,89 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * TabletExternallyReparentedRequest tablet. - * @member {topodata.ITabletAlias|null|undefined} tablet - * @memberof vtctldata.TabletExternallyReparentedRequest + * ValidateKeyspaceRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ValidateKeyspaceRequest * @instance */ - TabletExternallyReparentedRequest.prototype.tablet = null; + ValidateKeyspaceRequest.prototype.keyspace = ""; /** - * Creates a new TabletExternallyReparentedRequest instance using the specified properties. + * ValidateKeyspaceRequest ping_tablets. + * @member {boolean} ping_tablets + * @memberof vtctldata.ValidateKeyspaceRequest + * @instance + */ + ValidateKeyspaceRequest.prototype.ping_tablets = false; + + /** + * Creates a new ValidateKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static - * @param {vtctldata.ITabletExternallyReparentedRequest=} [properties] Properties to set - * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest instance + * @param {vtctldata.IValidateKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest instance */ - TabletExternallyReparentedRequest.create = function create(properties) { - return new TabletExternallyReparentedRequest(properties); + ValidateKeyspaceRequest.create = function create(properties) { + return new ValidateKeyspaceRequest(properties); }; /** - * Encodes the specified TabletExternallyReparentedRequest message. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. + * Encodes the specified ValidateKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static - * @param {vtctldata.ITabletExternallyReparentedRequest} message TabletExternallyReparentedRequest message or plain object to encode + * @param {vtctldata.IValidateKeyspaceRequest} message ValidateKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TabletExternallyReparentedRequest.encode = function encode(message, writer) { + ValidateKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) - $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.ping_tablets); return writer; }; /** - * Encodes the specified TabletExternallyReparentedRequest message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedRequest.verify|verify} messages. + * Encodes the specified ValidateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static - * @param {vtctldata.ITabletExternallyReparentedRequest} message TabletExternallyReparentedRequest message or plain object to encode + * @param {vtctldata.IValidateKeyspaceRequest} message ValidateKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TabletExternallyReparentedRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer. + * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest + * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TabletExternallyReparentedRequest.decode = function decode(reader, length) { + ValidateKeyspaceRequest.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.TabletExternallyReparentedRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + message.keyspace = reader.string(); + break; + } + case 2: { + message.ping_tablets = reader.bool(); break; } default: @@ -138284,130 +144954,134 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a TabletExternallyReparentedRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest + * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TabletExternallyReparentedRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TabletExternallyReparentedRequest message. + * Verifies a ValidateKeyspaceRequest message. * @function verify - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TabletExternallyReparentedRequest.verify = function verify(message) { + ValidateKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet != null && message.hasOwnProperty("tablet")) { - let error = $root.topodata.TabletAlias.verify(message.tablet); - if (error) - return "tablet." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + if (typeof message.ping_tablets !== "boolean") + return "ping_tablets: boolean expected"; return null; }; /** - * Creates a TabletExternallyReparentedRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.TabletExternallyReparentedRequest} TabletExternallyReparentedRequest + * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest */ - TabletExternallyReparentedRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.TabletExternallyReparentedRequest) + ValidateKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateKeyspaceRequest) return object; - let message = new $root.vtctldata.TabletExternallyReparentedRequest(); - if (object.tablet != null) { - if (typeof object.tablet !== "object") - throw TypeError(".vtctldata.TabletExternallyReparentedRequest.tablet: object expected"); - message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); - } + let message = new $root.vtctldata.ValidateKeyspaceRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.ping_tablets != null) + message.ping_tablets = Boolean(object.ping_tablets); return message; }; /** - * Creates a plain object from a TabletExternallyReparentedRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static - * @param {vtctldata.TabletExternallyReparentedRequest} message TabletExternallyReparentedRequest + * @param {vtctldata.ValidateKeyspaceRequest} message ValidateKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TabletExternallyReparentedRequest.toObject = function toObject(message, options) { + ValidateKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) - object.tablet = null; - if (message.tablet != null && message.hasOwnProperty("tablet")) - object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (options.defaults) { + object.keyspace = ""; + object.ping_tablets = false; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + object.ping_tablets = message.ping_tablets; return object; }; /** - * Converts this TabletExternallyReparentedRequest to JSON. + * Converts this ValidateKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @instance * @returns {Object.} JSON object */ - TabletExternallyReparentedRequest.prototype.toJSON = function toJSON() { + ValidateKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TabletExternallyReparentedRequest + * Gets the default type url for ValidateKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.TabletExternallyReparentedRequest + * @memberof vtctldata.ValidateKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TabletExternallyReparentedRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.TabletExternallyReparentedRequest"; + return typeUrlPrefix + "/vtctldata.ValidateKeyspaceRequest"; }; - return TabletExternallyReparentedRequest; + return ValidateKeyspaceRequest; })(); - vtctldata.TabletExternallyReparentedResponse = (function() { + vtctldata.ValidateKeyspaceResponse = (function() { /** - * Properties of a TabletExternallyReparentedResponse. + * Properties of a ValidateKeyspaceResponse. * @memberof vtctldata - * @interface ITabletExternallyReparentedResponse - * @property {string|null} [keyspace] TabletExternallyReparentedResponse keyspace - * @property {string|null} [shard] TabletExternallyReparentedResponse shard - * @property {topodata.ITabletAlias|null} [new_primary] TabletExternallyReparentedResponse new_primary - * @property {topodata.ITabletAlias|null} [old_primary] TabletExternallyReparentedResponse old_primary + * @interface IValidateKeyspaceResponse + * @property {Array.|null} [results] ValidateKeyspaceResponse results + * @property {Object.|null} [results_by_shard] ValidateKeyspaceResponse results_by_shard */ /** - * Constructs a new TabletExternallyReparentedResponse. + * Constructs a new ValidateKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a TabletExternallyReparentedResponse. - * @implements ITabletExternallyReparentedResponse + * @classdesc Represents a ValidateKeyspaceResponse. + * @implements IValidateKeyspaceResponse * @constructor - * @param {vtctldata.ITabletExternallyReparentedResponse=} [properties] Properties to set + * @param {vtctldata.IValidateKeyspaceResponse=} [properties] Properties to set */ - function TabletExternallyReparentedResponse(properties) { + function ValidateKeyspaceResponse(properties) { + this.results = []; + this.results_by_shard = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -138415,117 +145089,114 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * TabletExternallyReparentedResponse keyspace. - * @member {string} keyspace - * @memberof vtctldata.TabletExternallyReparentedResponse - * @instance - */ - TabletExternallyReparentedResponse.prototype.keyspace = ""; - - /** - * TabletExternallyReparentedResponse shard. - * @member {string} shard - * @memberof vtctldata.TabletExternallyReparentedResponse - * @instance - */ - TabletExternallyReparentedResponse.prototype.shard = ""; - - /** - * TabletExternallyReparentedResponse new_primary. - * @member {topodata.ITabletAlias|null|undefined} new_primary - * @memberof vtctldata.TabletExternallyReparentedResponse + * ValidateKeyspaceResponse results. + * @member {Array.} results + * @memberof vtctldata.ValidateKeyspaceResponse * @instance */ - TabletExternallyReparentedResponse.prototype.new_primary = null; + ValidateKeyspaceResponse.prototype.results = $util.emptyArray; /** - * TabletExternallyReparentedResponse old_primary. - * @member {topodata.ITabletAlias|null|undefined} old_primary - * @memberof vtctldata.TabletExternallyReparentedResponse + * ValidateKeyspaceResponse results_by_shard. + * @member {Object.} results_by_shard + * @memberof vtctldata.ValidateKeyspaceResponse * @instance */ - TabletExternallyReparentedResponse.prototype.old_primary = null; + ValidateKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; /** - * Creates a new TabletExternallyReparentedResponse instance using the specified properties. + * Creates a new ValidateKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static - * @param {vtctldata.ITabletExternallyReparentedResponse=} [properties] Properties to set - * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse instance + * @param {vtctldata.IValidateKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse instance */ - TabletExternallyReparentedResponse.create = function create(properties) { - return new TabletExternallyReparentedResponse(properties); + ValidateKeyspaceResponse.create = function create(properties) { + return new ValidateKeyspaceResponse(properties); }; /** - * Encodes the specified TabletExternallyReparentedResponse message. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. + * Encodes the specified ValidateKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static - * @param {vtctldata.ITabletExternallyReparentedResponse} message TabletExternallyReparentedResponse message or plain object to encode + * @param {vtctldata.IValidateKeyspaceResponse} message ValidateKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TabletExternallyReparentedResponse.encode = function encode(message, writer) { + ValidateKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.new_primary != null && Object.hasOwnProperty.call(message, "new_primary")) - $root.topodata.TabletAlias.encode(message.new_primary, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); - if (message.old_primary != null && Object.hasOwnProperty.call(message, "old_primary")) - $root.topodata.TabletAlias.encode(message.old_primary, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.results != null && message.results.length) + for (let i = 0; i < message.results.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); + if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) + for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified TabletExternallyReparentedResponse message, length delimited. Does not implicitly {@link vtctldata.TabletExternallyReparentedResponse.verify|verify} messages. + * Encodes the specified ValidateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static - * @param {vtctldata.ITabletExternallyReparentedResponse} message TabletExternallyReparentedResponse message or plain object to encode + * @param {vtctldata.IValidateKeyspaceResponse} message ValidateKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - TabletExternallyReparentedResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer. + * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse + * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TabletExternallyReparentedResponse.decode = function decode(reader, length) { + ValidateKeyspaceResponse.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.TabletExternallyReparentedResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateKeyspaceResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + if (!(message.results && message.results.length)) + message.results = []; + message.results.push(reader.string()); break; } case 2: { - message.shard = reader.string(); - break; - } - case 3: { - message.new_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); - break; - } - case 4: { - message.old_primary = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + if (message.results_by_shard === $util.emptyObject) + message.results_by_shard = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.results_by_shard[key] = value; break; } default: @@ -138537,158 +145208,167 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a TabletExternallyReparentedResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse + * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - TabletExternallyReparentedResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a TabletExternallyReparentedResponse message. + * Verifies a ValidateKeyspaceResponse message. * @function verify - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - TabletExternallyReparentedResponse.verify = function verify(message) { + ValidateKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) { - let error = $root.topodata.TabletAlias.verify(message.new_primary); - if (error) - return "new_primary." + error; + if (message.results != null && message.hasOwnProperty("results")) { + if (!Array.isArray(message.results)) + return "results: array expected"; + for (let i = 0; i < message.results.length; ++i) + if (!$util.isString(message.results[i])) + return "results: string[] expected"; } - if (message.old_primary != null && message.hasOwnProperty("old_primary")) { - let error = $root.topodata.TabletAlias.verify(message.old_primary); - if (error) - return "old_primary." + error; + if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { + if (!$util.isObject(message.results_by_shard)) + return "results_by_shard: object expected"; + let key = Object.keys(message.results_by_shard); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); + if (error) + return "results_by_shard." + error; + } } return null; }; /** - * Creates a TabletExternallyReparentedResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.TabletExternallyReparentedResponse} TabletExternallyReparentedResponse + * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse */ - TabletExternallyReparentedResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.TabletExternallyReparentedResponse) + ValidateKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateKeyspaceResponse) return object; - let message = new $root.vtctldata.TabletExternallyReparentedResponse(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.new_primary != null) { - if (typeof object.new_primary !== "object") - throw TypeError(".vtctldata.TabletExternallyReparentedResponse.new_primary: object expected"); - message.new_primary = $root.topodata.TabletAlias.fromObject(object.new_primary); + let message = new $root.vtctldata.ValidateKeyspaceResponse(); + if (object.results) { + if (!Array.isArray(object.results)) + throw TypeError(".vtctldata.ValidateKeyspaceResponse.results: array expected"); + message.results = []; + for (let i = 0; i < object.results.length; ++i) + message.results[i] = String(object.results[i]); } - if (object.old_primary != null) { - if (typeof object.old_primary !== "object") - throw TypeError(".vtctldata.TabletExternallyReparentedResponse.old_primary: object expected"); - message.old_primary = $root.topodata.TabletAlias.fromObject(object.old_primary); + if (object.results_by_shard) { + if (typeof object.results_by_shard !== "object") + throw TypeError(".vtctldata.ValidateKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard = {}; + for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { + if (typeof object.results_by_shard[keys[i]] !== "object") + throw TypeError(".vtctldata.ValidateKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); + } } return message; }; /** - * Creates a plain object from a TabletExternallyReparentedResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static - * @param {vtctldata.TabletExternallyReparentedResponse} message TabletExternallyReparentedResponse + * @param {vtctldata.ValidateKeyspaceResponse} message ValidateKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - TabletExternallyReparentedResponse.toObject = function toObject(message, options) { + ValidateKeyspaceResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - object.new_primary = null; - object.old_primary = null; + if (options.arrays || options.defaults) + object.results = []; + if (options.objects || options.defaults) + object.results_by_shard = {}; + if (message.results && message.results.length) { + object.results = []; + for (let j = 0; j < message.results.length; ++j) + object.results[j] = message.results[j]; + } + let keys2; + if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { + object.results_by_shard = {}; + for (let j = 0; j < keys2.length; ++j) + object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.new_primary != null && message.hasOwnProperty("new_primary")) - object.new_primary = $root.topodata.TabletAlias.toObject(message.new_primary, options); - if (message.old_primary != null && message.hasOwnProperty("old_primary")) - object.old_primary = $root.topodata.TabletAlias.toObject(message.old_primary, options); return object; }; /** - * Converts this TabletExternallyReparentedResponse to JSON. + * Converts this ValidateKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @instance * @returns {Object.} JSON object */ - TabletExternallyReparentedResponse.prototype.toJSON = function toJSON() { + ValidateKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for TabletExternallyReparentedResponse + * Gets the default type url for ValidateKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.TabletExternallyReparentedResponse + * @memberof vtctldata.ValidateKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - TabletExternallyReparentedResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.TabletExternallyReparentedResponse"; + return typeUrlPrefix + "/vtctldata.ValidateKeyspaceResponse"; }; - return TabletExternallyReparentedResponse; + return ValidateKeyspaceResponse; })(); - vtctldata.UpdateCellInfoRequest = (function() { + vtctldata.ValidateSchemaKeyspaceRequest = (function() { /** - * Properties of an UpdateCellInfoRequest. + * Properties of a ValidateSchemaKeyspaceRequest. * @memberof vtctldata - * @interface IUpdateCellInfoRequest - * @property {string|null} [name] UpdateCellInfoRequest name - * @property {topodata.ICellInfo|null} [cell_info] UpdateCellInfoRequest cell_info + * @interface IValidateSchemaKeyspaceRequest + * @property {string|null} [keyspace] ValidateSchemaKeyspaceRequest keyspace + * @property {Array.|null} [exclude_tables] ValidateSchemaKeyspaceRequest exclude_tables + * @property {boolean|null} [include_views] ValidateSchemaKeyspaceRequest include_views + * @property {boolean|null} [skip_no_primary] ValidateSchemaKeyspaceRequest skip_no_primary + * @property {boolean|null} [include_vschema] ValidateSchemaKeyspaceRequest include_vschema */ /** - * Constructs a new UpdateCellInfoRequest. + * Constructs a new ValidateSchemaKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents an UpdateCellInfoRequest. - * @implements IUpdateCellInfoRequest + * @classdesc Represents a ValidateSchemaKeyspaceRequest. + * @implements IValidateSchemaKeyspaceRequest * @constructor - * @param {vtctldata.IUpdateCellInfoRequest=} [properties] Properties to set + * @param {vtctldata.IValidateSchemaKeyspaceRequest=} [properties] Properties to set */ - function UpdateCellInfoRequest(properties) { + function ValidateSchemaKeyspaceRequest(properties) { + this.exclude_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -138696,89 +145376,134 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * UpdateCellInfoRequest name. - * @member {string} name - * @memberof vtctldata.UpdateCellInfoRequest + * ValidateSchemaKeyspaceRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @instance */ - UpdateCellInfoRequest.prototype.name = ""; + ValidateSchemaKeyspaceRequest.prototype.keyspace = ""; /** - * UpdateCellInfoRequest cell_info. - * @member {topodata.ICellInfo|null|undefined} cell_info - * @memberof vtctldata.UpdateCellInfoRequest + * ValidateSchemaKeyspaceRequest exclude_tables. + * @member {Array.} exclude_tables + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @instance */ - UpdateCellInfoRequest.prototype.cell_info = null; + ValidateSchemaKeyspaceRequest.prototype.exclude_tables = $util.emptyArray; /** - * Creates a new UpdateCellInfoRequest instance using the specified properties. + * ValidateSchemaKeyspaceRequest include_views. + * @member {boolean} include_views + * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @instance + */ + ValidateSchemaKeyspaceRequest.prototype.include_views = false; + + /** + * ValidateSchemaKeyspaceRequest skip_no_primary. + * @member {boolean} skip_no_primary + * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @instance + */ + ValidateSchemaKeyspaceRequest.prototype.skip_no_primary = false; + + /** + * ValidateSchemaKeyspaceRequest include_vschema. + * @member {boolean} include_vschema + * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @instance + */ + ValidateSchemaKeyspaceRequest.prototype.include_vschema = false; + + /** + * Creates a new ValidateSchemaKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static - * @param {vtctldata.IUpdateCellInfoRequest=} [properties] Properties to set - * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest instance + * @param {vtctldata.IValidateSchemaKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest instance */ - UpdateCellInfoRequest.create = function create(properties) { - return new UpdateCellInfoRequest(properties); + ValidateSchemaKeyspaceRequest.create = function create(properties) { + return new ValidateSchemaKeyspaceRequest(properties); }; /** - * Encodes the specified UpdateCellInfoRequest message. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. + * Encodes the specified ValidateSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static - * @param {vtctldata.IUpdateCellInfoRequest} message UpdateCellInfoRequest message or plain object to encode + * @param {vtctldata.IValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellInfoRequest.encode = function encode(message, writer) { + ValidateSchemaKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) - $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.exclude_tables != null && message.exclude_tables.length) + for (let i = 0; i < message.exclude_tables.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.exclude_tables[i]); + if (message.include_views != null && Object.hasOwnProperty.call(message, "include_views")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.include_views); + if (message.skip_no_primary != null && Object.hasOwnProperty.call(message, "skip_no_primary")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.skip_no_primary); + if (message.include_vschema != null && Object.hasOwnProperty.call(message, "include_vschema")) + writer.uint32(/* id 5, wireType 0 =*/40).bool(message.include_vschema); return writer; }; /** - * Encodes the specified UpdateCellInfoRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoRequest.verify|verify} messages. + * Encodes the specified ValidateSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static - * @param {vtctldata.IUpdateCellInfoRequest} message UpdateCellInfoRequest message or plain object to encode + * @param {vtctldata.IValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellInfoRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateSchemaKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateCellInfoRequest message from the specified reader or buffer. + * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest + * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellInfoRequest.decode = function decode(reader, length) { + ValidateSchemaKeyspaceRequest.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.UpdateCellInfoRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateSchemaKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.keyspace = reader.string(); break; } case 2: { - message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); + if (!(message.exclude_tables && message.exclude_tables.length)) + message.exclude_tables = []; + message.exclude_tables.push(reader.string()); + break; + } + case 3: { + message.include_views = reader.bool(); + break; + } + case 4: { + message.skip_no_primary = reader.bool(); + break; + } + case 5: { + message.include_vschema = reader.bool(); break; } default: @@ -138790,227 +145515,286 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateCellInfoRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest + * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellInfoRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateSchemaKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateCellInfoRequest message. + * Verifies a ValidateSchemaKeyspaceRequest message. * @function verify - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateCellInfoRequest.verify = function verify(message) { + ValidateSchemaKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) { - let error = $root.topodata.CellInfo.verify(message.cell_info); - if (error) - return "cell_info." + error; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { + if (!Array.isArray(message.exclude_tables)) + return "exclude_tables: array expected"; + for (let i = 0; i < message.exclude_tables.length; ++i) + if (!$util.isString(message.exclude_tables[i])) + return "exclude_tables: string[] expected"; } + if (message.include_views != null && message.hasOwnProperty("include_views")) + if (typeof message.include_views !== "boolean") + return "include_views: boolean expected"; + if (message.skip_no_primary != null && message.hasOwnProperty("skip_no_primary")) + if (typeof message.skip_no_primary !== "boolean") + return "skip_no_primary: boolean expected"; + if (message.include_vschema != null && message.hasOwnProperty("include_vschema")) + if (typeof message.include_vschema !== "boolean") + return "include_vschema: boolean expected"; return null; }; /** - * Creates an UpdateCellInfoRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateCellInfoRequest} UpdateCellInfoRequest + * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest */ - UpdateCellInfoRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateCellInfoRequest) + ValidateSchemaKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateSchemaKeyspaceRequest) return object; - let message = new $root.vtctldata.UpdateCellInfoRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.cell_info != null) { - if (typeof object.cell_info !== "object") - throw TypeError(".vtctldata.UpdateCellInfoRequest.cell_info: object expected"); - message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + let message = new $root.vtctldata.ValidateSchemaKeyspaceRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.exclude_tables) { + if (!Array.isArray(object.exclude_tables)) + throw TypeError(".vtctldata.ValidateSchemaKeyspaceRequest.exclude_tables: array expected"); + message.exclude_tables = []; + for (let i = 0; i < object.exclude_tables.length; ++i) + message.exclude_tables[i] = String(object.exclude_tables[i]); } + if (object.include_views != null) + message.include_views = Boolean(object.include_views); + if (object.skip_no_primary != null) + message.skip_no_primary = Boolean(object.skip_no_primary); + if (object.include_vschema != null) + message.include_vschema = Boolean(object.include_vschema); return message; }; /** - * Creates a plain object from an UpdateCellInfoRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateSchemaKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static - * @param {vtctldata.UpdateCellInfoRequest} message UpdateCellInfoRequest + * @param {vtctldata.ValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateCellInfoRequest.toObject = function toObject(message, options) { + ValidateSchemaKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) + object.exclude_tables = []; if (options.defaults) { - object.name = ""; - object.cell_info = null; + object.keyspace = ""; + object.include_views = false; + object.skip_no_primary = false; + object.include_vschema = false; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) - object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.exclude_tables && message.exclude_tables.length) { + object.exclude_tables = []; + for (let j = 0; j < message.exclude_tables.length; ++j) + object.exclude_tables[j] = message.exclude_tables[j]; + } + if (message.include_views != null && message.hasOwnProperty("include_views")) + object.include_views = message.include_views; + if (message.skip_no_primary != null && message.hasOwnProperty("skip_no_primary")) + object.skip_no_primary = message.skip_no_primary; + if (message.include_vschema != null && message.hasOwnProperty("include_vschema")) + object.include_vschema = message.include_vschema; return object; }; /** - * Converts this UpdateCellInfoRequest to JSON. + * Converts this ValidateSchemaKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @instance * @returns {Object.} JSON object */ - UpdateCellInfoRequest.prototype.toJSON = function toJSON() { + ValidateSchemaKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateCellInfoRequest + * Gets the default type url for ValidateSchemaKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.UpdateCellInfoRequest + * @memberof vtctldata.ValidateSchemaKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateCellInfoRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateSchemaKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateCellInfoRequest"; + return typeUrlPrefix + "/vtctldata.ValidateSchemaKeyspaceRequest"; }; - return UpdateCellInfoRequest; + return ValidateSchemaKeyspaceRequest; })(); - vtctldata.UpdateCellInfoResponse = (function() { + vtctldata.ValidateSchemaKeyspaceResponse = (function() { /** - * Properties of an UpdateCellInfoResponse. + * Properties of a ValidateSchemaKeyspaceResponse. * @memberof vtctldata - * @interface IUpdateCellInfoResponse - * @property {string|null} [name] UpdateCellInfoResponse name - * @property {topodata.ICellInfo|null} [cell_info] UpdateCellInfoResponse cell_info + * @interface IValidateSchemaKeyspaceResponse + * @property {Array.|null} [results] ValidateSchemaKeyspaceResponse results + * @property {Object.|null} [results_by_shard] ValidateSchemaKeyspaceResponse results_by_shard */ /** - * Constructs a new UpdateCellInfoResponse. + * Constructs a new ValidateSchemaKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents an UpdateCellInfoResponse. - * @implements IUpdateCellInfoResponse + * @classdesc Represents a ValidateSchemaKeyspaceResponse. + * @implements IValidateSchemaKeyspaceResponse * @constructor - * @param {vtctldata.IUpdateCellInfoResponse=} [properties] Properties to set + * @param {vtctldata.IValidateSchemaKeyspaceResponse=} [properties] Properties to set */ - function UpdateCellInfoResponse(properties) { + function ValidateSchemaKeyspaceResponse(properties) { + this.results = []; + this.results_by_shard = {}; 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]]; } - - /** - * UpdateCellInfoResponse name. - * @member {string} name - * @memberof vtctldata.UpdateCellInfoResponse + + /** + * ValidateSchemaKeyspaceResponse results. + * @member {Array.} results + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @instance */ - UpdateCellInfoResponse.prototype.name = ""; + ValidateSchemaKeyspaceResponse.prototype.results = $util.emptyArray; /** - * UpdateCellInfoResponse cell_info. - * @member {topodata.ICellInfo|null|undefined} cell_info - * @memberof vtctldata.UpdateCellInfoResponse + * ValidateSchemaKeyspaceResponse results_by_shard. + * @member {Object.} results_by_shard + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @instance */ - UpdateCellInfoResponse.prototype.cell_info = null; + ValidateSchemaKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; /** - * Creates a new UpdateCellInfoResponse instance using the specified properties. + * Creates a new ValidateSchemaKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static - * @param {vtctldata.IUpdateCellInfoResponse=} [properties] Properties to set - * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse instance + * @param {vtctldata.IValidateSchemaKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse instance */ - UpdateCellInfoResponse.create = function create(properties) { - return new UpdateCellInfoResponse(properties); + ValidateSchemaKeyspaceResponse.create = function create(properties) { + return new ValidateSchemaKeyspaceResponse(properties); }; /** - * Encodes the specified UpdateCellInfoResponse message. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. + * Encodes the specified ValidateSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static - * @param {vtctldata.IUpdateCellInfoResponse} message UpdateCellInfoResponse message or plain object to encode + * @param {vtctldata.IValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellInfoResponse.encode = function encode(message, writer) { + ValidateSchemaKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cell_info != null && Object.hasOwnProperty.call(message, "cell_info")) - $root.topodata.CellInfo.encode(message.cell_info, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.results != null && message.results.length) + for (let i = 0; i < message.results.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); + if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) + for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } return writer; }; /** - * Encodes the specified UpdateCellInfoResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellInfoResponse.verify|verify} messages. + * Encodes the specified ValidateSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static - * @param {vtctldata.IUpdateCellInfoResponse} message UpdateCellInfoResponse message or plain object to encode + * @param {vtctldata.IValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellInfoResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateSchemaKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateCellInfoResponse message from the specified reader or buffer. + * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse + * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellInfoResponse.decode = function decode(reader, length) { + ValidateSchemaKeyspaceResponse.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.UpdateCellInfoResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateSchemaKeyspaceResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + if (!(message.results && message.results.length)) + message.results = []; + message.results.push(reader.string()); break; } case 2: { - message.cell_info = $root.topodata.CellInfo.decode(reader, reader.uint32()); + if (message.results_by_shard === $util.emptyObject) + message.results_by_shard = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.results_by_shard[key] = value; break; } default: @@ -139022,137 +145806,164 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateCellInfoResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse + * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellInfoResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateSchemaKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateCellInfoResponse message. + * Verifies a ValidateSchemaKeyspaceResponse message. * @function verify - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateCellInfoResponse.verify = function verify(message) { + ValidateSchemaKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) { - let error = $root.topodata.CellInfo.verify(message.cell_info); - if (error) - return "cell_info." + error; + if (message.results != null && message.hasOwnProperty("results")) { + if (!Array.isArray(message.results)) + return "results: array expected"; + for (let i = 0; i < message.results.length; ++i) + if (!$util.isString(message.results[i])) + return "results: string[] expected"; + } + if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { + if (!$util.isObject(message.results_by_shard)) + return "results_by_shard: object expected"; + let key = Object.keys(message.results_by_shard); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); + if (error) + return "results_by_shard." + error; + } } return null; }; /** - * Creates an UpdateCellInfoResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateCellInfoResponse} UpdateCellInfoResponse + * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse */ - UpdateCellInfoResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateCellInfoResponse) + ValidateSchemaKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateSchemaKeyspaceResponse) return object; - let message = new $root.vtctldata.UpdateCellInfoResponse(); - if (object.name != null) - message.name = String(object.name); - if (object.cell_info != null) { - if (typeof object.cell_info !== "object") - throw TypeError(".vtctldata.UpdateCellInfoResponse.cell_info: object expected"); - message.cell_info = $root.topodata.CellInfo.fromObject(object.cell_info); + let message = new $root.vtctldata.ValidateSchemaKeyspaceResponse(); + if (object.results) { + if (!Array.isArray(object.results)) + throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results: array expected"); + message.results = []; + for (let i = 0; i < object.results.length; ++i) + message.results[i] = String(object.results[i]); + } + if (object.results_by_shard) { + if (typeof object.results_by_shard !== "object") + throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard = {}; + for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { + if (typeof object.results_by_shard[keys[i]] !== "object") + throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); + } } return message; }; /** - * Creates a plain object from an UpdateCellInfoResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateSchemaKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static - * @param {vtctldata.UpdateCellInfoResponse} message UpdateCellInfoResponse + * @param {vtctldata.ValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateCellInfoResponse.toObject = function toObject(message, options) { + ValidateSchemaKeyspaceResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.name = ""; - object.cell_info = null; + if (options.arrays || options.defaults) + object.results = []; + if (options.objects || options.defaults) + object.results_by_shard = {}; + if (message.results && message.results.length) { + object.results = []; + for (let j = 0; j < message.results.length; ++j) + object.results[j] = message.results[j]; + } + let keys2; + if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { + object.results_by_shard = {}; + for (let j = 0; j < keys2.length; ++j) + object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cell_info != null && message.hasOwnProperty("cell_info")) - object.cell_info = $root.topodata.CellInfo.toObject(message.cell_info, options); return object; }; /** - * Converts this UpdateCellInfoResponse to JSON. + * Converts this ValidateSchemaKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @instance * @returns {Object.} JSON object */ - UpdateCellInfoResponse.prototype.toJSON = function toJSON() { + ValidateSchemaKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateCellInfoResponse + * Gets the default type url for ValidateSchemaKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.UpdateCellInfoResponse + * @memberof vtctldata.ValidateSchemaKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateCellInfoResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateSchemaKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateCellInfoResponse"; + return typeUrlPrefix + "/vtctldata.ValidateSchemaKeyspaceResponse"; }; - return UpdateCellInfoResponse; + return ValidateSchemaKeyspaceResponse; })(); - vtctldata.UpdateCellsAliasRequest = (function() { + vtctldata.ValidateShardRequest = (function() { /** - * Properties of an UpdateCellsAliasRequest. + * Properties of a ValidateShardRequest. * @memberof vtctldata - * @interface IUpdateCellsAliasRequest - * @property {string|null} [name] UpdateCellsAliasRequest name - * @property {topodata.ICellsAlias|null} [cells_alias] UpdateCellsAliasRequest cells_alias + * @interface IValidateShardRequest + * @property {string|null} [keyspace] ValidateShardRequest keyspace + * @property {string|null} [shard] ValidateShardRequest shard + * @property {boolean|null} [ping_tablets] ValidateShardRequest ping_tablets */ /** - * Constructs a new UpdateCellsAliasRequest. + * Constructs a new ValidateShardRequest. * @memberof vtctldata - * @classdesc Represents an UpdateCellsAliasRequest. - * @implements IUpdateCellsAliasRequest + * @classdesc Represents a ValidateShardRequest. + * @implements IValidateShardRequest * @constructor - * @param {vtctldata.IUpdateCellsAliasRequest=} [properties] Properties to set + * @param {vtctldata.IValidateShardRequest=} [properties] Properties to set */ - function UpdateCellsAliasRequest(properties) { + function ValidateShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -139160,89 +145971,103 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * UpdateCellsAliasRequest name. - * @member {string} name - * @memberof vtctldata.UpdateCellsAliasRequest + * ValidateShardRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ValidateShardRequest * @instance */ - UpdateCellsAliasRequest.prototype.name = ""; + ValidateShardRequest.prototype.keyspace = ""; /** - * UpdateCellsAliasRequest cells_alias. - * @member {topodata.ICellsAlias|null|undefined} cells_alias - * @memberof vtctldata.UpdateCellsAliasRequest + * ValidateShardRequest shard. + * @member {string} shard + * @memberof vtctldata.ValidateShardRequest * @instance */ - UpdateCellsAliasRequest.prototype.cells_alias = null; + ValidateShardRequest.prototype.shard = ""; /** - * Creates a new UpdateCellsAliasRequest instance using the specified properties. + * ValidateShardRequest ping_tablets. + * @member {boolean} ping_tablets + * @memberof vtctldata.ValidateShardRequest + * @instance + */ + ValidateShardRequest.prototype.ping_tablets = false; + + /** + * Creates a new ValidateShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static - * @param {vtctldata.IUpdateCellsAliasRequest=} [properties] Properties to set - * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest instance + * @param {vtctldata.IValidateShardRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest instance */ - UpdateCellsAliasRequest.create = function create(properties) { - return new UpdateCellsAliasRequest(properties); + ValidateShardRequest.create = function create(properties) { + return new ValidateShardRequest(properties); }; /** - * Encodes the specified UpdateCellsAliasRequest message. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. + * Encodes the specified ValidateShardRequest message. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static - * @param {vtctldata.IUpdateCellsAliasRequest} message UpdateCellsAliasRequest message or plain object to encode + * @param {vtctldata.IValidateShardRequest} message ValidateShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellsAliasRequest.encode = function encode(message, writer) { + ValidateShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cells_alias != null && Object.hasOwnProperty.call(message, "cells_alias")) - $root.topodata.CellsAlias.encode(message.cells_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); + if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.ping_tablets); return writer; }; /** - * Encodes the specified UpdateCellsAliasRequest message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasRequest.verify|verify} messages. + * Encodes the specified ValidateShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static - * @param {vtctldata.IUpdateCellsAliasRequest} message UpdateCellsAliasRequest message or plain object to encode + * @param {vtctldata.IValidateShardRequest} message ValidateShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellsAliasRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer. + * Decodes a ValidateShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest + * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellsAliasRequest.decode = function decode(reader, length) { + ValidateShardRequest.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.UpdateCellsAliasRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); + message.keyspace = reader.string(); break; } case 2: { - message.cells_alias = $root.topodata.CellsAlias.decode(reader, reader.uint32()); + message.shard = reader.string(); + break; + } + case 3: { + message.ping_tablets = reader.bool(); break; } default: @@ -139254,137 +146079,140 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateCellsAliasRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest + * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellsAliasRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateCellsAliasRequest message. + * Verifies a ValidateShardRequest message. * @function verify - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateCellsAliasRequest.verify = function verify(message) { + ValidateShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) { - let error = $root.topodata.CellsAlias.verify(message.cells_alias); - if (error) - return "cells_alias." + error; - } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + if (typeof message.ping_tablets !== "boolean") + return "ping_tablets: boolean expected"; return null; }; /** - * Creates an UpdateCellsAliasRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateCellsAliasRequest} UpdateCellsAliasRequest + * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest */ - UpdateCellsAliasRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateCellsAliasRequest) + ValidateShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateShardRequest) return object; - let message = new $root.vtctldata.UpdateCellsAliasRequest(); - if (object.name != null) - message.name = String(object.name); - if (object.cells_alias != null) { - if (typeof object.cells_alias !== "object") - throw TypeError(".vtctldata.UpdateCellsAliasRequest.cells_alias: object expected"); - message.cells_alias = $root.topodata.CellsAlias.fromObject(object.cells_alias); - } + let message = new $root.vtctldata.ValidateShardRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.shard != null) + message.shard = String(object.shard); + if (object.ping_tablets != null) + message.ping_tablets = Boolean(object.ping_tablets); return message; }; /** - * Creates a plain object from an UpdateCellsAliasRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static - * @param {vtctldata.UpdateCellsAliasRequest} message UpdateCellsAliasRequest + * @param {vtctldata.ValidateShardRequest} message ValidateShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateCellsAliasRequest.toObject = function toObject(message, options) { + ValidateShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.name = ""; - object.cells_alias = null; + object.keyspace = ""; + object.shard = ""; + object.ping_tablets = false; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) - object.cells_alias = $root.topodata.CellsAlias.toObject(message.cells_alias, options); + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; + if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) + object.ping_tablets = message.ping_tablets; return object; }; /** - * Converts this UpdateCellsAliasRequest to JSON. + * Converts this ValidateShardRequest to JSON. * @function toJSON - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @instance * @returns {Object.} JSON object */ - UpdateCellsAliasRequest.prototype.toJSON = function toJSON() { + ValidateShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateCellsAliasRequest + * Gets the default type url for ValidateShardRequest * @function getTypeUrl - * @memberof vtctldata.UpdateCellsAliasRequest + * @memberof vtctldata.ValidateShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateCellsAliasRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateCellsAliasRequest"; + return typeUrlPrefix + "/vtctldata.ValidateShardRequest"; }; - return UpdateCellsAliasRequest; + return ValidateShardRequest; })(); - vtctldata.UpdateCellsAliasResponse = (function() { + vtctldata.ValidateShardResponse = (function() { /** - * Properties of an UpdateCellsAliasResponse. + * Properties of a ValidateShardResponse. * @memberof vtctldata - * @interface IUpdateCellsAliasResponse - * @property {string|null} [name] UpdateCellsAliasResponse name - * @property {topodata.ICellsAlias|null} [cells_alias] UpdateCellsAliasResponse cells_alias + * @interface IValidateShardResponse + * @property {Array.|null} [results] ValidateShardResponse results */ /** - * Constructs a new UpdateCellsAliasResponse. + * Constructs a new ValidateShardResponse. * @memberof vtctldata - * @classdesc Represents an UpdateCellsAliasResponse. - * @implements IUpdateCellsAliasResponse + * @classdesc Represents a ValidateShardResponse. + * @implements IValidateShardResponse * @constructor - * @param {vtctldata.IUpdateCellsAliasResponse=} [properties] Properties to set + * @param {vtctldata.IValidateShardResponse=} [properties] Properties to set */ - function UpdateCellsAliasResponse(properties) { + function ValidateShardResponse(properties) { + this.results = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -139392,89 +146220,78 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * UpdateCellsAliasResponse name. - * @member {string} name - * @memberof vtctldata.UpdateCellsAliasResponse - * @instance - */ - UpdateCellsAliasResponse.prototype.name = ""; - - /** - * UpdateCellsAliasResponse cells_alias. - * @member {topodata.ICellsAlias|null|undefined} cells_alias - * @memberof vtctldata.UpdateCellsAliasResponse + * ValidateShardResponse results. + * @member {Array.} results + * @memberof vtctldata.ValidateShardResponse * @instance */ - UpdateCellsAliasResponse.prototype.cells_alias = null; + ValidateShardResponse.prototype.results = $util.emptyArray; /** - * Creates a new UpdateCellsAliasResponse instance using the specified properties. + * Creates a new ValidateShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static - * @param {vtctldata.IUpdateCellsAliasResponse=} [properties] Properties to set - * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse instance + * @param {vtctldata.IValidateShardResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse instance */ - UpdateCellsAliasResponse.create = function create(properties) { - return new UpdateCellsAliasResponse(properties); + ValidateShardResponse.create = function create(properties) { + return new ValidateShardResponse(properties); }; /** - * Encodes the specified UpdateCellsAliasResponse message. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. + * Encodes the specified ValidateShardResponse message. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static - * @param {vtctldata.IUpdateCellsAliasResponse} message UpdateCellsAliasResponse message or plain object to encode + * @param {vtctldata.IValidateShardResponse} message ValidateShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellsAliasResponse.encode = function encode(message, writer) { + ValidateShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.name != null && Object.hasOwnProperty.call(message, "name")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.name); - if (message.cells_alias != null && Object.hasOwnProperty.call(message, "cells_alias")) - $root.topodata.CellsAlias.encode(message.cells_alias, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.results != null && message.results.length) + for (let i = 0; i < message.results.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); return writer; }; /** - * Encodes the specified UpdateCellsAliasResponse message, length delimited. Does not implicitly {@link vtctldata.UpdateCellsAliasResponse.verify|verify} messages. + * Encodes the specified ValidateShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static - * @param {vtctldata.IUpdateCellsAliasResponse} message UpdateCellsAliasResponse message or plain object to encode + * @param {vtctldata.IValidateShardResponse} message ValidateShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateCellsAliasResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer. + * Decodes a ValidateShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse + * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellsAliasResponse.decode = function decode(reader, length) { + ValidateShardResponse.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.UpdateCellsAliasResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.name = reader.string(); - break; - } - case 2: { - message.cells_alias = $root.topodata.CellsAlias.decode(reader, reader.uint32()); + if (!(message.results && message.results.length)) + message.results = []; + message.results.push(reader.string()); break; } default: @@ -139486,136 +146303,134 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes an UpdateCellsAliasResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse + * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateCellsAliasResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateCellsAliasResponse message. + * Verifies a ValidateShardResponse message. * @function verify - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateCellsAliasResponse.verify = function verify(message) { + ValidateShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.name != null && message.hasOwnProperty("name")) - if (!$util.isString(message.name)) - return "name: string expected"; - if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) { - let error = $root.topodata.CellsAlias.verify(message.cells_alias); - if (error) - return "cells_alias." + error; + if (message.results != null && message.hasOwnProperty("results")) { + if (!Array.isArray(message.results)) + return "results: array expected"; + for (let i = 0; i < message.results.length; ++i) + if (!$util.isString(message.results[i])) + return "results: string[] expected"; } return null; }; /** - * Creates an UpdateCellsAliasResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.UpdateCellsAliasResponse} UpdateCellsAliasResponse + * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse */ - UpdateCellsAliasResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.UpdateCellsAliasResponse) + ValidateShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateShardResponse) return object; - let message = new $root.vtctldata.UpdateCellsAliasResponse(); - if (object.name != null) - message.name = String(object.name); - if (object.cells_alias != null) { - if (typeof object.cells_alias !== "object") - throw TypeError(".vtctldata.UpdateCellsAliasResponse.cells_alias: object expected"); - message.cells_alias = $root.topodata.CellsAlias.fromObject(object.cells_alias); + let message = new $root.vtctldata.ValidateShardResponse(); + if (object.results) { + if (!Array.isArray(object.results)) + throw TypeError(".vtctldata.ValidateShardResponse.results: array expected"); + message.results = []; + for (let i = 0; i < object.results.length; ++i) + message.results[i] = String(object.results[i]); } return message; }; /** - * Creates a plain object from an UpdateCellsAliasResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static - * @param {vtctldata.UpdateCellsAliasResponse} message UpdateCellsAliasResponse + * @param {vtctldata.ValidateShardResponse} message ValidateShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateCellsAliasResponse.toObject = function toObject(message, options) { + ValidateShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.name = ""; - object.cells_alias = null; + if (options.arrays || options.defaults) + object.results = []; + if (message.results && message.results.length) { + object.results = []; + for (let j = 0; j < message.results.length; ++j) + object.results[j] = message.results[j]; } - if (message.name != null && message.hasOwnProperty("name")) - object.name = message.name; - if (message.cells_alias != null && message.hasOwnProperty("cells_alias")) - object.cells_alias = $root.topodata.CellsAlias.toObject(message.cells_alias, options); return object; }; /** - * Converts this UpdateCellsAliasResponse to JSON. + * Converts this ValidateShardResponse to JSON. * @function toJSON - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @instance * @returns {Object.} JSON object */ - UpdateCellsAliasResponse.prototype.toJSON = function toJSON() { + ValidateShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateCellsAliasResponse + * Gets the default type url for ValidateShardResponse * @function getTypeUrl - * @memberof vtctldata.UpdateCellsAliasResponse + * @memberof vtctldata.ValidateShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateCellsAliasResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.UpdateCellsAliasResponse"; + return typeUrlPrefix + "/vtctldata.ValidateShardResponse"; }; - return UpdateCellsAliasResponse; + return ValidateShardResponse; })(); - vtctldata.ValidateRequest = (function() { + vtctldata.ValidateVersionKeyspaceRequest = (function() { /** - * Properties of a ValidateRequest. + * Properties of a ValidateVersionKeyspaceRequest. * @memberof vtctldata - * @interface IValidateRequest - * @property {boolean|null} [ping_tablets] ValidateRequest ping_tablets + * @interface IValidateVersionKeyspaceRequest + * @property {string|null} [keyspace] ValidateVersionKeyspaceRequest keyspace */ /** - * Constructs a new ValidateRequest. + * Constructs a new ValidateVersionKeyspaceRequest. * @memberof vtctldata - * @classdesc Represents a ValidateRequest. - * @implements IValidateRequest + * @classdesc Represents a ValidateVersionKeyspaceRequest. + * @implements IValidateVersionKeyspaceRequest * @constructor - * @param {vtctldata.IValidateRequest=} [properties] Properties to set + * @param {vtctldata.IValidateVersionKeyspaceRequest=} [properties] Properties to set */ - function ValidateRequest(properties) { + function ValidateVersionKeyspaceRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -139623,75 +146438,75 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateRequest ping_tablets. - * @member {boolean} ping_tablets - * @memberof vtctldata.ValidateRequest + * ValidateVersionKeyspaceRequest keyspace. + * @member {string} keyspace + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @instance */ - ValidateRequest.prototype.ping_tablets = false; + ValidateVersionKeyspaceRequest.prototype.keyspace = ""; /** - * Creates a new ValidateRequest instance using the specified properties. + * Creates a new ValidateVersionKeyspaceRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static - * @param {vtctldata.IValidateRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateRequest} ValidateRequest instance + * @param {vtctldata.IValidateVersionKeyspaceRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest instance */ - ValidateRequest.create = function create(properties) { - return new ValidateRequest(properties); + ValidateVersionKeyspaceRequest.create = function create(properties) { + return new ValidateVersionKeyspaceRequest(properties); }; /** - * Encodes the specified ValidateRequest message. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. + * Encodes the specified ValidateVersionKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static - * @param {vtctldata.IValidateRequest} message ValidateRequest message or plain object to encode + * @param {vtctldata.IValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateRequest.encode = function encode(message, writer) { + ValidateVersionKeyspaceRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) - writer.uint32(/* id 1, wireType 0 =*/8).bool(message.ping_tablets); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); return writer; }; /** - * Encodes the specified ValidateRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateRequest.verify|verify} messages. + * Encodes the specified ValidateVersionKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static - * @param {vtctldata.IValidateRequest} message ValidateRequest message or plain object to encode + * @param {vtctldata.IValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVersionKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateRequest message from the specified reader or buffer. + * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateRequest} ValidateRequest + * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateRequest.decode = function decode(reader, length) { + ValidateVersionKeyspaceRequest.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.ValidateRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVersionKeyspaceRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.ping_tablets = reader.bool(); + message.keyspace = reader.string(); break; } default: @@ -139703,125 +146518,125 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateRequest} ValidateRequest + * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateVersionKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateRequest message. + * Verifies a ValidateVersionKeyspaceRequest message. * @function verify - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateRequest.verify = function verify(message) { + ValidateVersionKeyspaceRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - if (typeof message.ping_tablets !== "boolean") - return "ping_tablets: boolean expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; return null; }; /** - * Creates a ValidateRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionKeyspaceRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateRequest} ValidateRequest + * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest */ - ValidateRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateRequest) + ValidateVersionKeyspaceRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVersionKeyspaceRequest) return object; - let message = new $root.vtctldata.ValidateRequest(); - if (object.ping_tablets != null) - message.ping_tablets = Boolean(object.ping_tablets); + let message = new $root.vtctldata.ValidateVersionKeyspaceRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); return message; }; /** - * Creates a plain object from a ValidateRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVersionKeyspaceRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static - * @param {vtctldata.ValidateRequest} message ValidateRequest + * @param {vtctldata.ValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateRequest.toObject = function toObject(message, options) { + ValidateVersionKeyspaceRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) - object.ping_tablets = false; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - object.ping_tablets = message.ping_tablets; + object.keyspace = ""; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; return object; }; /** - * Converts this ValidateRequest to JSON. + * Converts this ValidateVersionKeyspaceRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @instance * @returns {Object.} JSON object */ - ValidateRequest.prototype.toJSON = function toJSON() { + ValidateVersionKeyspaceRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateRequest + * Gets the default type url for ValidateVersionKeyspaceRequest * @function getTypeUrl - * @memberof vtctldata.ValidateRequest + * @memberof vtctldata.ValidateVersionKeyspaceRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVersionKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateRequest"; + return typeUrlPrefix + "/vtctldata.ValidateVersionKeyspaceRequest"; }; - return ValidateRequest; + return ValidateVersionKeyspaceRequest; })(); - vtctldata.ValidateResponse = (function() { + vtctldata.ValidateVersionKeyspaceResponse = (function() { /** - * Properties of a ValidateResponse. + * Properties of a ValidateVersionKeyspaceResponse. * @memberof vtctldata - * @interface IValidateResponse - * @property {Array.|null} [results] ValidateResponse results - * @property {Object.|null} [results_by_keyspace] ValidateResponse results_by_keyspace + * @interface IValidateVersionKeyspaceResponse + * @property {Array.|null} [results] ValidateVersionKeyspaceResponse results + * @property {Object.|null} [results_by_shard] ValidateVersionKeyspaceResponse results_by_shard */ /** - * Constructs a new ValidateResponse. + * Constructs a new ValidateVersionKeyspaceResponse. * @memberof vtctldata - * @classdesc Represents a ValidateResponse. - * @implements IValidateResponse + * @classdesc Represents a ValidateVersionKeyspaceResponse. + * @implements IValidateVersionKeyspaceResponse * @constructor - * @param {vtctldata.IValidateResponse=} [properties] Properties to set + * @param {vtctldata.IValidateVersionKeyspaceResponse=} [properties] Properties to set */ - function ValidateResponse(properties) { + function ValidateVersionKeyspaceResponse(properties) { this.results = []; - this.results_by_keyspace = {}; + this.results_by_shard = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -139829,84 +146644,84 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateResponse results. + * ValidateVersionKeyspaceResponse results. * @member {Array.} results - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @instance */ - ValidateResponse.prototype.results = $util.emptyArray; + ValidateVersionKeyspaceResponse.prototype.results = $util.emptyArray; /** - * ValidateResponse results_by_keyspace. - * @member {Object.} results_by_keyspace - * @memberof vtctldata.ValidateResponse + * ValidateVersionKeyspaceResponse results_by_shard. + * @member {Object.} results_by_shard + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @instance */ - ValidateResponse.prototype.results_by_keyspace = $util.emptyObject; + ValidateVersionKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; /** - * Creates a new ValidateResponse instance using the specified properties. + * Creates a new ValidateVersionKeyspaceResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static - * @param {vtctldata.IValidateResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateResponse} ValidateResponse instance + * @param {vtctldata.IValidateVersionKeyspaceResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse instance */ - ValidateResponse.create = function create(properties) { - return new ValidateResponse(properties); + ValidateVersionKeyspaceResponse.create = function create(properties) { + return new ValidateVersionKeyspaceResponse(properties); }; /** - * Encodes the specified ValidateResponse message. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. + * Encodes the specified ValidateVersionKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static - * @param {vtctldata.IValidateResponse} message ValidateResponse message or plain object to encode + * @param {vtctldata.IValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateResponse.encode = function encode(message, writer) { + ValidateVersionKeyspaceResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.results != null && message.results.length) for (let i = 0; i < message.results.length; ++i) writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); - if (message.results_by_keyspace != null && Object.hasOwnProperty.call(message, "results_by_keyspace")) - for (let keys = Object.keys(message.results_by_keyspace), i = 0; i < keys.length; ++i) { + if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) + for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.ValidateKeyspaceResponse.encode(message.results_by_keyspace[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); } return writer; }; /** - * Encodes the specified ValidateResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateResponse.verify|verify} messages. + * Encodes the specified ValidateVersionKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static - * @param {vtctldata.IValidateResponse} message ValidateResponse message or plain object to encode + * @param {vtctldata.IValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVersionKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateResponse message from the specified reader or buffer. + * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateResponse} ValidateResponse + * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateResponse.decode = function decode(reader, length) { + ValidateVersionKeyspaceResponse.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.ValidateResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVersionKeyspaceResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -139917,8 +146732,8 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - if (message.results_by_keyspace === $util.emptyObject) - message.results_by_keyspace = {}; + if (message.results_by_shard === $util.emptyObject) + message.results_by_shard = {}; let end2 = reader.uint32() + reader.pos; key = ""; value = null; @@ -139929,14 +146744,14 @@ export const vtctldata = $root.vtctldata = (() => { key = reader.string(); break; case 2: - value = $root.vtctldata.ValidateKeyspaceResponse.decode(reader, reader.uint32()); + value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); break; default: reader.skipType(tag2 & 7); break; } } - message.results_by_keyspace[key] = value; + message.results_by_shard[key] = value; break; } default: @@ -139948,30 +146763,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateResponse} ValidateResponse + * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateVersionKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateResponse message. + * Verifies a ValidateVersionKeyspaceResponse message. * @function verify - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateResponse.verify = function verify(message) { + ValidateVersionKeyspaceResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.results != null && message.hasOwnProperty("results")) { @@ -139981,130 +146796,130 @@ export const vtctldata = $root.vtctldata = (() => { if (!$util.isString(message.results[i])) return "results: string[] expected"; } - if (message.results_by_keyspace != null && message.hasOwnProperty("results_by_keyspace")) { - if (!$util.isObject(message.results_by_keyspace)) - return "results_by_keyspace: object expected"; - let key = Object.keys(message.results_by_keyspace); + if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { + if (!$util.isObject(message.results_by_shard)) + return "results_by_shard: object expected"; + let key = Object.keys(message.results_by_shard); for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.ValidateKeyspaceResponse.verify(message.results_by_keyspace[key[i]]); + let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); if (error) - return "results_by_keyspace." + error; + return "results_by_shard." + error; } } return null; }; /** - * Creates a ValidateResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionKeyspaceResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateResponse} ValidateResponse + * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse */ - ValidateResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateResponse) + ValidateVersionKeyspaceResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVersionKeyspaceResponse) return object; - let message = new $root.vtctldata.ValidateResponse(); + let message = new $root.vtctldata.ValidateVersionKeyspaceResponse(); if (object.results) { if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateResponse.results: array expected"); + throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results: array expected"); message.results = []; for (let i = 0; i < object.results.length; ++i) message.results[i] = String(object.results[i]); } - if (object.results_by_keyspace) { - if (typeof object.results_by_keyspace !== "object") - throw TypeError(".vtctldata.ValidateResponse.results_by_keyspace: object expected"); - message.results_by_keyspace = {}; - for (let keys = Object.keys(object.results_by_keyspace), i = 0; i < keys.length; ++i) { - if (typeof object.results_by_keyspace[keys[i]] !== "object") - throw TypeError(".vtctldata.ValidateResponse.results_by_keyspace: object expected"); - message.results_by_keyspace[keys[i]] = $root.vtctldata.ValidateKeyspaceResponse.fromObject(object.results_by_keyspace[keys[i]]); + if (object.results_by_shard) { + if (typeof object.results_by_shard !== "object") + throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard = {}; + for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { + if (typeof object.results_by_shard[keys[i]] !== "object") + throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results_by_shard: object expected"); + message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); } } return message; }; /** - * Creates a plain object from a ValidateResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVersionKeyspaceResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static - * @param {vtctldata.ValidateResponse} message ValidateResponse + * @param {vtctldata.ValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateResponse.toObject = function toObject(message, options) { + ValidateVersionKeyspaceResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) object.results = []; if (options.objects || options.defaults) - object.results_by_keyspace = {}; + object.results_by_shard = {}; if (message.results && message.results.length) { object.results = []; for (let j = 0; j < message.results.length; ++j) object.results[j] = message.results[j]; } let keys2; - if (message.results_by_keyspace && (keys2 = Object.keys(message.results_by_keyspace)).length) { - object.results_by_keyspace = {}; + if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { + object.results_by_shard = {}; for (let j = 0; j < keys2.length; ++j) - object.results_by_keyspace[keys2[j]] = $root.vtctldata.ValidateKeyspaceResponse.toObject(message.results_by_keyspace[keys2[j]], options); + object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); } return object; }; /** - * Converts this ValidateResponse to JSON. + * Converts this ValidateVersionKeyspaceResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @instance * @returns {Object.} JSON object */ - ValidateResponse.prototype.toJSON = function toJSON() { + ValidateVersionKeyspaceResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateResponse + * Gets the default type url for ValidateVersionKeyspaceResponse * @function getTypeUrl - * @memberof vtctldata.ValidateResponse + * @memberof vtctldata.ValidateVersionKeyspaceResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVersionKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateResponse"; + return typeUrlPrefix + "/vtctldata.ValidateVersionKeyspaceResponse"; }; - return ValidateResponse; + return ValidateVersionKeyspaceResponse; })(); - vtctldata.ValidateKeyspaceRequest = (function() { + vtctldata.ValidateVersionShardRequest = (function() { /** - * Properties of a ValidateKeyspaceRequest. + * Properties of a ValidateVersionShardRequest. * @memberof vtctldata - * @interface IValidateKeyspaceRequest - * @property {string|null} [keyspace] ValidateKeyspaceRequest keyspace - * @property {boolean|null} [ping_tablets] ValidateKeyspaceRequest ping_tablets + * @interface IValidateVersionShardRequest + * @property {string|null} [keyspace] ValidateVersionShardRequest keyspace + * @property {string|null} [shard] ValidateVersionShardRequest shard */ /** - * Constructs a new ValidateKeyspaceRequest. + * Constructs a new ValidateVersionShardRequest. * @memberof vtctldata - * @classdesc Represents a ValidateKeyspaceRequest. - * @implements IValidateKeyspaceRequest + * @classdesc Represents a ValidateVersionShardRequest. + * @implements IValidateVersionShardRequest * @constructor - * @param {vtctldata.IValidateKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IValidateVersionShardRequest=} [properties] Properties to set */ - function ValidateKeyspaceRequest(properties) { + function ValidateVersionShardRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -140112,80 +146927,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateKeyspaceRequest keyspace. + * ValidateVersionShardRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @instance */ - ValidateKeyspaceRequest.prototype.keyspace = ""; + ValidateVersionShardRequest.prototype.keyspace = ""; /** - * ValidateKeyspaceRequest ping_tablets. - * @member {boolean} ping_tablets - * @memberof vtctldata.ValidateKeyspaceRequest + * ValidateVersionShardRequest shard. + * @member {string} shard + * @memberof vtctldata.ValidateVersionShardRequest * @instance */ - ValidateKeyspaceRequest.prototype.ping_tablets = false; + ValidateVersionShardRequest.prototype.shard = ""; /** - * Creates a new ValidateKeyspaceRequest instance using the specified properties. + * Creates a new ValidateVersionShardRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static - * @param {vtctldata.IValidateKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest instance + * @param {vtctldata.IValidateVersionShardRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest instance */ - ValidateKeyspaceRequest.create = function create(properties) { - return new ValidateKeyspaceRequest(properties); + ValidateVersionShardRequest.create = function create(properties) { + return new ValidateVersionShardRequest(properties); }; /** - * Encodes the specified ValidateKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. + * Encodes the specified ValidateVersionShardRequest message. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static - * @param {vtctldata.IValidateKeyspaceRequest} message ValidateKeyspaceRequest message or plain object to encode + * @param {vtctldata.IValidateVersionShardRequest} message ValidateVersionShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateKeyspaceRequest.encode = function encode(message, writer) { + ValidateVersionShardRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.ping_tablets); + if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); return writer; }; /** - * Encodes the specified ValidateKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceRequest.verify|verify} messages. + * Encodes the specified ValidateVersionShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static - * @param {vtctldata.IValidateKeyspaceRequest} message ValidateKeyspaceRequest message or plain object to encode + * @param {vtctldata.IValidateVersionShardRequest} message ValidateVersionShardRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVersionShardRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer. + * Decodes a ValidateVersionShardRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest + * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateKeyspaceRequest.decode = function decode(reader, length) { + ValidateVersionShardRequest.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.ValidateKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVersionShardRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -140194,7 +147009,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.ping_tablets = reader.bool(); + message.shard = reader.string(); break; } default: @@ -140206,134 +147021,132 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionShardRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest + * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateVersionShardRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateKeyspaceRequest message. + * Verifies a ValidateVersionShardRequest message. * @function verify - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateKeyspaceRequest.verify = function verify(message) { + ValidateVersionShardRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - if (typeof message.ping_tablets !== "boolean") - return "ping_tablets: boolean expected"; + if (message.shard != null && message.hasOwnProperty("shard")) + if (!$util.isString(message.shard)) + return "shard: string expected"; return null; }; /** - * Creates a ValidateKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionShardRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateKeyspaceRequest} ValidateKeyspaceRequest + * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest */ - ValidateKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateKeyspaceRequest) + ValidateVersionShardRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVersionShardRequest) return object; - let message = new $root.vtctldata.ValidateKeyspaceRequest(); + let message = new $root.vtctldata.ValidateVersionShardRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.ping_tablets != null) - message.ping_tablets = Boolean(object.ping_tablets); + if (object.shard != null) + message.shard = String(object.shard); return message; }; /** - * Creates a plain object from a ValidateKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVersionShardRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static - * @param {vtctldata.ValidateKeyspaceRequest} message ValidateKeyspaceRequest + * @param {vtctldata.ValidateVersionShardRequest} message ValidateVersionShardRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateKeyspaceRequest.toObject = function toObject(message, options) { + ValidateVersionShardRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; - object.ping_tablets = false; + object.shard = ""; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - object.ping_tablets = message.ping_tablets; + if (message.shard != null && message.hasOwnProperty("shard")) + object.shard = message.shard; return object; }; /** - * Converts this ValidateKeyspaceRequest to JSON. + * Converts this ValidateVersionShardRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @instance * @returns {Object.} JSON object */ - ValidateKeyspaceRequest.prototype.toJSON = function toJSON() { + ValidateVersionShardRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateKeyspaceRequest + * Gets the default type url for ValidateVersionShardRequest * @function getTypeUrl - * @memberof vtctldata.ValidateKeyspaceRequest + * @memberof vtctldata.ValidateVersionShardRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVersionShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.ValidateVersionShardRequest"; }; - return ValidateKeyspaceRequest; + return ValidateVersionShardRequest; })(); - vtctldata.ValidateKeyspaceResponse = (function() { + vtctldata.ValidateVersionShardResponse = (function() { /** - * Properties of a ValidateKeyspaceResponse. + * Properties of a ValidateVersionShardResponse. * @memberof vtctldata - * @interface IValidateKeyspaceResponse - * @property {Array.|null} [results] ValidateKeyspaceResponse results - * @property {Object.|null} [results_by_shard] ValidateKeyspaceResponse results_by_shard + * @interface IValidateVersionShardResponse + * @property {Array.|null} [results] ValidateVersionShardResponse results */ /** - * Constructs a new ValidateKeyspaceResponse. + * Constructs a new ValidateVersionShardResponse. * @memberof vtctldata - * @classdesc Represents a ValidateKeyspaceResponse. - * @implements IValidateKeyspaceResponse + * @classdesc Represents a ValidateVersionShardResponse. + * @implements IValidateVersionShardResponse * @constructor - * @param {vtctldata.IValidateKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IValidateVersionShardResponse=} [properties] Properties to set */ - function ValidateKeyspaceResponse(properties) { + function ValidateVersionShardResponse(properties) { this.results = []; - this.results_by_shard = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -140341,84 +147154,71 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateKeyspaceResponse results. + * ValidateVersionShardResponse results. * @member {Array.} results - * @memberof vtctldata.ValidateKeyspaceResponse - * @instance - */ - ValidateKeyspaceResponse.prototype.results = $util.emptyArray; - - /** - * ValidateKeyspaceResponse results_by_shard. - * @member {Object.} results_by_shard - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @instance */ - ValidateKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; + ValidateVersionShardResponse.prototype.results = $util.emptyArray; /** - * Creates a new ValidateKeyspaceResponse instance using the specified properties. + * Creates a new ValidateVersionShardResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static - * @param {vtctldata.IValidateKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse instance + * @param {vtctldata.IValidateVersionShardResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse instance */ - ValidateKeyspaceResponse.create = function create(properties) { - return new ValidateKeyspaceResponse(properties); + ValidateVersionShardResponse.create = function create(properties) { + return new ValidateVersionShardResponse(properties); }; /** - * Encodes the specified ValidateKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. + * Encodes the specified ValidateVersionShardResponse message. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static - * @param {vtctldata.IValidateKeyspaceResponse} message ValidateKeyspaceResponse message or plain object to encode + * @param {vtctldata.IValidateVersionShardResponse} message ValidateVersionShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateKeyspaceResponse.encode = function encode(message, writer) { + ValidateVersionShardResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.results != null && message.results.length) for (let i = 0; i < message.results.length; ++i) writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); - if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) - for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } return writer; }; /** - * Encodes the specified ValidateKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateKeyspaceResponse.verify|verify} messages. + * Encodes the specified ValidateVersionShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static - * @param {vtctldata.IValidateKeyspaceResponse} message ValidateKeyspaceResponse message or plain object to encode + * @param {vtctldata.IValidateVersionShardResponse} message ValidateVersionShardResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVersionShardResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer. + * Decodes a ValidateVersionShardResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse + * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateKeyspaceResponse.decode = function decode(reader, length) { + ValidateVersionShardResponse.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.ValidateKeyspaceResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVersionShardResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -140428,29 +147228,6 @@ export const vtctldata = $root.vtctldata = (() => { message.results.push(reader.string()); break; } - case 2: { - if (message.results_by_shard === $util.emptyObject) - message.results_by_shard = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.results_by_shard[key] = value; - break; - } default: reader.skipType(tag & 7); break; @@ -140460,30 +147237,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVersionShardResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse + * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateVersionShardResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateKeyspaceResponse message. + * Verifies a ValidateVersionShardResponse message. * @function verify - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateKeyspaceResponse.verify = function verify(message) { + ValidateVersionShardResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.results != null && message.hasOwnProperty("results")) { @@ -140493,133 +147270,105 @@ export const vtctldata = $root.vtctldata = (() => { if (!$util.isString(message.results[i])) return "results: string[] expected"; } - if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { - if (!$util.isObject(message.results_by_shard)) - return "results_by_shard: object expected"; - let key = Object.keys(message.results_by_shard); - for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); - if (error) - return "results_by_shard." + error; - } - } return null; }; /** - * Creates a ValidateKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVersionShardResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateKeyspaceResponse} ValidateKeyspaceResponse + * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse */ - ValidateKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateKeyspaceResponse) + ValidateVersionShardResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVersionShardResponse) return object; - let message = new $root.vtctldata.ValidateKeyspaceResponse(); + let message = new $root.vtctldata.ValidateVersionShardResponse(); if (object.results) { if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateKeyspaceResponse.results: array expected"); + throw TypeError(".vtctldata.ValidateVersionShardResponse.results: array expected"); message.results = []; for (let i = 0; i < object.results.length; ++i) message.results[i] = String(object.results[i]); } - if (object.results_by_shard) { - if (typeof object.results_by_shard !== "object") - throw TypeError(".vtctldata.ValidateKeyspaceResponse.results_by_shard: object expected"); - message.results_by_shard = {}; - for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { - if (typeof object.results_by_shard[keys[i]] !== "object") - throw TypeError(".vtctldata.ValidateKeyspaceResponse.results_by_shard: object expected"); - message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); - } - } return message; }; /** - * Creates a plain object from a ValidateKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVersionShardResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static - * @param {vtctldata.ValidateKeyspaceResponse} message ValidateKeyspaceResponse + * @param {vtctldata.ValidateVersionShardResponse} message ValidateVersionShardResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateKeyspaceResponse.toObject = function toObject(message, options) { + ValidateVersionShardResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) object.results = []; - if (options.objects || options.defaults) - object.results_by_shard = {}; if (message.results && message.results.length) { object.results = []; for (let j = 0; j < message.results.length; ++j) object.results[j] = message.results[j]; } - let keys2; - if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { - object.results_by_shard = {}; - for (let j = 0; j < keys2.length; ++j) - object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); - } return object; }; /** - * Converts this ValidateKeyspaceResponse to JSON. + * Converts this ValidateVersionShardResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @instance * @returns {Object.} JSON object */ - ValidateKeyspaceResponse.prototype.toJSON = function toJSON() { + ValidateVersionShardResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateKeyspaceResponse + * Gets the default type url for ValidateVersionShardResponse * @function getTypeUrl - * @memberof vtctldata.ValidateKeyspaceResponse + * @memberof vtctldata.ValidateVersionShardResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVersionShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.ValidateVersionShardResponse"; }; - return ValidateKeyspaceResponse; + return ValidateVersionShardResponse; })(); - vtctldata.ValidateSchemaKeyspaceRequest = (function() { + vtctldata.ValidateVSchemaRequest = (function() { /** - * Properties of a ValidateSchemaKeyspaceRequest. + * Properties of a ValidateVSchemaRequest. * @memberof vtctldata - * @interface IValidateSchemaKeyspaceRequest - * @property {string|null} [keyspace] ValidateSchemaKeyspaceRequest keyspace - * @property {Array.|null} [exclude_tables] ValidateSchemaKeyspaceRequest exclude_tables - * @property {boolean|null} [include_views] ValidateSchemaKeyspaceRequest include_views - * @property {boolean|null} [skip_no_primary] ValidateSchemaKeyspaceRequest skip_no_primary - * @property {boolean|null} [include_vschema] ValidateSchemaKeyspaceRequest include_vschema + * @interface IValidateVSchemaRequest + * @property {string|null} [keyspace] ValidateVSchemaRequest keyspace + * @property {Array.|null} [shards] ValidateVSchemaRequest shards + * @property {Array.|null} [exclude_tables] ValidateVSchemaRequest exclude_tables + * @property {boolean|null} [include_views] ValidateVSchemaRequest include_views */ /** - * Constructs a new ValidateSchemaKeyspaceRequest. + * Constructs a new ValidateVSchemaRequest. * @memberof vtctldata - * @classdesc Represents a ValidateSchemaKeyspaceRequest. - * @implements IValidateSchemaKeyspaceRequest + * @classdesc Represents a ValidateVSchemaRequest. + * @implements IValidateVSchemaRequest * @constructor - * @param {vtctldata.IValidateSchemaKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IValidateVSchemaRequest=} [properties] Properties to set */ - function ValidateSchemaKeyspaceRequest(properties) { + function ValidateVSchemaRequest(properties) { + this.shards = []; this.exclude_tables = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) @@ -140628,111 +147377,102 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateSchemaKeyspaceRequest keyspace. + * ValidateVSchemaRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ValidateSchemaKeyspaceRequest - * @instance - */ - ValidateSchemaKeyspaceRequest.prototype.keyspace = ""; - - /** - * ValidateSchemaKeyspaceRequest exclude_tables. - * @member {Array.} exclude_tables - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @instance */ - ValidateSchemaKeyspaceRequest.prototype.exclude_tables = $util.emptyArray; + ValidateVSchemaRequest.prototype.keyspace = ""; /** - * ValidateSchemaKeyspaceRequest include_views. - * @member {boolean} include_views - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * ValidateVSchemaRequest shards. + * @member {Array.} shards + * @memberof vtctldata.ValidateVSchemaRequest * @instance */ - ValidateSchemaKeyspaceRequest.prototype.include_views = false; + ValidateVSchemaRequest.prototype.shards = $util.emptyArray; /** - * ValidateSchemaKeyspaceRequest skip_no_primary. - * @member {boolean} skip_no_primary - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * ValidateVSchemaRequest exclude_tables. + * @member {Array.} exclude_tables + * @memberof vtctldata.ValidateVSchemaRequest * @instance */ - ValidateSchemaKeyspaceRequest.prototype.skip_no_primary = false; + ValidateVSchemaRequest.prototype.exclude_tables = $util.emptyArray; /** - * ValidateSchemaKeyspaceRequest include_vschema. - * @member {boolean} include_vschema - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * ValidateVSchemaRequest include_views. + * @member {boolean} include_views + * @memberof vtctldata.ValidateVSchemaRequest * @instance */ - ValidateSchemaKeyspaceRequest.prototype.include_vschema = false; + ValidateVSchemaRequest.prototype.include_views = false; /** - * Creates a new ValidateSchemaKeyspaceRequest instance using the specified properties. + * Creates a new ValidateVSchemaRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static - * @param {vtctldata.IValidateSchemaKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest instance + * @param {vtctldata.IValidateVSchemaRequest=} [properties] Properties to set + * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest instance */ - ValidateSchemaKeyspaceRequest.create = function create(properties) { - return new ValidateSchemaKeyspaceRequest(properties); + ValidateVSchemaRequest.create = function create(properties) { + return new ValidateVSchemaRequest(properties); }; /** - * Encodes the specified ValidateSchemaKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. + * Encodes the specified ValidateVSchemaRequest message. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static - * @param {vtctldata.IValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest message or plain object to encode + * @param {vtctldata.IValidateVSchemaRequest} message ValidateVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateSchemaKeyspaceRequest.encode = function encode(message, writer) { + ValidateVSchemaRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.shards != null && message.shards.length) + for (let i = 0; i < message.shards.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.shards[i]); if (message.exclude_tables != null && message.exclude_tables.length) for (let i = 0; i < message.exclude_tables.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.exclude_tables[i]); + writer.uint32(/* id 3, wireType 2 =*/26).string(message.exclude_tables[i]); if (message.include_views != null && Object.hasOwnProperty.call(message, "include_views")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.include_views); - if (message.skip_no_primary != null && Object.hasOwnProperty.call(message, "skip_no_primary")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.skip_no_primary); - if (message.include_vschema != null && Object.hasOwnProperty.call(message, "include_vschema")) - writer.uint32(/* id 5, wireType 0 =*/40).bool(message.include_vschema); + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_views); return writer; }; /** - * Encodes the specified ValidateSchemaKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceRequest.verify|verify} messages. + * Encodes the specified ValidateVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static - * @param {vtctldata.IValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest message or plain object to encode + * @param {vtctldata.IValidateVSchemaRequest} message ValidateVSchemaRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateSchemaKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer. + * Decodes a ValidateVSchemaRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest + * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateSchemaKeyspaceRequest.decode = function decode(reader, length) { + ValidateVSchemaRequest.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.ValidateSchemaKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVSchemaRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -140741,21 +147481,19 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - if (!(message.exclude_tables && message.exclude_tables.length)) - message.exclude_tables = []; - message.exclude_tables.push(reader.string()); + if (!(message.shards && message.shards.length)) + message.shards = []; + message.shards.push(reader.string()); break; } case 3: { - message.include_views = reader.bool(); + if (!(message.exclude_tables && message.exclude_tables.length)) + message.exclude_tables = []; + message.exclude_tables.push(reader.string()); break; } case 4: { - message.skip_no_primary = reader.bool(); - break; - } - case 5: { - message.include_vschema = reader.bool(); + message.include_views = reader.bool(); break; } default: @@ -140767,35 +147505,42 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateSchemaKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a ValidateVSchemaRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest + * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateSchemaKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + ValidateVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateSchemaKeyspaceRequest message. + * Verifies a ValidateVSchemaRequest message. * @function verify - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateSchemaKeyspaceRequest.verify = function verify(message) { + ValidateVSchemaRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; + if (message.shards != null && message.hasOwnProperty("shards")) { + if (!Array.isArray(message.shards)) + return "shards: array expected"; + for (let i = 0; i < message.shards.length; ++i) + if (!$util.isString(message.shards[i])) + return "shards: string[] expected"; + } if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { if (!Array.isArray(message.exclude_tables)) return "exclude_tables: array expected"; @@ -140806,68 +147551,70 @@ export const vtctldata = $root.vtctldata = (() => { if (message.include_views != null && message.hasOwnProperty("include_views")) if (typeof message.include_views !== "boolean") return "include_views: boolean expected"; - if (message.skip_no_primary != null && message.hasOwnProperty("skip_no_primary")) - if (typeof message.skip_no_primary !== "boolean") - return "skip_no_primary: boolean expected"; - if (message.include_vschema != null && message.hasOwnProperty("include_vschema")) - if (typeof message.include_vschema !== "boolean") - return "include_vschema: boolean expected"; return null; }; /** - * Creates a ValidateSchemaKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVSchemaRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateSchemaKeyspaceRequest} ValidateSchemaKeyspaceRequest + * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest */ - ValidateSchemaKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateSchemaKeyspaceRequest) + ValidateVSchemaRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVSchemaRequest) return object; - let message = new $root.vtctldata.ValidateSchemaKeyspaceRequest(); + let message = new $root.vtctldata.ValidateVSchemaRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); + if (object.shards) { + if (!Array.isArray(object.shards)) + throw TypeError(".vtctldata.ValidateVSchemaRequest.shards: array expected"); + message.shards = []; + for (let i = 0; i < object.shards.length; ++i) + message.shards[i] = String(object.shards[i]); + } if (object.exclude_tables) { if (!Array.isArray(object.exclude_tables)) - throw TypeError(".vtctldata.ValidateSchemaKeyspaceRequest.exclude_tables: array expected"); + throw TypeError(".vtctldata.ValidateVSchemaRequest.exclude_tables: array expected"); message.exclude_tables = []; for (let i = 0; i < object.exclude_tables.length; ++i) message.exclude_tables[i] = String(object.exclude_tables[i]); } if (object.include_views != null) message.include_views = Boolean(object.include_views); - if (object.skip_no_primary != null) - message.skip_no_primary = Boolean(object.skip_no_primary); - if (object.include_vschema != null) - message.include_vschema = Boolean(object.include_vschema); return message; }; /** - * Creates a plain object from a ValidateSchemaKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVSchemaRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static - * @param {vtctldata.ValidateSchemaKeyspaceRequest} message ValidateSchemaKeyspaceRequest + * @param {vtctldata.ValidateVSchemaRequest} message ValidateVSchemaRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateSchemaKeyspaceRequest.toObject = function toObject(message, options) { + ValidateVSchemaRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) + if (options.arrays || options.defaults) { + object.shards = []; object.exclude_tables = []; + } if (options.defaults) { object.keyspace = ""; object.include_views = false; - object.skip_no_primary = false; - object.include_vschema = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; + if (message.shards && message.shards.length) { + object.shards = []; + for (let j = 0; j < message.shards.length; ++j) + object.shards[j] = message.shards[j]; + } if (message.exclude_tables && message.exclude_tables.length) { object.exclude_tables = []; for (let j = 0; j < message.exclude_tables.length; ++j) @@ -140875,61 +147622,57 @@ export const vtctldata = $root.vtctldata = (() => { } if (message.include_views != null && message.hasOwnProperty("include_views")) object.include_views = message.include_views; - if (message.skip_no_primary != null && message.hasOwnProperty("skip_no_primary")) - object.skip_no_primary = message.skip_no_primary; - if (message.include_vschema != null && message.hasOwnProperty("include_vschema")) - object.include_vschema = message.include_vschema; return object; }; /** - * Converts this ValidateSchemaKeyspaceRequest to JSON. + * Converts this ValidateVSchemaRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @instance * @returns {Object.} JSON object */ - ValidateSchemaKeyspaceRequest.prototype.toJSON = function toJSON() { + ValidateVSchemaRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateSchemaKeyspaceRequest + * Gets the default type url for ValidateVSchemaRequest * @function getTypeUrl - * @memberof vtctldata.ValidateSchemaKeyspaceRequest + * @memberof vtctldata.ValidateVSchemaRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateSchemaKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateSchemaKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.ValidateVSchemaRequest"; }; - return ValidateSchemaKeyspaceRequest; + return ValidateVSchemaRequest; })(); - vtctldata.ValidateSchemaKeyspaceResponse = (function() { + vtctldata.ValidateVSchemaResponse = (function() { /** - * Properties of a ValidateSchemaKeyspaceResponse. + * Properties of a ValidateVSchemaResponse. * @memberof vtctldata - * @interface IValidateSchemaKeyspaceResponse - * @property {Array.|null} [results] ValidateSchemaKeyspaceResponse results - * @property {Object.|null} [results_by_shard] ValidateSchemaKeyspaceResponse results_by_shard + * @interface IValidateVSchemaResponse + * @property {Array.|null} [results] ValidateVSchemaResponse results + * @property {Object.|null} [results_by_shard] ValidateVSchemaResponse results_by_shard */ /** - * Constructs a new ValidateSchemaKeyspaceResponse. + * Constructs a new ValidateVSchemaResponse. * @memberof vtctldata - * @classdesc Represents a ValidateSchemaKeyspaceResponse. - * @implements IValidateSchemaKeyspaceResponse + * @classdesc Represents a ValidateVSchemaResponse. + * @implements IValidateVSchemaResponse * @constructor - * @param {vtctldata.IValidateSchemaKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IValidateVSchemaResponse=} [properties] Properties to set */ - function ValidateSchemaKeyspaceResponse(properties) { + function ValidateVSchemaResponse(properties) { this.results = []; this.results_by_shard = {}; if (properties) @@ -140939,43 +147682,43 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateSchemaKeyspaceResponse results. + * ValidateVSchemaResponse results. * @member {Array.} results - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @instance */ - ValidateSchemaKeyspaceResponse.prototype.results = $util.emptyArray; + ValidateVSchemaResponse.prototype.results = $util.emptyArray; /** - * ValidateSchemaKeyspaceResponse results_by_shard. + * ValidateVSchemaResponse results_by_shard. * @member {Object.} results_by_shard - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @instance */ - ValidateSchemaKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; + ValidateVSchemaResponse.prototype.results_by_shard = $util.emptyObject; /** - * Creates a new ValidateSchemaKeyspaceResponse instance using the specified properties. + * Creates a new ValidateVSchemaResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static - * @param {vtctldata.IValidateSchemaKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse instance + * @param {vtctldata.IValidateVSchemaResponse=} [properties] Properties to set + * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse instance */ - ValidateSchemaKeyspaceResponse.create = function create(properties) { - return new ValidateSchemaKeyspaceResponse(properties); + ValidateVSchemaResponse.create = function create(properties) { + return new ValidateVSchemaResponse(properties); }; /** - * Encodes the specified ValidateSchemaKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. + * Encodes the specified ValidateVSchemaResponse message. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static - * @param {vtctldata.IValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse message or plain object to encode + * @param {vtctldata.IValidateVSchemaResponse} message ValidateVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateSchemaKeyspaceResponse.encode = function encode(message, writer) { + ValidateVSchemaResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.results != null && message.results.length) @@ -140990,33 +147733,33 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Encodes the specified ValidateSchemaKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateSchemaKeyspaceResponse.verify|verify} messages. + * Encodes the specified ValidateVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static - * @param {vtctldata.IValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse message or plain object to encode + * @param {vtctldata.IValidateVSchemaResponse} message ValidateVSchemaResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateSchemaKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + ValidateVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer. + * Decodes a ValidateVSchemaResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse + * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateSchemaKeyspaceResponse.decode = function decode(reader, length) { + ValidateVSchemaResponse.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.ValidateSchemaKeyspaceResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.ValidateVSchemaResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -141058,30 +147801,30 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateSchemaKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a ValidateVSchemaResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse + * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateSchemaKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + ValidateVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateSchemaKeyspaceResponse message. + * Verifies a ValidateVSchemaResponse message. * @function verify - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateSchemaKeyspaceResponse.verify = function verify(message) { + ValidateVSchemaResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.results != null && message.hasOwnProperty("results")) { @@ -141105,31 +147848,31 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Creates a ValidateSchemaKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a ValidateVSchemaResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateSchemaKeyspaceResponse} ValidateSchemaKeyspaceResponse + * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse */ - ValidateSchemaKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateSchemaKeyspaceResponse) + ValidateVSchemaResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ValidateVSchemaResponse) return object; - let message = new $root.vtctldata.ValidateSchemaKeyspaceResponse(); + let message = new $root.vtctldata.ValidateVSchemaResponse(); if (object.results) { if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results: array expected"); + throw TypeError(".vtctldata.ValidateVSchemaResponse.results: array expected"); message.results = []; for (let i = 0; i < object.results.length; ++i) message.results[i] = String(object.results[i]); } if (object.results_by_shard) { if (typeof object.results_by_shard !== "object") - throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard: object expected"); + throw TypeError(".vtctldata.ValidateVSchemaResponse.results_by_shard: object expected"); message.results_by_shard = {}; for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { if (typeof object.results_by_shard[keys[i]] !== "object") - throw TypeError(".vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard: object expected"); + throw TypeError(".vtctldata.ValidateVSchemaResponse.results_by_shard: object expected"); message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); } } @@ -141137,15 +147880,15 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Creates a plain object from a ValidateSchemaKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a ValidateVSchemaResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static - * @param {vtctldata.ValidateSchemaKeyspaceResponse} message ValidateSchemaKeyspaceResponse + * @param {vtctldata.ValidateVSchemaResponse} message ValidateVSchemaResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateSchemaKeyspaceResponse.toObject = function toObject(message, options) { + ValidateVSchemaResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -141168,54 +147911,55 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Converts this ValidateSchemaKeyspaceResponse to JSON. + * Converts this ValidateVSchemaResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @instance * @returns {Object.} JSON object */ - ValidateSchemaKeyspaceResponse.prototype.toJSON = function toJSON() { + ValidateVSchemaResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateSchemaKeyspaceResponse + * Gets the default type url for ValidateVSchemaResponse * @function getTypeUrl - * @memberof vtctldata.ValidateSchemaKeyspaceResponse + * @memberof vtctldata.ValidateVSchemaResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateSchemaKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + ValidateVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateSchemaKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.ValidateVSchemaResponse"; }; - return ValidateSchemaKeyspaceResponse; + return ValidateVSchemaResponse; })(); - vtctldata.ValidateShardRequest = (function() { + vtctldata.WorkflowDeleteRequest = (function() { /** - * Properties of a ValidateShardRequest. + * Properties of a WorkflowDeleteRequest. * @memberof vtctldata - * @interface IValidateShardRequest - * @property {string|null} [keyspace] ValidateShardRequest keyspace - * @property {string|null} [shard] ValidateShardRequest shard - * @property {boolean|null} [ping_tablets] ValidateShardRequest ping_tablets + * @interface IWorkflowDeleteRequest + * @property {string|null} [keyspace] WorkflowDeleteRequest keyspace + * @property {string|null} [workflow] WorkflowDeleteRequest workflow + * @property {boolean|null} [keep_data] WorkflowDeleteRequest keep_data + * @property {boolean|null} [keep_routing_rules] WorkflowDeleteRequest keep_routing_rules */ /** - * Constructs a new ValidateShardRequest. + * Constructs a new WorkflowDeleteRequest. * @memberof vtctldata - * @classdesc Represents a ValidateShardRequest. - * @implements IValidateShardRequest + * @classdesc Represents a WorkflowDeleteRequest. + * @implements IWorkflowDeleteRequest * @constructor - * @param {vtctldata.IValidateShardRequest=} [properties] Properties to set + * @param {vtctldata.IWorkflowDeleteRequest=} [properties] Properties to set */ - function ValidateShardRequest(properties) { + function WorkflowDeleteRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -141223,90 +147967,100 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateShardRequest keyspace. + * WorkflowDeleteRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @instance */ - ValidateShardRequest.prototype.keyspace = ""; + WorkflowDeleteRequest.prototype.keyspace = ""; /** - * ValidateShardRequest shard. - * @member {string} shard - * @memberof vtctldata.ValidateShardRequest + * WorkflowDeleteRequest workflow. + * @member {string} workflow + * @memberof vtctldata.WorkflowDeleteRequest * @instance */ - ValidateShardRequest.prototype.shard = ""; + WorkflowDeleteRequest.prototype.workflow = ""; /** - * ValidateShardRequest ping_tablets. - * @member {boolean} ping_tablets - * @memberof vtctldata.ValidateShardRequest + * WorkflowDeleteRequest keep_data. + * @member {boolean} keep_data + * @memberof vtctldata.WorkflowDeleteRequest * @instance */ - ValidateShardRequest.prototype.ping_tablets = false; + WorkflowDeleteRequest.prototype.keep_data = false; /** - * Creates a new ValidateShardRequest instance using the specified properties. + * WorkflowDeleteRequest keep_routing_rules. + * @member {boolean} keep_routing_rules + * @memberof vtctldata.WorkflowDeleteRequest + * @instance + */ + WorkflowDeleteRequest.prototype.keep_routing_rules = false; + + /** + * Creates a new WorkflowDeleteRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static - * @param {vtctldata.IValidateShardRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest instance + * @param {vtctldata.IWorkflowDeleteRequest=} [properties] Properties to set + * @returns {vtctldata.WorkflowDeleteRequest} WorkflowDeleteRequest instance */ - ValidateShardRequest.create = function create(properties) { - return new ValidateShardRequest(properties); + WorkflowDeleteRequest.create = function create(properties) { + return new WorkflowDeleteRequest(properties); }; /** - * Encodes the specified ValidateShardRequest message. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. + * Encodes the specified WorkflowDeleteRequest message. Does not implicitly {@link vtctldata.WorkflowDeleteRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static - * @param {vtctldata.IValidateShardRequest} message ValidateShardRequest message or plain object to encode + * @param {vtctldata.IWorkflowDeleteRequest} message WorkflowDeleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateShardRequest.encode = function encode(message, writer) { + WorkflowDeleteRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - if (message.ping_tablets != null && Object.hasOwnProperty.call(message, "ping_tablets")) - writer.uint32(/* id 3, wireType 0 =*/24).bool(message.ping_tablets); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.keep_data != null && Object.hasOwnProperty.call(message, "keep_data")) + writer.uint32(/* id 3, wireType 0 =*/24).bool(message.keep_data); + if (message.keep_routing_rules != null && Object.hasOwnProperty.call(message, "keep_routing_rules")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.keep_routing_rules); return writer; }; /** - * Encodes the specified ValidateShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateShardRequest.verify|verify} messages. + * Encodes the specified WorkflowDeleteRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static - * @param {vtctldata.IValidateShardRequest} message ValidateShardRequest message or plain object to encode + * @param {vtctldata.IWorkflowDeleteRequest} message WorkflowDeleteRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateShardRequest.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowDeleteRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateShardRequest message from the specified reader or buffer. + * Decodes a WorkflowDeleteRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest + * @returns {vtctldata.WorkflowDeleteRequest} WorkflowDeleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateShardRequest.decode = function decode(reader, length) { + WorkflowDeleteRequest.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.ValidateShardRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowDeleteRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -141315,11 +148069,15 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.shard = reader.string(); + message.workflow = reader.string(); break; } case 3: { - message.ping_tablets = reader.bool(); + message.keep_data = reader.bool(); + break; + } + case 4: { + message.keep_routing_rules = reader.bool(); break; } default: @@ -141331,140 +148089,149 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateShardRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowDeleteRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest + * @returns {vtctldata.WorkflowDeleteRequest} WorkflowDeleteRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateShardRequest.decodeDelimited = function decodeDelimited(reader) { + WorkflowDeleteRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateShardRequest message. + * Verifies a WorkflowDeleteRequest message. * @function verify - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateShardRequest.verify = function verify(message) { + WorkflowDeleteRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - if (typeof message.ping_tablets !== "boolean") - return "ping_tablets: boolean expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.keep_data != null && message.hasOwnProperty("keep_data")) + if (typeof message.keep_data !== "boolean") + return "keep_data: boolean expected"; + if (message.keep_routing_rules != null && message.hasOwnProperty("keep_routing_rules")) + if (typeof message.keep_routing_rules !== "boolean") + return "keep_routing_rules: boolean expected"; return null; }; /** - * Creates a ValidateShardRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowDeleteRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateShardRequest} ValidateShardRequest + * @returns {vtctldata.WorkflowDeleteRequest} WorkflowDeleteRequest */ - ValidateShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateShardRequest) + WorkflowDeleteRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowDeleteRequest) return object; - let message = new $root.vtctldata.ValidateShardRequest(); + let message = new $root.vtctldata.WorkflowDeleteRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - if (object.ping_tablets != null) - message.ping_tablets = Boolean(object.ping_tablets); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.keep_data != null) + message.keep_data = Boolean(object.keep_data); + if (object.keep_routing_rules != null) + message.keep_routing_rules = Boolean(object.keep_routing_rules); return message; }; /** - * Creates a plain object from a ValidateShardRequest message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowDeleteRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static - * @param {vtctldata.ValidateShardRequest} message ValidateShardRequest + * @param {vtctldata.WorkflowDeleteRequest} message WorkflowDeleteRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateShardRequest.toObject = function toObject(message, options) { + WorkflowDeleteRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { object.keyspace = ""; - object.shard = ""; - object.ping_tablets = false; + object.workflow = ""; + object.keep_data = false; + object.keep_routing_rules = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - if (message.ping_tablets != null && message.hasOwnProperty("ping_tablets")) - object.ping_tablets = message.ping_tablets; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.keep_data != null && message.hasOwnProperty("keep_data")) + object.keep_data = message.keep_data; + if (message.keep_routing_rules != null && message.hasOwnProperty("keep_routing_rules")) + object.keep_routing_rules = message.keep_routing_rules; return object; }; /** - * Converts this ValidateShardRequest to JSON. + * Converts this WorkflowDeleteRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @instance * @returns {Object.} JSON object */ - ValidateShardRequest.prototype.toJSON = function toJSON() { + WorkflowDeleteRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateShardRequest + * Gets the default type url for WorkflowDeleteRequest * @function getTypeUrl - * @memberof vtctldata.ValidateShardRequest + * @memberof vtctldata.WorkflowDeleteRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowDeleteRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateShardRequest"; + return typeUrlPrefix + "/vtctldata.WorkflowDeleteRequest"; }; - return ValidateShardRequest; + return WorkflowDeleteRequest; })(); - vtctldata.ValidateShardResponse = (function() { + vtctldata.WorkflowDeleteResponse = (function() { /** - * Properties of a ValidateShardResponse. + * Properties of a WorkflowDeleteResponse. * @memberof vtctldata - * @interface IValidateShardResponse - * @property {Array.|null} [results] ValidateShardResponse results + * @interface IWorkflowDeleteResponse + * @property {string|null} [summary] WorkflowDeleteResponse summary + * @property {Array.|null} [details] WorkflowDeleteResponse details */ /** - * Constructs a new ValidateShardResponse. + * Constructs a new WorkflowDeleteResponse. * @memberof vtctldata - * @classdesc Represents a ValidateShardResponse. - * @implements IValidateShardResponse + * @classdesc Represents a WorkflowDeleteResponse. + * @implements IWorkflowDeleteResponse * @constructor - * @param {vtctldata.IValidateShardResponse=} [properties] Properties to set + * @param {vtctldata.IWorkflowDeleteResponse=} [properties] Properties to set */ - function ValidateShardResponse(properties) { - this.results = []; + function WorkflowDeleteResponse(properties) { + this.details = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -141472,78 +148239,92 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateShardResponse results. - * @member {Array.} results - * @memberof vtctldata.ValidateShardResponse + * WorkflowDeleteResponse summary. + * @member {string} summary + * @memberof vtctldata.WorkflowDeleteResponse * @instance */ - ValidateShardResponse.prototype.results = $util.emptyArray; + WorkflowDeleteResponse.prototype.summary = ""; /** - * Creates a new ValidateShardResponse instance using the specified properties. + * WorkflowDeleteResponse details. + * @member {Array.} details + * @memberof vtctldata.WorkflowDeleteResponse + * @instance + */ + WorkflowDeleteResponse.prototype.details = $util.emptyArray; + + /** + * Creates a new WorkflowDeleteResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static - * @param {vtctldata.IValidateShardResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse instance + * @param {vtctldata.IWorkflowDeleteResponse=} [properties] Properties to set + * @returns {vtctldata.WorkflowDeleteResponse} WorkflowDeleteResponse instance */ - ValidateShardResponse.create = function create(properties) { - return new ValidateShardResponse(properties); + WorkflowDeleteResponse.create = function create(properties) { + return new WorkflowDeleteResponse(properties); }; /** - * Encodes the specified ValidateShardResponse message. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. + * Encodes the specified WorkflowDeleteResponse message. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static - * @param {vtctldata.IValidateShardResponse} message ValidateShardResponse message or plain object to encode + * @param {vtctldata.IWorkflowDeleteResponse} message WorkflowDeleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateShardResponse.encode = function encode(message, writer) { + WorkflowDeleteResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.results != null && message.results.length) - for (let i = 0; i < message.results.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); + if (message.summary != null && Object.hasOwnProperty.call(message, "summary")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.summary); + if (message.details != null && message.details.length) + for (let i = 0; i < message.details.length; ++i) + $root.vtctldata.WorkflowDeleteResponse.TabletInfo.encode(message.details[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; /** - * Encodes the specified ValidateShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateShardResponse.verify|verify} messages. + * Encodes the specified WorkflowDeleteResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static - * @param {vtctldata.IValidateShardResponse} message ValidateShardResponse message or plain object to encode + * @param {vtctldata.IWorkflowDeleteResponse} message WorkflowDeleteResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateShardResponse.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowDeleteResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateShardResponse message from the specified reader or buffer. + * Decodes a WorkflowDeleteResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse + * @returns {vtctldata.WorkflowDeleteResponse} WorkflowDeleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateShardResponse.decode = function decode(reader, length) { + WorkflowDeleteResponse.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.ValidateShardResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowDeleteResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.results && message.results.length)) - message.results = []; - message.results.push(reader.string()); + message.summary = reader.string(); + break; + } + case 2: { + if (!(message.details && message.details.length)) + message.details = []; + message.details.push($root.vtctldata.WorkflowDeleteResponse.TabletInfo.decode(reader, reader.uint32())); break; } default: @@ -141555,134 +148336,381 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateShardResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowDeleteResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse + * @returns {vtctldata.WorkflowDeleteResponse} WorkflowDeleteResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateShardResponse.decodeDelimited = function decodeDelimited(reader) { + WorkflowDeleteResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateShardResponse message. + * Verifies a WorkflowDeleteResponse message. * @function verify - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateShardResponse.verify = function verify(message) { + WorkflowDeleteResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.results != null && message.hasOwnProperty("results")) { - if (!Array.isArray(message.results)) - return "results: array expected"; - for (let i = 0; i < message.results.length; ++i) - if (!$util.isString(message.results[i])) - return "results: string[] expected"; + if (message.summary != null && message.hasOwnProperty("summary")) + if (!$util.isString(message.summary)) + return "summary: string expected"; + if (message.details != null && message.hasOwnProperty("details")) { + if (!Array.isArray(message.details)) + return "details: array expected"; + for (let i = 0; i < message.details.length; ++i) { + let error = $root.vtctldata.WorkflowDeleteResponse.TabletInfo.verify(message.details[i]); + if (error) + return "details." + error; + } } return null; }; /** - * Creates a ValidateShardResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowDeleteResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateShardResponse} ValidateShardResponse + * @returns {vtctldata.WorkflowDeleteResponse} WorkflowDeleteResponse */ - ValidateShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateShardResponse) + WorkflowDeleteResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowDeleteResponse) return object; - let message = new $root.vtctldata.ValidateShardResponse(); - if (object.results) { - if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateShardResponse.results: array expected"); - message.results = []; - for (let i = 0; i < object.results.length; ++i) - message.results[i] = String(object.results[i]); + let message = new $root.vtctldata.WorkflowDeleteResponse(); + if (object.summary != null) + message.summary = String(object.summary); + if (object.details) { + if (!Array.isArray(object.details)) + throw TypeError(".vtctldata.WorkflowDeleteResponse.details: array expected"); + message.details = []; + for (let i = 0; i < object.details.length; ++i) { + if (typeof object.details[i] !== "object") + throw TypeError(".vtctldata.WorkflowDeleteResponse.details: object expected"); + message.details[i] = $root.vtctldata.WorkflowDeleteResponse.TabletInfo.fromObject(object.details[i]); + } } return message; }; /** - * Creates a plain object from a ValidateShardResponse message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowDeleteResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static - * @param {vtctldata.ValidateShardResponse} message ValidateShardResponse + * @param {vtctldata.WorkflowDeleteResponse} message WorkflowDeleteResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateShardResponse.toObject = function toObject(message, options) { + WorkflowDeleteResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.results = []; - if (message.results && message.results.length) { - object.results = []; - for (let j = 0; j < message.results.length; ++j) - object.results[j] = message.results[j]; + object.details = []; + if (options.defaults) + object.summary = ""; + if (message.summary != null && message.hasOwnProperty("summary")) + object.summary = message.summary; + if (message.details && message.details.length) { + object.details = []; + for (let j = 0; j < message.details.length; ++j) + object.details[j] = $root.vtctldata.WorkflowDeleteResponse.TabletInfo.toObject(message.details[j], options); } return object; }; /** - * Converts this ValidateShardResponse to JSON. + * Converts this WorkflowDeleteResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @instance * @returns {Object.} JSON object */ - ValidateShardResponse.prototype.toJSON = function toJSON() { + WorkflowDeleteResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateShardResponse + * Gets the default type url for WorkflowDeleteResponse * @function getTypeUrl - * @memberof vtctldata.ValidateShardResponse + * @memberof vtctldata.WorkflowDeleteResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowDeleteResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateShardResponse"; + return typeUrlPrefix + "/vtctldata.WorkflowDeleteResponse"; }; - return ValidateShardResponse; + WorkflowDeleteResponse.TabletInfo = (function() { + + /** + * Properties of a TabletInfo. + * @memberof vtctldata.WorkflowDeleteResponse + * @interface ITabletInfo + * @property {topodata.ITabletAlias|null} [tablet] TabletInfo tablet + * @property {boolean|null} [deleted] TabletInfo deleted + */ + + /** + * Constructs a new TabletInfo. + * @memberof vtctldata.WorkflowDeleteResponse + * @classdesc Represents a TabletInfo. + * @implements ITabletInfo + * @constructor + * @param {vtctldata.WorkflowDeleteResponse.ITabletInfo=} [properties] Properties to set + */ + function TabletInfo(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]]; + } + + /** + * TabletInfo tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @instance + */ + TabletInfo.prototype.tablet = null; + + /** + * TabletInfo deleted. + * @member {boolean} deleted + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @instance + */ + TabletInfo.prototype.deleted = false; + + /** + * Creates a new TabletInfo instance using the specified properties. + * @function create + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {vtctldata.WorkflowDeleteResponse.ITabletInfo=} [properties] Properties to set + * @returns {vtctldata.WorkflowDeleteResponse.TabletInfo} TabletInfo instance + */ + TabletInfo.create = function create(properties) { + return new TabletInfo(properties); + }; + + /** + * Encodes the specified TabletInfo message. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.TabletInfo.verify|verify} messages. + * @function encode + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {vtctldata.WorkflowDeleteResponse.ITabletInfo} message TabletInfo message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TabletInfo.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.deleted != null && Object.hasOwnProperty.call(message, "deleted")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.deleted); + return writer; + }; + + /** + * Encodes the specified TabletInfo message, length delimited. Does not implicitly {@link vtctldata.WorkflowDeleteResponse.TabletInfo.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {vtctldata.WorkflowDeleteResponse.ITabletInfo} message TabletInfo message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TabletInfo.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a TabletInfo message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.WorkflowDeleteResponse.TabletInfo} TabletInfo + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TabletInfo.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.WorkflowDeleteResponse.TabletInfo(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 2: { + message.deleted = reader.bool(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a TabletInfo message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.WorkflowDeleteResponse.TabletInfo} TabletInfo + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TabletInfo.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a TabletInfo message. + * @function verify + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + TabletInfo.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } + if (message.deleted != null && message.hasOwnProperty("deleted")) + if (typeof message.deleted !== "boolean") + return "deleted: boolean expected"; + return null; + }; + + /** + * Creates a TabletInfo message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.WorkflowDeleteResponse.TabletInfo} TabletInfo + */ + TabletInfo.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowDeleteResponse.TabletInfo) + return object; + let message = new $root.vtctldata.WorkflowDeleteResponse.TabletInfo(); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.WorkflowDeleteResponse.TabletInfo.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } + if (object.deleted != null) + message.deleted = Boolean(object.deleted); + return message; + }; + + /** + * Creates a plain object from a TabletInfo message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {vtctldata.WorkflowDeleteResponse.TabletInfo} message TabletInfo + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + TabletInfo.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tablet = null; + object.deleted = false; + } + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (message.deleted != null && message.hasOwnProperty("deleted")) + object.deleted = message.deleted; + return object; + }; + + /** + * Converts this TabletInfo to JSON. + * @function toJSON + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @instance + * @returns {Object.} JSON object + */ + TabletInfo.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for TabletInfo + * @function getTypeUrl + * @memberof vtctldata.WorkflowDeleteResponse.TabletInfo + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + TabletInfo.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.WorkflowDeleteResponse.TabletInfo"; + }; + + return TabletInfo; + })(); + + return WorkflowDeleteResponse; })(); - vtctldata.ValidateVersionKeyspaceRequest = (function() { + vtctldata.WorkflowStatusRequest = (function() { /** - * Properties of a ValidateVersionKeyspaceRequest. + * Properties of a WorkflowStatusRequest. * @memberof vtctldata - * @interface IValidateVersionKeyspaceRequest - * @property {string|null} [keyspace] ValidateVersionKeyspaceRequest keyspace + * @interface IWorkflowStatusRequest + * @property {string|null} [keyspace] WorkflowStatusRequest keyspace + * @property {string|null} [workflow] WorkflowStatusRequest workflow */ /** - * Constructs a new ValidateVersionKeyspaceRequest. + * Constructs a new WorkflowStatusRequest. * @memberof vtctldata - * @classdesc Represents a ValidateVersionKeyspaceRequest. - * @implements IValidateVersionKeyspaceRequest + * @classdesc Represents a WorkflowStatusRequest. + * @implements IWorkflowStatusRequest * @constructor - * @param {vtctldata.IValidateVersionKeyspaceRequest=} [properties] Properties to set + * @param {vtctldata.IWorkflowStatusRequest=} [properties] Properties to set */ - function ValidateVersionKeyspaceRequest(properties) { + function WorkflowStatusRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -141690,70 +148718,80 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateVersionKeyspaceRequest keyspace. + * WorkflowStatusRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @instance */ - ValidateVersionKeyspaceRequest.prototype.keyspace = ""; + WorkflowStatusRequest.prototype.keyspace = ""; /** - * Creates a new ValidateVersionKeyspaceRequest instance using the specified properties. + * WorkflowStatusRequest workflow. + * @member {string} workflow + * @memberof vtctldata.WorkflowStatusRequest + * @instance + */ + WorkflowStatusRequest.prototype.workflow = ""; + + /** + * Creates a new WorkflowStatusRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static - * @param {vtctldata.IValidateVersionKeyspaceRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest instance + * @param {vtctldata.IWorkflowStatusRequest=} [properties] Properties to set + * @returns {vtctldata.WorkflowStatusRequest} WorkflowStatusRequest instance */ - ValidateVersionKeyspaceRequest.create = function create(properties) { - return new ValidateVersionKeyspaceRequest(properties); + WorkflowStatusRequest.create = function create(properties) { + return new WorkflowStatusRequest(properties); }; /** - * Encodes the specified ValidateVersionKeyspaceRequest message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. + * Encodes the specified WorkflowStatusRequest message. Does not implicitly {@link vtctldata.WorkflowStatusRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static - * @param {vtctldata.IValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest message or plain object to encode + * @param {vtctldata.IWorkflowStatusRequest} message WorkflowStatusRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVersionKeyspaceRequest.encode = function encode(message, writer) { + WorkflowStatusRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); return writer; }; /** - * Encodes the specified ValidateVersionKeyspaceRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceRequest.verify|verify} messages. + * Encodes the specified WorkflowStatusRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static - * @param {vtctldata.IValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest message or plain object to encode + * @param {vtctldata.IWorkflowStatusRequest} message WorkflowStatusRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVersionKeyspaceRequest.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowStatusRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer. + * Decodes a WorkflowStatusRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest + * @returns {vtctldata.WorkflowStatusRequest} WorkflowStatusRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVersionKeyspaceRequest.decode = function decode(reader, length) { + WorkflowStatusRequest.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.ValidateVersionKeyspaceRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowStatusRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -141761,6 +148799,10 @@ export const vtctldata = $root.vtctldata = (() => { message.keyspace = reader.string(); break; } + case 2: { + message.workflow = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -141770,125 +148812,134 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateVersionKeyspaceRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowStatusRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest + * @returns {vtctldata.WorkflowStatusRequest} WorkflowStatusRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVersionKeyspaceRequest.decodeDelimited = function decodeDelimited(reader) { + WorkflowStatusRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateVersionKeyspaceRequest message. + * Verifies a WorkflowStatusRequest message. * @function verify - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateVersionKeyspaceRequest.verify = function verify(message) { + WorkflowStatusRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; return null; }; /** - * Creates a ValidateVersionKeyspaceRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowStatusRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVersionKeyspaceRequest} ValidateVersionKeyspaceRequest + * @returns {vtctldata.WorkflowStatusRequest} WorkflowStatusRequest */ - ValidateVersionKeyspaceRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVersionKeyspaceRequest) + WorkflowStatusRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowStatusRequest) return object; - let message = new $root.vtctldata.ValidateVersionKeyspaceRequest(); + let message = new $root.vtctldata.WorkflowStatusRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); + if (object.workflow != null) + message.workflow = String(object.workflow); return message; }; /** - * Creates a plain object from a ValidateVersionKeyspaceRequest message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowStatusRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static - * @param {vtctldata.ValidateVersionKeyspaceRequest} message ValidateVersionKeyspaceRequest + * @param {vtctldata.WorkflowStatusRequest} message WorkflowStatusRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateVersionKeyspaceRequest.toObject = function toObject(message, options) { + WorkflowStatusRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) + if (options.defaults) { object.keyspace = ""; + object.workflow = ""; + } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; return object; }; /** - * Converts this ValidateVersionKeyspaceRequest to JSON. + * Converts this WorkflowStatusRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @instance * @returns {Object.} JSON object */ - ValidateVersionKeyspaceRequest.prototype.toJSON = function toJSON() { + WorkflowStatusRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateVersionKeyspaceRequest + * Gets the default type url for WorkflowStatusRequest * @function getTypeUrl - * @memberof vtctldata.ValidateVersionKeyspaceRequest + * @memberof vtctldata.WorkflowStatusRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateVersionKeyspaceRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowStatusRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateVersionKeyspaceRequest"; + return typeUrlPrefix + "/vtctldata.WorkflowStatusRequest"; }; - return ValidateVersionKeyspaceRequest; + return WorkflowStatusRequest; })(); - vtctldata.ValidateVersionKeyspaceResponse = (function() { + vtctldata.WorkflowStatusResponse = (function() { /** - * Properties of a ValidateVersionKeyspaceResponse. + * Properties of a WorkflowStatusResponse. * @memberof vtctldata - * @interface IValidateVersionKeyspaceResponse - * @property {Array.|null} [results] ValidateVersionKeyspaceResponse results - * @property {Object.|null} [results_by_shard] ValidateVersionKeyspaceResponse results_by_shard + * @interface IWorkflowStatusResponse + * @property {Object.|null} [table_copy_state] WorkflowStatusResponse table_copy_state + * @property {Object.|null} [shard_streams] WorkflowStatusResponse shard_streams */ /** - * Constructs a new ValidateVersionKeyspaceResponse. + * Constructs a new WorkflowStatusResponse. * @memberof vtctldata - * @classdesc Represents a ValidateVersionKeyspaceResponse. - * @implements IValidateVersionKeyspaceResponse + * @classdesc Represents a WorkflowStatusResponse. + * @implements IWorkflowStatusResponse * @constructor - * @param {vtctldata.IValidateVersionKeyspaceResponse=} [properties] Properties to set + * @param {vtctldata.IWorkflowStatusResponse=} [properties] Properties to set */ - function ValidateVersionKeyspaceResponse(properties) { - this.results = []; - this.results_by_shard = {}; + function WorkflowStatusResponse(properties) { + this.table_copy_state = {}; + this.shard_streams = {}; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -141896,96 +148947,115 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateVersionKeyspaceResponse results. - * @member {Array.} results - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * WorkflowStatusResponse table_copy_state. + * @member {Object.} table_copy_state + * @memberof vtctldata.WorkflowStatusResponse * @instance */ - ValidateVersionKeyspaceResponse.prototype.results = $util.emptyArray; + WorkflowStatusResponse.prototype.table_copy_state = $util.emptyObject; /** - * ValidateVersionKeyspaceResponse results_by_shard. - * @member {Object.} results_by_shard - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * WorkflowStatusResponse shard_streams. + * @member {Object.} shard_streams + * @memberof vtctldata.WorkflowStatusResponse * @instance */ - ValidateVersionKeyspaceResponse.prototype.results_by_shard = $util.emptyObject; + WorkflowStatusResponse.prototype.shard_streams = $util.emptyObject; /** - * Creates a new ValidateVersionKeyspaceResponse instance using the specified properties. + * Creates a new WorkflowStatusResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static - * @param {vtctldata.IValidateVersionKeyspaceResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse instance + * @param {vtctldata.IWorkflowStatusResponse=} [properties] Properties to set + * @returns {vtctldata.WorkflowStatusResponse} WorkflowStatusResponse instance */ - ValidateVersionKeyspaceResponse.create = function create(properties) { - return new ValidateVersionKeyspaceResponse(properties); + WorkflowStatusResponse.create = function create(properties) { + return new WorkflowStatusResponse(properties); }; /** - * Encodes the specified ValidateVersionKeyspaceResponse message. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. + * Encodes the specified WorkflowStatusResponse message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static - * @param {vtctldata.IValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse message or plain object to encode + * @param {vtctldata.IWorkflowStatusResponse} message WorkflowStatusResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVersionKeyspaceResponse.encode = function encode(message, writer) { + WorkflowStatusResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.results != null && message.results.length) - for (let i = 0; i < message.results.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); - if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) - for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { + if (message.table_copy_state != null && Object.hasOwnProperty.call(message, "table_copy_state")) + for (let keys = Object.keys(message.table_copy_state), i = 0; i < keys.length; ++i) { + writer.uint32(/* id 1, wireType 2 =*/10).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); + $root.vtctldata.WorkflowStatusResponse.TableCopyState.encode(message.table_copy_state[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + } + if (message.shard_streams != null && Object.hasOwnProperty.call(message, "shard_streams")) + for (let keys = Object.keys(message.shard_streams), i = 0; i < keys.length; ++i) { writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); + $root.vtctldata.WorkflowStatusResponse.ShardStreams.encode(message.shard_streams[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); } return writer; }; /** - * Encodes the specified ValidateVersionKeyspaceResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionKeyspaceResponse.verify|verify} messages. + * Encodes the specified WorkflowStatusResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static - * @param {vtctldata.IValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse message or plain object to encode + * @param {vtctldata.IWorkflowStatusResponse} message WorkflowStatusResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVersionKeyspaceResponse.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowStatusResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer. + * Decodes a WorkflowStatusResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse + * @returns {vtctldata.WorkflowStatusResponse} WorkflowStatusResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVersionKeyspaceResponse.decode = function decode(reader, length) { + WorkflowStatusResponse.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.ValidateVersionKeyspaceResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowStatusResponse(), key, value; while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.results && message.results.length)) - message.results = []; - message.results.push(reader.string()); + if (message.table_copy_state === $util.emptyObject) + message.table_copy_state = {}; + let end2 = reader.uint32() + reader.pos; + key = ""; + value = null; + while (reader.pos < end2) { + let tag2 = reader.uint32(); + switch (tag2 >>> 3) { + case 1: + key = reader.string(); + break; + case 2: + value = $root.vtctldata.WorkflowStatusResponse.TableCopyState.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag2 & 7); + break; + } + } + message.table_copy_state[key] = value; break; } case 2: { - if (message.results_by_shard === $util.emptyObject) - message.results_by_shard = {}; + if (message.shard_streams === $util.emptyObject) + message.shard_streams = {}; let end2 = reader.uint32() + reader.pos; key = ""; value = null; @@ -141996,14 +149066,14 @@ export const vtctldata = $root.vtctldata = (() => { key = reader.string(); break; case 2: - value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); + value = $root.vtctldata.WorkflowStatusResponse.ShardStreams.decode(reader, reader.uint32()); break; default: reader.skipType(tag2 & 7); break; } } - message.results_by_shard[key] = value; + message.shard_streams[key] = value; break; } default: @@ -142015,613 +149085,1101 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateVersionKeyspaceResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowStatusResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse + * @returns {vtctldata.WorkflowStatusResponse} WorkflowStatusResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVersionKeyspaceResponse.decodeDelimited = function decodeDelimited(reader) { + WorkflowStatusResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateVersionKeyspaceResponse message. + * Verifies a WorkflowStatusResponse message. * @function verify - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateVersionKeyspaceResponse.verify = function verify(message) { + WorkflowStatusResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.results != null && message.hasOwnProperty("results")) { - if (!Array.isArray(message.results)) - return "results: array expected"; - for (let i = 0; i < message.results.length; ++i) - if (!$util.isString(message.results[i])) - return "results: string[] expected"; + if (message.table_copy_state != null && message.hasOwnProperty("table_copy_state")) { + if (!$util.isObject(message.table_copy_state)) + return "table_copy_state: object expected"; + let key = Object.keys(message.table_copy_state); + for (let i = 0; i < key.length; ++i) { + let error = $root.vtctldata.WorkflowStatusResponse.TableCopyState.verify(message.table_copy_state[key[i]]); + if (error) + return "table_copy_state." + error; + } } - if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { - if (!$util.isObject(message.results_by_shard)) - return "results_by_shard: object expected"; - let key = Object.keys(message.results_by_shard); + if (message.shard_streams != null && message.hasOwnProperty("shard_streams")) { + if (!$util.isObject(message.shard_streams)) + return "shard_streams: object expected"; + let key = Object.keys(message.shard_streams); for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); + let error = $root.vtctldata.WorkflowStatusResponse.ShardStreams.verify(message.shard_streams[key[i]]); if (error) - return "results_by_shard." + error; + return "shard_streams." + error; } } return null; }; /** - * Creates a ValidateVersionKeyspaceResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowStatusResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVersionKeyspaceResponse} ValidateVersionKeyspaceResponse + * @returns {vtctldata.WorkflowStatusResponse} WorkflowStatusResponse */ - ValidateVersionKeyspaceResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVersionKeyspaceResponse) + WorkflowStatusResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowStatusResponse) return object; - let message = new $root.vtctldata.ValidateVersionKeyspaceResponse(); - if (object.results) { - if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results: array expected"); - message.results = []; - for (let i = 0; i < object.results.length; ++i) - message.results[i] = String(object.results[i]); + let message = new $root.vtctldata.WorkflowStatusResponse(); + if (object.table_copy_state) { + if (typeof object.table_copy_state !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.table_copy_state: object expected"); + message.table_copy_state = {}; + for (let keys = Object.keys(object.table_copy_state), i = 0; i < keys.length; ++i) { + if (typeof object.table_copy_state[keys[i]] !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.table_copy_state: object expected"); + message.table_copy_state[keys[i]] = $root.vtctldata.WorkflowStatusResponse.TableCopyState.fromObject(object.table_copy_state[keys[i]]); + } } - if (object.results_by_shard) { - if (typeof object.results_by_shard !== "object") - throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results_by_shard: object expected"); - message.results_by_shard = {}; - for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { - if (typeof object.results_by_shard[keys[i]] !== "object") - throw TypeError(".vtctldata.ValidateVersionKeyspaceResponse.results_by_shard: object expected"); - message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); + if (object.shard_streams) { + if (typeof object.shard_streams !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.shard_streams: object expected"); + message.shard_streams = {}; + for (let keys = Object.keys(object.shard_streams), i = 0; i < keys.length; ++i) { + if (typeof object.shard_streams[keys[i]] !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.shard_streams: object expected"); + message.shard_streams[keys[i]] = $root.vtctldata.WorkflowStatusResponse.ShardStreams.fromObject(object.shard_streams[keys[i]]); } } return message; }; /** - * Creates a plain object from a ValidateVersionKeyspaceResponse message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowStatusResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static - * @param {vtctldata.ValidateVersionKeyspaceResponse} message ValidateVersionKeyspaceResponse + * @param {vtctldata.WorkflowStatusResponse} message WorkflowStatusResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateVersionKeyspaceResponse.toObject = function toObject(message, options) { + WorkflowStatusResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) - object.results = []; - if (options.objects || options.defaults) - object.results_by_shard = {}; - if (message.results && message.results.length) { - object.results = []; - for (let j = 0; j < message.results.length; ++j) - object.results[j] = message.results[j]; + if (options.objects || options.defaults) { + object.table_copy_state = {}; + object.shard_streams = {}; } let keys2; - if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { - object.results_by_shard = {}; + if (message.table_copy_state && (keys2 = Object.keys(message.table_copy_state)).length) { + object.table_copy_state = {}; for (let j = 0; j < keys2.length; ++j) - object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); + object.table_copy_state[keys2[j]] = $root.vtctldata.WorkflowStatusResponse.TableCopyState.toObject(message.table_copy_state[keys2[j]], options); + } + if (message.shard_streams && (keys2 = Object.keys(message.shard_streams)).length) { + object.shard_streams = {}; + for (let j = 0; j < keys2.length; ++j) + object.shard_streams[keys2[j]] = $root.vtctldata.WorkflowStatusResponse.ShardStreams.toObject(message.shard_streams[keys2[j]], options); } return object; }; /** - * Converts this ValidateVersionKeyspaceResponse to JSON. + * Converts this WorkflowStatusResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @instance * @returns {Object.} JSON object */ - ValidateVersionKeyspaceResponse.prototype.toJSON = function toJSON() { + WorkflowStatusResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateVersionKeyspaceResponse + * Gets the default type url for WorkflowStatusResponse * @function getTypeUrl - * @memberof vtctldata.ValidateVersionKeyspaceResponse + * @memberof vtctldata.WorkflowStatusResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateVersionKeyspaceResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowStatusResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateVersionKeyspaceResponse"; + return typeUrlPrefix + "/vtctldata.WorkflowStatusResponse"; }; - return ValidateVersionKeyspaceResponse; - })(); + WorkflowStatusResponse.TableCopyState = (function() { - vtctldata.ValidateVersionShardRequest = (function() { + /** + * Properties of a TableCopyState. + * @memberof vtctldata.WorkflowStatusResponse + * @interface ITableCopyState + * @property {number|Long|null} [rows_copied] TableCopyState rows_copied + * @property {number|Long|null} [rows_total] TableCopyState rows_total + * @property {number|null} [rows_percentage] TableCopyState rows_percentage + * @property {number|Long|null} [bytes_copied] TableCopyState bytes_copied + * @property {number|Long|null} [bytes_total] TableCopyState bytes_total + * @property {number|null} [bytes_percentage] TableCopyState bytes_percentage + */ - /** - * Properties of a ValidateVersionShardRequest. - * @memberof vtctldata - * @interface IValidateVersionShardRequest - * @property {string|null} [keyspace] ValidateVersionShardRequest keyspace - * @property {string|null} [shard] ValidateVersionShardRequest shard - */ + /** + * Constructs a new TableCopyState. + * @memberof vtctldata.WorkflowStatusResponse + * @classdesc Represents a TableCopyState. + * @implements ITableCopyState + * @constructor + * @param {vtctldata.WorkflowStatusResponse.ITableCopyState=} [properties] Properties to set + */ + function TableCopyState(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]]; + } - /** - * Constructs a new ValidateVersionShardRequest. - * @memberof vtctldata - * @classdesc Represents a ValidateVersionShardRequest. - * @implements IValidateVersionShardRequest - * @constructor - * @param {vtctldata.IValidateVersionShardRequest=} [properties] Properties to set - */ - function ValidateVersionShardRequest(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]]; - } + /** + * TableCopyState rows_copied. + * @member {number|Long} rows_copied + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.rows_copied = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * ValidateVersionShardRequest keyspace. - * @member {string} keyspace - * @memberof vtctldata.ValidateVersionShardRequest - * @instance - */ - ValidateVersionShardRequest.prototype.keyspace = ""; + /** + * TableCopyState rows_total. + * @member {number|Long} rows_total + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.rows_total = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * ValidateVersionShardRequest shard. - * @member {string} shard - * @memberof vtctldata.ValidateVersionShardRequest - * @instance - */ - ValidateVersionShardRequest.prototype.shard = ""; + /** + * TableCopyState rows_percentage. + * @member {number} rows_percentage + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.rows_percentage = 0; - /** - * Creates a new ValidateVersionShardRequest instance using the specified properties. - * @function create - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {vtctldata.IValidateVersionShardRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest instance - */ - ValidateVersionShardRequest.create = function create(properties) { - return new ValidateVersionShardRequest(properties); - }; + /** + * TableCopyState bytes_copied. + * @member {number|Long} bytes_copied + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.bytes_copied = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * Encodes the specified ValidateVersionShardRequest message. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. - * @function encode - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {vtctldata.IValidateVersionShardRequest} message ValidateVersionShardRequest message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ValidateVersionShardRequest.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shard); - return writer; - }; + /** + * TableCopyState bytes_total. + * @member {number|Long} bytes_total + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.bytes_total = $util.Long ? $util.Long.fromBits(0,0,false) : 0; - /** - * Encodes the specified ValidateVersionShardRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardRequest.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {vtctldata.IValidateVersionShardRequest} message ValidateVersionShardRequest message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ValidateVersionShardRequest.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * TableCopyState bytes_percentage. + * @member {number} bytes_percentage + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + */ + TableCopyState.prototype.bytes_percentage = 0; - /** - * Decodes a ValidateVersionShardRequest message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ValidateVersionShardRequest.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.ValidateVersionShardRequest(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.keyspace = reader.string(); + /** + * Creates a new TableCopyState instance using the specified properties. + * @function create + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {vtctldata.WorkflowStatusResponse.ITableCopyState=} [properties] Properties to set + * @returns {vtctldata.WorkflowStatusResponse.TableCopyState} TableCopyState instance + */ + TableCopyState.create = function create(properties) { + return new TableCopyState(properties); + }; + + /** + * Encodes the specified TableCopyState message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.TableCopyState.verify|verify} messages. + * @function encode + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {vtctldata.WorkflowStatusResponse.ITableCopyState} message TableCopyState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TableCopyState.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.rows_copied != null && Object.hasOwnProperty.call(message, "rows_copied")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.rows_copied); + if (message.rows_total != null && Object.hasOwnProperty.call(message, "rows_total")) + writer.uint32(/* id 2, wireType 0 =*/16).int64(message.rows_total); + if (message.rows_percentage != null && Object.hasOwnProperty.call(message, "rows_percentage")) + writer.uint32(/* id 3, wireType 5 =*/29).float(message.rows_percentage); + if (message.bytes_copied != null && Object.hasOwnProperty.call(message, "bytes_copied")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.bytes_copied); + if (message.bytes_total != null && Object.hasOwnProperty.call(message, "bytes_total")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.bytes_total); + if (message.bytes_percentage != null && Object.hasOwnProperty.call(message, "bytes_percentage")) + writer.uint32(/* id 6, wireType 5 =*/53).float(message.bytes_percentage); + return writer; + }; + + /** + * Encodes the specified TableCopyState message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.TableCopyState.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {vtctldata.WorkflowStatusResponse.ITableCopyState} message TableCopyState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + TableCopyState.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a TableCopyState message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.WorkflowStatusResponse.TableCopyState} TableCopyState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TableCopyState.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.WorkflowStatusResponse.TableCopyState(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.rows_copied = reader.int64(); + break; + } + case 2: { + message.rows_total = reader.int64(); + break; + } + case 3: { + message.rows_percentage = reader.float(); + break; + } + case 4: { + message.bytes_copied = reader.int64(); + break; + } + case 5: { + message.bytes_total = reader.int64(); + break; + } + case 6: { + message.bytes_percentage = reader.float(); + break; + } + default: + reader.skipType(tag & 7); break; } - case 2: { - message.shard = reader.string(); + } + return message; + }; + + /** + * Decodes a TableCopyState message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.WorkflowStatusResponse.TableCopyState} TableCopyState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + TableCopyState.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a TableCopyState message. + * @function verify + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + TableCopyState.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) + if (!$util.isInteger(message.rows_copied) && !(message.rows_copied && $util.isInteger(message.rows_copied.low) && $util.isInteger(message.rows_copied.high))) + return "rows_copied: integer|Long expected"; + if (message.rows_total != null && message.hasOwnProperty("rows_total")) + if (!$util.isInteger(message.rows_total) && !(message.rows_total && $util.isInteger(message.rows_total.low) && $util.isInteger(message.rows_total.high))) + return "rows_total: integer|Long expected"; + if (message.rows_percentage != null && message.hasOwnProperty("rows_percentage")) + if (typeof message.rows_percentage !== "number") + return "rows_percentage: number expected"; + if (message.bytes_copied != null && message.hasOwnProperty("bytes_copied")) + if (!$util.isInteger(message.bytes_copied) && !(message.bytes_copied && $util.isInteger(message.bytes_copied.low) && $util.isInteger(message.bytes_copied.high))) + return "bytes_copied: integer|Long expected"; + if (message.bytes_total != null && message.hasOwnProperty("bytes_total")) + if (!$util.isInteger(message.bytes_total) && !(message.bytes_total && $util.isInteger(message.bytes_total.low) && $util.isInteger(message.bytes_total.high))) + return "bytes_total: integer|Long expected"; + if (message.bytes_percentage != null && message.hasOwnProperty("bytes_percentage")) + if (typeof message.bytes_percentage !== "number") + return "bytes_percentage: number expected"; + return null; + }; + + /** + * Creates a TableCopyState message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.WorkflowStatusResponse.TableCopyState} TableCopyState + */ + TableCopyState.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowStatusResponse.TableCopyState) + return object; + let message = new $root.vtctldata.WorkflowStatusResponse.TableCopyState(); + if (object.rows_copied != null) + if ($util.Long) + (message.rows_copied = $util.Long.fromValue(object.rows_copied)).unsigned = false; + else if (typeof object.rows_copied === "string") + message.rows_copied = parseInt(object.rows_copied, 10); + else if (typeof object.rows_copied === "number") + message.rows_copied = object.rows_copied; + else if (typeof object.rows_copied === "object") + message.rows_copied = new $util.LongBits(object.rows_copied.low >>> 0, object.rows_copied.high >>> 0).toNumber(); + if (object.rows_total != null) + if ($util.Long) + (message.rows_total = $util.Long.fromValue(object.rows_total)).unsigned = false; + else if (typeof object.rows_total === "string") + message.rows_total = parseInt(object.rows_total, 10); + else if (typeof object.rows_total === "number") + message.rows_total = object.rows_total; + else if (typeof object.rows_total === "object") + message.rows_total = new $util.LongBits(object.rows_total.low >>> 0, object.rows_total.high >>> 0).toNumber(); + if (object.rows_percentage != null) + message.rows_percentage = Number(object.rows_percentage); + if (object.bytes_copied != null) + if ($util.Long) + (message.bytes_copied = $util.Long.fromValue(object.bytes_copied)).unsigned = false; + else if (typeof object.bytes_copied === "string") + message.bytes_copied = parseInt(object.bytes_copied, 10); + else if (typeof object.bytes_copied === "number") + message.bytes_copied = object.bytes_copied; + else if (typeof object.bytes_copied === "object") + message.bytes_copied = new $util.LongBits(object.bytes_copied.low >>> 0, object.bytes_copied.high >>> 0).toNumber(); + if (object.bytes_total != null) + if ($util.Long) + (message.bytes_total = $util.Long.fromValue(object.bytes_total)).unsigned = false; + else if (typeof object.bytes_total === "string") + message.bytes_total = parseInt(object.bytes_total, 10); + else if (typeof object.bytes_total === "number") + message.bytes_total = object.bytes_total; + else if (typeof object.bytes_total === "object") + message.bytes_total = new $util.LongBits(object.bytes_total.low >>> 0, object.bytes_total.high >>> 0).toNumber(); + if (object.bytes_percentage != null) + message.bytes_percentage = Number(object.bytes_percentage); + return message; + }; + + /** + * Creates a plain object from a TableCopyState message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {vtctldata.WorkflowStatusResponse.TableCopyState} message TableCopyState + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + TableCopyState.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.rows_copied = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.rows_copied = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.rows_total = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.rows_total = options.longs === String ? "0" : 0; + object.rows_percentage = 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.bytes_copied = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.bytes_copied = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.bytes_total = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.bytes_total = options.longs === String ? "0" : 0; + object.bytes_percentage = 0; + } + if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) + if (typeof message.rows_copied === "number") + object.rows_copied = options.longs === String ? String(message.rows_copied) : message.rows_copied; + else + object.rows_copied = options.longs === String ? $util.Long.prototype.toString.call(message.rows_copied) : options.longs === Number ? new $util.LongBits(message.rows_copied.low >>> 0, message.rows_copied.high >>> 0).toNumber() : message.rows_copied; + if (message.rows_total != null && message.hasOwnProperty("rows_total")) + if (typeof message.rows_total === "number") + object.rows_total = options.longs === String ? String(message.rows_total) : message.rows_total; + else + object.rows_total = options.longs === String ? $util.Long.prototype.toString.call(message.rows_total) : options.longs === Number ? new $util.LongBits(message.rows_total.low >>> 0, message.rows_total.high >>> 0).toNumber() : message.rows_total; + if (message.rows_percentage != null && message.hasOwnProperty("rows_percentage")) + object.rows_percentage = options.json && !isFinite(message.rows_percentage) ? String(message.rows_percentage) : message.rows_percentage; + if (message.bytes_copied != null && message.hasOwnProperty("bytes_copied")) + if (typeof message.bytes_copied === "number") + object.bytes_copied = options.longs === String ? String(message.bytes_copied) : message.bytes_copied; + else + object.bytes_copied = options.longs === String ? $util.Long.prototype.toString.call(message.bytes_copied) : options.longs === Number ? new $util.LongBits(message.bytes_copied.low >>> 0, message.bytes_copied.high >>> 0).toNumber() : message.bytes_copied; + if (message.bytes_total != null && message.hasOwnProperty("bytes_total")) + if (typeof message.bytes_total === "number") + object.bytes_total = options.longs === String ? String(message.bytes_total) : message.bytes_total; + else + object.bytes_total = options.longs === String ? $util.Long.prototype.toString.call(message.bytes_total) : options.longs === Number ? new $util.LongBits(message.bytes_total.low >>> 0, message.bytes_total.high >>> 0).toNumber() : message.bytes_total; + if (message.bytes_percentage != null && message.hasOwnProperty("bytes_percentage")) + object.bytes_percentage = options.json && !isFinite(message.bytes_percentage) ? String(message.bytes_percentage) : message.bytes_percentage; + return object; + }; + + /** + * Converts this TableCopyState to JSON. + * @function toJSON + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @instance + * @returns {Object.} JSON object + */ + TableCopyState.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for TableCopyState + * @function getTypeUrl + * @memberof vtctldata.WorkflowStatusResponse.TableCopyState + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + TableCopyState.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.WorkflowStatusResponse.TableCopyState"; + }; + + return TableCopyState; + })(); + + WorkflowStatusResponse.ShardStreamState = (function() { + + /** + * Properties of a ShardStreamState. + * @memberof vtctldata.WorkflowStatusResponse + * @interface IShardStreamState + * @property {number|null} [id] ShardStreamState id + * @property {topodata.ITabletAlias|null} [tablet] ShardStreamState tablet + * @property {string|null} [source_shard] ShardStreamState source_shard + * @property {string|null} [position] ShardStreamState position + * @property {string|null} [status] ShardStreamState status + * @property {string|null} [info] ShardStreamState info + */ + + /** + * Constructs a new ShardStreamState. + * @memberof vtctldata.WorkflowStatusResponse + * @classdesc Represents a ShardStreamState. + * @implements IShardStreamState + * @constructor + * @param {vtctldata.WorkflowStatusResponse.IShardStreamState=} [properties] Properties to set + */ + function ShardStreamState(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]]; + } + + /** + * ShardStreamState id. + * @member {number} id + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.id = 0; + + /** + * ShardStreamState tablet. + * @member {topodata.ITabletAlias|null|undefined} tablet + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.tablet = null; + + /** + * ShardStreamState source_shard. + * @member {string} source_shard + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.source_shard = ""; + + /** + * ShardStreamState position. + * @member {string} position + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.position = ""; + + /** + * ShardStreamState status. + * @member {string} status + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.status = ""; + + /** + * ShardStreamState info. + * @member {string} info + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + */ + ShardStreamState.prototype.info = ""; + + /** + * Creates a new ShardStreamState instance using the specified properties. + * @function create + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreamState=} [properties] Properties to set + * @returns {vtctldata.WorkflowStatusResponse.ShardStreamState} ShardStreamState instance + */ + ShardStreamState.create = function create(properties) { + return new ShardStreamState(properties); + }; + + /** + * Encodes the specified ShardStreamState message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreamState.verify|verify} messages. + * @function encode + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreamState} message ShardStreamState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStreamState.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.id); + if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.source_shard != null && Object.hasOwnProperty.call(message, "source_shard")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.source_shard); + if (message.position != null && Object.hasOwnProperty.call(message, "position")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.position); + if (message.status != null && Object.hasOwnProperty.call(message, "status")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.status); + if (message.info != null && Object.hasOwnProperty.call(message, "info")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.info); + return writer; + }; + + /** + * Encodes the specified ShardStreamState message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreamState.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreamState} message ShardStreamState message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStreamState.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ShardStreamState message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.WorkflowStatusResponse.ShardStreamState} ShardStreamState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStreamState.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.WorkflowStatusResponse.ShardStreamState(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int32(); + break; + } + case 2: { + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); + break; + } + case 3: { + message.source_shard = reader.string(); + break; + } + case 4: { + message.position = reader.string(); + break; + } + case 5: { + message.status = reader.string(); + break; + } + case 6: { + message.info = reader.string(); + break; + } + default: + reader.skipType(tag & 7); break; } - default: - reader.skipType(tag & 7); - break; } - } - return message; - }; + return message; + }; - /** - * Decodes a ValidateVersionShardRequest message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ValidateVersionShardRequest.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + /** + * Decodes a ShardStreamState message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.WorkflowStatusResponse.ShardStreamState} ShardStreamState + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStreamState.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Verifies a ValidateVersionShardRequest message. - * @function verify - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - ValidateVersionShardRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.shard != null && message.hasOwnProperty("shard")) - if (!$util.isString(message.shard)) - return "shard: string expected"; - return null; - }; + /** + * Verifies a ShardStreamState message. + * @function verify + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ShardStreamState.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id)) + return "id: integer expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } + if (message.source_shard != null && message.hasOwnProperty("source_shard")) + if (!$util.isString(message.source_shard)) + return "source_shard: string expected"; + if (message.position != null && message.hasOwnProperty("position")) + if (!$util.isString(message.position)) + return "position: string expected"; + if (message.status != null && message.hasOwnProperty("status")) + if (!$util.isString(message.status)) + return "status: string expected"; + if (message.info != null && message.hasOwnProperty("info")) + if (!$util.isString(message.info)) + return "info: string expected"; + return null; + }; - /** - * Creates a ValidateVersionShardRequest message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVersionShardRequest} ValidateVersionShardRequest - */ - ValidateVersionShardRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVersionShardRequest) - return object; - let message = new $root.vtctldata.ValidateVersionShardRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.shard != null) - message.shard = String(object.shard); - return message; - }; + /** + * Creates a ShardStreamState message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.WorkflowStatusResponse.ShardStreamState} ShardStreamState + */ + ShardStreamState.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowStatusResponse.ShardStreamState) + return object; + let message = new $root.vtctldata.WorkflowStatusResponse.ShardStreamState(); + if (object.id != null) + message.id = object.id | 0; + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.ShardStreamState.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } + if (object.source_shard != null) + message.source_shard = String(object.source_shard); + if (object.position != null) + message.position = String(object.position); + if (object.status != null) + message.status = String(object.status); + if (object.info != null) + message.info = String(object.info); + return message; + }; - /** - * Creates a plain object from a ValidateVersionShardRequest message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {vtctldata.ValidateVersionShardRequest} message ValidateVersionShardRequest - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - ValidateVersionShardRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.keyspace = ""; - object.shard = ""; - } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.shard != null && message.hasOwnProperty("shard")) - object.shard = message.shard; - return object; - }; + /** + * Creates a plain object from a ShardStreamState message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {vtctldata.WorkflowStatusResponse.ShardStreamState} message ShardStreamState + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ShardStreamState.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.id = 0; + object.tablet = null; + object.source_shard = ""; + object.position = ""; + object.status = ""; + object.info = ""; + } + if (message.id != null && message.hasOwnProperty("id")) + object.id = message.id; + if (message.tablet != null && message.hasOwnProperty("tablet")) + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); + if (message.source_shard != null && message.hasOwnProperty("source_shard")) + object.source_shard = message.source_shard; + if (message.position != null && message.hasOwnProperty("position")) + object.position = message.position; + if (message.status != null && message.hasOwnProperty("status")) + object.status = message.status; + if (message.info != null && message.hasOwnProperty("info")) + object.info = message.info; + return object; + }; - /** - * Converts this ValidateVersionShardRequest to JSON. - * @function toJSON - * @memberof vtctldata.ValidateVersionShardRequest - * @instance - * @returns {Object.} JSON object - */ - ValidateVersionShardRequest.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Converts this ShardStreamState to JSON. + * @function toJSON + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @instance + * @returns {Object.} JSON object + */ + ShardStreamState.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * Gets the default type url for ValidateVersionShardRequest - * @function getTypeUrl - * @memberof vtctldata.ValidateVersionShardRequest - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - ValidateVersionShardRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.ValidateVersionShardRequest"; - }; + /** + * Gets the default type url for ShardStreamState + * @function getTypeUrl + * @memberof vtctldata.WorkflowStatusResponse.ShardStreamState + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ShardStreamState.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.WorkflowStatusResponse.ShardStreamState"; + }; - return ValidateVersionShardRequest; - })(); + return ShardStreamState; + })(); - vtctldata.ValidateVersionShardResponse = (function() { + WorkflowStatusResponse.ShardStreams = (function() { - /** - * Properties of a ValidateVersionShardResponse. - * @memberof vtctldata - * @interface IValidateVersionShardResponse - * @property {Array.|null} [results] ValidateVersionShardResponse results - */ + /** + * Properties of a ShardStreams. + * @memberof vtctldata.WorkflowStatusResponse + * @interface IShardStreams + * @property {Array.|null} [streams] ShardStreams streams + */ - /** - * Constructs a new ValidateVersionShardResponse. - * @memberof vtctldata - * @classdesc Represents a ValidateVersionShardResponse. - * @implements IValidateVersionShardResponse - * @constructor - * @param {vtctldata.IValidateVersionShardResponse=} [properties] Properties to set - */ - function ValidateVersionShardResponse(properties) { - this.results = []; - 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]]; - } + /** + * Constructs a new ShardStreams. + * @memberof vtctldata.WorkflowStatusResponse + * @classdesc Represents a ShardStreams. + * @implements IShardStreams + * @constructor + * @param {vtctldata.WorkflowStatusResponse.IShardStreams=} [properties] Properties to set + */ + function ShardStreams(properties) { + this.streams = []; + 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]]; + } - /** - * ValidateVersionShardResponse results. - * @member {Array.} results - * @memberof vtctldata.ValidateVersionShardResponse - * @instance - */ - ValidateVersionShardResponse.prototype.results = $util.emptyArray; + /** + * ShardStreams streams. + * @member {Array.} streams + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @instance + */ + ShardStreams.prototype.streams = $util.emptyArray; - /** - * Creates a new ValidateVersionShardResponse instance using the specified properties. - * @function create - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {vtctldata.IValidateVersionShardResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse instance - */ - ValidateVersionShardResponse.create = function create(properties) { - return new ValidateVersionShardResponse(properties); - }; + /** + * Creates a new ShardStreams instance using the specified properties. + * @function create + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreams=} [properties] Properties to set + * @returns {vtctldata.WorkflowStatusResponse.ShardStreams} ShardStreams instance + */ + ShardStreams.create = function create(properties) { + return new ShardStreams(properties); + }; - /** - * Encodes the specified ValidateVersionShardResponse message. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. - * @function encode - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {vtctldata.IValidateVersionShardResponse} message ValidateVersionShardResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ValidateVersionShardResponse.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.results != null && message.results.length) - for (let i = 0; i < message.results.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); - return writer; - }; + /** + * Encodes the specified ShardStreams message. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreams.verify|verify} messages. + * @function encode + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreams} message ShardStreams message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStreams.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.streams != null && message.streams.length) + for (let i = 0; i < message.streams.length; ++i) + $root.vtctldata.WorkflowStatusResponse.ShardStreamState.encode(message.streams[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + return writer; + }; - /** - * Encodes the specified ValidateVersionShardResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVersionShardResponse.verify|verify} messages. - * @function encodeDelimited - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {vtctldata.IValidateVersionShardResponse} message ValidateVersionShardResponse message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - ValidateVersionShardResponse.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * Encodes the specified ShardStreams message, length delimited. Does not implicitly {@link vtctldata.WorkflowStatusResponse.ShardStreams.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {vtctldata.WorkflowStatusResponse.IShardStreams} message ShardStreams message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ShardStreams.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * Decodes a ValidateVersionShardResponse message from the specified reader or buffer. - * @function decode - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ValidateVersionShardResponse.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.ValidateVersionShardResponse(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - if (!(message.results && message.results.length)) - message.results = []; - message.results.push(reader.string()); + /** + * Decodes a ShardStreams message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.WorkflowStatusResponse.ShardStreams} ShardStreams + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStreams.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.WorkflowStatusResponse.ShardStreams(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 2: { + if (!(message.streams && message.streams.length)) + message.streams = []; + message.streams.push($root.vtctldata.WorkflowStatusResponse.ShardStreamState.decode(reader, reader.uint32())); + break; + } + default: + reader.skipType(tag & 7); break; } - default: - reader.skipType(tag & 7); - break; } - } - return message; - }; + return message; + }; - /** - * Decodes a ValidateVersionShardResponse message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - ValidateVersionShardResponse.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; + /** + * Decodes a ShardStreams message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.WorkflowStatusResponse.ShardStreams} ShardStreams + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ShardStreams.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Verifies a ValidateVersionShardResponse message. - * @function verify - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - ValidateVersionShardResponse.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.results != null && message.hasOwnProperty("results")) { - if (!Array.isArray(message.results)) - return "results: array expected"; - for (let i = 0; i < message.results.length; ++i) - if (!$util.isString(message.results[i])) - return "results: string[] expected"; - } - return null; - }; + /** + * Verifies a ShardStreams message. + * @function verify + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ShardStreams.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.streams != null && message.hasOwnProperty("streams")) { + if (!Array.isArray(message.streams)) + return "streams: array expected"; + for (let i = 0; i < message.streams.length; ++i) { + let error = $root.vtctldata.WorkflowStatusResponse.ShardStreamState.verify(message.streams[i]); + if (error) + return "streams." + error; + } + } + return null; + }; - /** - * Creates a ValidateVersionShardResponse message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVersionShardResponse} ValidateVersionShardResponse - */ - ValidateVersionShardResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVersionShardResponse) - return object; - let message = new $root.vtctldata.ValidateVersionShardResponse(); - if (object.results) { - if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateVersionShardResponse.results: array expected"); - message.results = []; - for (let i = 0; i < object.results.length; ++i) - message.results[i] = String(object.results[i]); - } - return message; - }; + /** + * Creates a ShardStreams message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.WorkflowStatusResponse.ShardStreams} ShardStreams + */ + ShardStreams.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowStatusResponse.ShardStreams) + return object; + let message = new $root.vtctldata.WorkflowStatusResponse.ShardStreams(); + if (object.streams) { + if (!Array.isArray(object.streams)) + throw TypeError(".vtctldata.WorkflowStatusResponse.ShardStreams.streams: array expected"); + message.streams = []; + for (let i = 0; i < object.streams.length; ++i) { + if (typeof object.streams[i] !== "object") + throw TypeError(".vtctldata.WorkflowStatusResponse.ShardStreams.streams: object expected"); + message.streams[i] = $root.vtctldata.WorkflowStatusResponse.ShardStreamState.fromObject(object.streams[i]); + } + } + return message; + }; - /** - * Creates a plain object from a ValidateVersionShardResponse message. Also converts values to other types if specified. - * @function toObject - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {vtctldata.ValidateVersionShardResponse} message ValidateVersionShardResponse - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - ValidateVersionShardResponse.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.arrays || options.defaults) - object.results = []; - if (message.results && message.results.length) { - object.results = []; - for (let j = 0; j < message.results.length; ++j) - object.results[j] = message.results[j]; - } - return object; - }; + /** + * Creates a plain object from a ShardStreams message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {vtctldata.WorkflowStatusResponse.ShardStreams} message ShardStreams + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ShardStreams.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.streams = []; + if (message.streams && message.streams.length) { + object.streams = []; + for (let j = 0; j < message.streams.length; ++j) + object.streams[j] = $root.vtctldata.WorkflowStatusResponse.ShardStreamState.toObject(message.streams[j], options); + } + return object; + }; - /** - * Converts this ValidateVersionShardResponse to JSON. - * @function toJSON - * @memberof vtctldata.ValidateVersionShardResponse - * @instance - * @returns {Object.} JSON object - */ - ValidateVersionShardResponse.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Converts this ShardStreams to JSON. + * @function toJSON + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @instance + * @returns {Object.} JSON object + */ + ShardStreams.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - /** - * Gets the default type url for ValidateVersionShardResponse - * @function getTypeUrl - * @memberof vtctldata.ValidateVersionShardResponse - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - ValidateVersionShardResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/vtctldata.ValidateVersionShardResponse"; - }; + /** + * Gets the default type url for ShardStreams + * @function getTypeUrl + * @memberof vtctldata.WorkflowStatusResponse.ShardStreams + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ShardStreams.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.WorkflowStatusResponse.ShardStreams"; + }; - return ValidateVersionShardResponse; + return ShardStreams; + })(); + + return WorkflowStatusResponse; })(); - vtctldata.ValidateVSchemaRequest = (function() { + vtctldata.WorkflowSwitchTrafficRequest = (function() { /** - * Properties of a ValidateVSchemaRequest. + * Properties of a WorkflowSwitchTrafficRequest. * @memberof vtctldata - * @interface IValidateVSchemaRequest - * @property {string|null} [keyspace] ValidateVSchemaRequest keyspace - * @property {Array.|null} [shards] ValidateVSchemaRequest shards - * @property {Array.|null} [exclude_tables] ValidateVSchemaRequest exclude_tables - * @property {boolean|null} [include_views] ValidateVSchemaRequest include_views + * @interface IWorkflowSwitchTrafficRequest + * @property {string|null} [keyspace] WorkflowSwitchTrafficRequest keyspace + * @property {string|null} [workflow] WorkflowSwitchTrafficRequest workflow + * @property {Array.|null} [cells] WorkflowSwitchTrafficRequest cells + * @property {Array.|null} [tablet_types] WorkflowSwitchTrafficRequest tablet_types + * @property {vttime.IDuration|null} [max_replication_lag_allowed] WorkflowSwitchTrafficRequest max_replication_lag_allowed + * @property {boolean|null} [enable_reverse_replication] WorkflowSwitchTrafficRequest enable_reverse_replication + * @property {number|null} [direction] WorkflowSwitchTrafficRequest direction + * @property {vttime.IDuration|null} [timeout] WorkflowSwitchTrafficRequest timeout + * @property {boolean|null} [dry_run] WorkflowSwitchTrafficRequest dry_run */ /** - * Constructs a new ValidateVSchemaRequest. + * Constructs a new WorkflowSwitchTrafficRequest. * @memberof vtctldata - * @classdesc Represents a ValidateVSchemaRequest. - * @implements IValidateVSchemaRequest + * @classdesc Represents a WorkflowSwitchTrafficRequest. + * @implements IWorkflowSwitchTrafficRequest * @constructor - * @param {vtctldata.IValidateVSchemaRequest=} [properties] Properties to set + * @param {vtctldata.IWorkflowSwitchTrafficRequest=} [properties] Properties to set */ - function ValidateVSchemaRequest(properties) { - this.shards = []; - this.exclude_tables = []; + function WorkflowSwitchTrafficRequest(properties) { + this.cells = []; + this.tablet_types = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -142629,102 +150187,155 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateVSchemaRequest keyspace. + * WorkflowSwitchTrafficRequest keyspace. * @member {string} keyspace - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @instance */ - ValidateVSchemaRequest.prototype.keyspace = ""; + WorkflowSwitchTrafficRequest.prototype.keyspace = ""; /** - * ValidateVSchemaRequest shards. - * @member {Array.} shards - * @memberof vtctldata.ValidateVSchemaRequest + * WorkflowSwitchTrafficRequest workflow. + * @member {string} workflow + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @instance */ - ValidateVSchemaRequest.prototype.shards = $util.emptyArray; + WorkflowSwitchTrafficRequest.prototype.workflow = ""; /** - * ValidateVSchemaRequest exclude_tables. - * @member {Array.} exclude_tables - * @memberof vtctldata.ValidateVSchemaRequest + * WorkflowSwitchTrafficRequest cells. + * @member {Array.} cells + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @instance */ - ValidateVSchemaRequest.prototype.exclude_tables = $util.emptyArray; + WorkflowSwitchTrafficRequest.prototype.cells = $util.emptyArray; /** - * ValidateVSchemaRequest include_views. - * @member {boolean} include_views - * @memberof vtctldata.ValidateVSchemaRequest + * WorkflowSwitchTrafficRequest tablet_types. + * @member {Array.} tablet_types + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @instance */ - ValidateVSchemaRequest.prototype.include_views = false; + WorkflowSwitchTrafficRequest.prototype.tablet_types = $util.emptyArray; /** - * Creates a new ValidateVSchemaRequest instance using the specified properties. + * WorkflowSwitchTrafficRequest max_replication_lag_allowed. + * @member {vttime.IDuration|null|undefined} max_replication_lag_allowed + * @memberof vtctldata.WorkflowSwitchTrafficRequest + * @instance + */ + WorkflowSwitchTrafficRequest.prototype.max_replication_lag_allowed = null; + + /** + * WorkflowSwitchTrafficRequest enable_reverse_replication. + * @member {boolean} enable_reverse_replication + * @memberof vtctldata.WorkflowSwitchTrafficRequest + * @instance + */ + WorkflowSwitchTrafficRequest.prototype.enable_reverse_replication = false; + + /** + * WorkflowSwitchTrafficRequest direction. + * @member {number} direction + * @memberof vtctldata.WorkflowSwitchTrafficRequest + * @instance + */ + WorkflowSwitchTrafficRequest.prototype.direction = 0; + + /** + * WorkflowSwitchTrafficRequest timeout. + * @member {vttime.IDuration|null|undefined} timeout + * @memberof vtctldata.WorkflowSwitchTrafficRequest + * @instance + */ + WorkflowSwitchTrafficRequest.prototype.timeout = null; + + /** + * WorkflowSwitchTrafficRequest dry_run. + * @member {boolean} dry_run + * @memberof vtctldata.WorkflowSwitchTrafficRequest + * @instance + */ + WorkflowSwitchTrafficRequest.prototype.dry_run = false; + + /** + * Creates a new WorkflowSwitchTrafficRequest instance using the specified properties. * @function create - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static - * @param {vtctldata.IValidateVSchemaRequest=} [properties] Properties to set - * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest instance + * @param {vtctldata.IWorkflowSwitchTrafficRequest=} [properties] Properties to set + * @returns {vtctldata.WorkflowSwitchTrafficRequest} WorkflowSwitchTrafficRequest instance */ - ValidateVSchemaRequest.create = function create(properties) { - return new ValidateVSchemaRequest(properties); + WorkflowSwitchTrafficRequest.create = function create(properties) { + return new WorkflowSwitchTrafficRequest(properties); }; /** - * Encodes the specified ValidateVSchemaRequest message. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. + * Encodes the specified WorkflowSwitchTrafficRequest message. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficRequest.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static - * @param {vtctldata.IValidateVSchemaRequest} message ValidateVSchemaRequest message or plain object to encode + * @param {vtctldata.IWorkflowSwitchTrafficRequest} message WorkflowSwitchTrafficRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVSchemaRequest.encode = function encode(message, writer) { + WorkflowSwitchTrafficRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.shards != null && message.shards.length) - for (let i = 0; i < message.shards.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.shards[i]); - if (message.exclude_tables != null && message.exclude_tables.length) - for (let i = 0; i < message.exclude_tables.length; ++i) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.exclude_tables[i]); - if (message.include_views != null && Object.hasOwnProperty.call(message, "include_views")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.include_views); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells[i]); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 4, wireType 2 =*/34).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.max_replication_lag_allowed != null && Object.hasOwnProperty.call(message, "max_replication_lag_allowed")) + $root.vttime.Duration.encode(message.max_replication_lag_allowed, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.enable_reverse_replication != null && Object.hasOwnProperty.call(message, "enable_reverse_replication")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.enable_reverse_replication); + if (message.direction != null && Object.hasOwnProperty.call(message, "direction")) + writer.uint32(/* id 7, wireType 0 =*/56).int32(message.direction); + if (message.timeout != null && Object.hasOwnProperty.call(message, "timeout")) + $root.vttime.Duration.encode(message.timeout, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); + if (message.dry_run != null && Object.hasOwnProperty.call(message, "dry_run")) + writer.uint32(/* id 9, wireType 0 =*/72).bool(message.dry_run); return writer; }; /** - * Encodes the specified ValidateVSchemaRequest message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaRequest.verify|verify} messages. + * Encodes the specified WorkflowSwitchTrafficRequest message, length delimited. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficRequest.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static - * @param {vtctldata.IValidateVSchemaRequest} message ValidateVSchemaRequest message or plain object to encode + * @param {vtctldata.IWorkflowSwitchTrafficRequest} message WorkflowSwitchTrafficRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVSchemaRequest.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowSwitchTrafficRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateVSchemaRequest message from the specified reader or buffer. + * Decodes a WorkflowSwitchTrafficRequest message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest + * @returns {vtctldata.WorkflowSwitchTrafficRequest} WorkflowSwitchTrafficRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVSchemaRequest.decode = function decode(reader, length) { + WorkflowSwitchTrafficRequest.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.ValidateVSchemaRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowSwitchTrafficRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -142733,19 +150344,44 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - if (!(message.shards && message.shards.length)) - message.shards = []; - message.shards.push(reader.string()); + message.workflow = reader.string(); break; } case 3: { - if (!(message.exclude_tables && message.exclude_tables.length)) - message.exclude_tables = []; - message.exclude_tables.push(reader.string()); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); break; } case 4: { - message.include_views = reader.bool(); + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); + break; + } + case 5: { + message.max_replication_lag_allowed = $root.vttime.Duration.decode(reader, reader.uint32()); + break; + } + case 6: { + message.enable_reverse_replication = reader.bool(); + break; + } + case 7: { + message.direction = reader.int32(); + break; + } + case 8: { + message.timeout = $root.vttime.Duration.decode(reader, reader.uint32()); + break; + } + case 9: { + message.dry_run = reader.bool(); break; } default: @@ -142757,176 +150393,291 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateVSchemaRequest message from the specified reader or buffer, length delimited. + * Decodes a WorkflowSwitchTrafficRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest + * @returns {vtctldata.WorkflowSwitchTrafficRequest} WorkflowSwitchTrafficRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVSchemaRequest.decodeDelimited = function decodeDelimited(reader) { + WorkflowSwitchTrafficRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateVSchemaRequest message. + * Verifies a WorkflowSwitchTrafficRequest message. * @function verify - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateVSchemaRequest.verify = function verify(message) { + WorkflowSwitchTrafficRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.keyspace != null && message.hasOwnProperty("keyspace")) if (!$util.isString(message.keyspace)) return "keyspace: string expected"; - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!Array.isArray(message.shards)) - return "shards: array expected"; - for (let i = 0; i < message.shards.length; ++i) - if (!$util.isString(message.shards[i])) - return "shards: string[] expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; } - if (message.exclude_tables != null && message.hasOwnProperty("exclude_tables")) { - if (!Array.isArray(message.exclude_tables)) - return "exclude_tables: array expected"; - for (let i = 0; i < message.exclude_tables.length; ++i) - if (!$util.isString(message.exclude_tables[i])) - return "exclude_tables: string[] expected"; + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } } - if (message.include_views != null && message.hasOwnProperty("include_views")) - if (typeof message.include_views !== "boolean") - return "include_views: boolean expected"; + if (message.max_replication_lag_allowed != null && message.hasOwnProperty("max_replication_lag_allowed")) { + let error = $root.vttime.Duration.verify(message.max_replication_lag_allowed); + if (error) + return "max_replication_lag_allowed." + error; + } + if (message.enable_reverse_replication != null && message.hasOwnProperty("enable_reverse_replication")) + if (typeof message.enable_reverse_replication !== "boolean") + return "enable_reverse_replication: boolean expected"; + if (message.direction != null && message.hasOwnProperty("direction")) + if (!$util.isInteger(message.direction)) + return "direction: integer expected"; + if (message.timeout != null && message.hasOwnProperty("timeout")) { + let error = $root.vttime.Duration.verify(message.timeout); + if (error) + return "timeout." + error; + } + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + if (typeof message.dry_run !== "boolean") + return "dry_run: boolean expected"; return null; }; /** - * Creates a ValidateVSchemaRequest message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowSwitchTrafficRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVSchemaRequest} ValidateVSchemaRequest + * @returns {vtctldata.WorkflowSwitchTrafficRequest} WorkflowSwitchTrafficRequest */ - ValidateVSchemaRequest.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVSchemaRequest) + WorkflowSwitchTrafficRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowSwitchTrafficRequest) return object; - let message = new $root.vtctldata.ValidateVSchemaRequest(); + let message = new $root.vtctldata.WorkflowSwitchTrafficRequest(); if (object.keyspace != null) message.keyspace = String(object.keyspace); - if (object.shards) { - if (!Array.isArray(object.shards)) - throw TypeError(".vtctldata.ValidateVSchemaRequest.shards: array expected"); - message.shards = []; - for (let i = 0; i < object.shards.length; ++i) - message.shards[i] = String(object.shards[i]); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".vtctldata.WorkflowSwitchTrafficRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); } - if (object.exclude_tables) { - if (!Array.isArray(object.exclude_tables)) - throw TypeError(".vtctldata.ValidateVSchemaRequest.exclude_tables: array expected"); - message.exclude_tables = []; - for (let i = 0; i < object.exclude_tables.length; ++i) - message.exclude_tables[i] = String(object.exclude_tables[i]); + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".vtctldata.WorkflowSwitchTrafficRequest.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } } - if (object.include_views != null) - message.include_views = Boolean(object.include_views); + if (object.max_replication_lag_allowed != null) { + if (typeof object.max_replication_lag_allowed !== "object") + throw TypeError(".vtctldata.WorkflowSwitchTrafficRequest.max_replication_lag_allowed: object expected"); + message.max_replication_lag_allowed = $root.vttime.Duration.fromObject(object.max_replication_lag_allowed); + } + if (object.enable_reverse_replication != null) + message.enable_reverse_replication = Boolean(object.enable_reverse_replication); + if (object.direction != null) + message.direction = object.direction | 0; + if (object.timeout != null) { + if (typeof object.timeout !== "object") + throw TypeError(".vtctldata.WorkflowSwitchTrafficRequest.timeout: object expected"); + message.timeout = $root.vttime.Duration.fromObject(object.timeout); + } + if (object.dry_run != null) + message.dry_run = Boolean(object.dry_run); return message; }; /** - * Creates a plain object from a ValidateVSchemaRequest message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowSwitchTrafficRequest message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static - * @param {vtctldata.ValidateVSchemaRequest} message ValidateVSchemaRequest + * @param {vtctldata.WorkflowSwitchTrafficRequest} message WorkflowSwitchTrafficRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateVSchemaRequest.toObject = function toObject(message, options) { + WorkflowSwitchTrafficRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) { - object.shards = []; - object.exclude_tables = []; + object.cells = []; + object.tablet_types = []; } if (options.defaults) { object.keyspace = ""; - object.include_views = false; + object.workflow = ""; + object.max_replication_lag_allowed = null; + object.enable_reverse_replication = false; + object.direction = 0; + object.timeout = null; + object.dry_run = false; } if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; - if (message.shards && message.shards.length) { - object.shards = []; - for (let j = 0; j < message.shards.length; ++j) - object.shards[j] = message.shards[j]; - } - if (message.exclude_tables && message.exclude_tables.length) { - object.exclude_tables = []; - for (let j = 0; j < message.exclude_tables.length; ++j) - object.exclude_tables[j] = message.exclude_tables[j]; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; } - if (message.include_views != null && message.hasOwnProperty("include_views")) - object.include_views = message.include_views; + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.max_replication_lag_allowed != null && message.hasOwnProperty("max_replication_lag_allowed")) + object.max_replication_lag_allowed = $root.vttime.Duration.toObject(message.max_replication_lag_allowed, options); + if (message.enable_reverse_replication != null && message.hasOwnProperty("enable_reverse_replication")) + object.enable_reverse_replication = message.enable_reverse_replication; + if (message.direction != null && message.hasOwnProperty("direction")) + object.direction = message.direction; + if (message.timeout != null && message.hasOwnProperty("timeout")) + object.timeout = $root.vttime.Duration.toObject(message.timeout, options); + if (message.dry_run != null && message.hasOwnProperty("dry_run")) + object.dry_run = message.dry_run; return object; }; /** - * Converts this ValidateVSchemaRequest to JSON. + * Converts this WorkflowSwitchTrafficRequest to JSON. * @function toJSON - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @instance * @returns {Object.} JSON object */ - ValidateVSchemaRequest.prototype.toJSON = function toJSON() { + WorkflowSwitchTrafficRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateVSchemaRequest + * Gets the default type url for WorkflowSwitchTrafficRequest * @function getTypeUrl - * @memberof vtctldata.ValidateVSchemaRequest + * @memberof vtctldata.WorkflowSwitchTrafficRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateVSchemaRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowSwitchTrafficRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateVSchemaRequest"; + return typeUrlPrefix + "/vtctldata.WorkflowSwitchTrafficRequest"; }; - return ValidateVSchemaRequest; + return WorkflowSwitchTrafficRequest; })(); - vtctldata.ValidateVSchemaResponse = (function() { + vtctldata.WorkflowSwitchTrafficResponse = (function() { /** - * Properties of a ValidateVSchemaResponse. + * Properties of a WorkflowSwitchTrafficResponse. * @memberof vtctldata - * @interface IValidateVSchemaResponse - * @property {Array.|null} [results] ValidateVSchemaResponse results - * @property {Object.|null} [results_by_shard] ValidateVSchemaResponse results_by_shard + * @interface IWorkflowSwitchTrafficResponse + * @property {string|null} [summary] WorkflowSwitchTrafficResponse summary + * @property {string|null} [start_state] WorkflowSwitchTrafficResponse start_state + * @property {string|null} [current_state] WorkflowSwitchTrafficResponse current_state + * @property {Array.|null} [dry_run_results] WorkflowSwitchTrafficResponse dry_run_results */ /** - * Constructs a new ValidateVSchemaResponse. + * Constructs a new WorkflowSwitchTrafficResponse. * @memberof vtctldata - * @classdesc Represents a ValidateVSchemaResponse. - * @implements IValidateVSchemaResponse + * @classdesc Represents a WorkflowSwitchTrafficResponse. + * @implements IWorkflowSwitchTrafficResponse * @constructor - * @param {vtctldata.IValidateVSchemaResponse=} [properties] Properties to set + * @param {vtctldata.IWorkflowSwitchTrafficResponse=} [properties] Properties to set */ - function ValidateVSchemaResponse(properties) { - this.results = []; - this.results_by_shard = {}; + function WorkflowSwitchTrafficResponse(properties) { + this.dry_run_results = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -142934,114 +150685,120 @@ export const vtctldata = $root.vtctldata = (() => { } /** - * ValidateVSchemaResponse results. - * @member {Array.} results - * @memberof vtctldata.ValidateVSchemaResponse + * WorkflowSwitchTrafficResponse summary. + * @member {string} summary + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @instance */ - ValidateVSchemaResponse.prototype.results = $util.emptyArray; + WorkflowSwitchTrafficResponse.prototype.summary = ""; /** - * ValidateVSchemaResponse results_by_shard. - * @member {Object.} results_by_shard - * @memberof vtctldata.ValidateVSchemaResponse + * WorkflowSwitchTrafficResponse start_state. + * @member {string} start_state + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @instance */ - ValidateVSchemaResponse.prototype.results_by_shard = $util.emptyObject; + WorkflowSwitchTrafficResponse.prototype.start_state = ""; /** - * Creates a new ValidateVSchemaResponse instance using the specified properties. + * WorkflowSwitchTrafficResponse current_state. + * @member {string} current_state + * @memberof vtctldata.WorkflowSwitchTrafficResponse + * @instance + */ + WorkflowSwitchTrafficResponse.prototype.current_state = ""; + + /** + * WorkflowSwitchTrafficResponse dry_run_results. + * @member {Array.} dry_run_results + * @memberof vtctldata.WorkflowSwitchTrafficResponse + * @instance + */ + WorkflowSwitchTrafficResponse.prototype.dry_run_results = $util.emptyArray; + + /** + * Creates a new WorkflowSwitchTrafficResponse instance using the specified properties. * @function create - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static - * @param {vtctldata.IValidateVSchemaResponse=} [properties] Properties to set - * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse instance + * @param {vtctldata.IWorkflowSwitchTrafficResponse=} [properties] Properties to set + * @returns {vtctldata.WorkflowSwitchTrafficResponse} WorkflowSwitchTrafficResponse instance */ - ValidateVSchemaResponse.create = function create(properties) { - return new ValidateVSchemaResponse(properties); + WorkflowSwitchTrafficResponse.create = function create(properties) { + return new WorkflowSwitchTrafficResponse(properties); }; /** - * Encodes the specified ValidateVSchemaResponse message. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. + * Encodes the specified WorkflowSwitchTrafficResponse message. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficResponse.verify|verify} messages. * @function encode - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static - * @param {vtctldata.IValidateVSchemaResponse} message ValidateVSchemaResponse message or plain object to encode + * @param {vtctldata.IWorkflowSwitchTrafficResponse} message WorkflowSwitchTrafficResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVSchemaResponse.encode = function encode(message, writer) { + WorkflowSwitchTrafficResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.results != null && message.results.length) - for (let i = 0; i < message.results.length; ++i) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.results[i]); - if (message.results_by_shard != null && Object.hasOwnProperty.call(message, "results_by_shard")) - for (let keys = Object.keys(message.results_by_shard), i = 0; i < keys.length; ++i) { - writer.uint32(/* id 2, wireType 2 =*/18).fork().uint32(/* id 1, wireType 2 =*/10).string(keys[i]); - $root.vtctldata.ValidateShardResponse.encode(message.results_by_shard[keys[i]], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim().ldelim(); - } + if (message.summary != null && Object.hasOwnProperty.call(message, "summary")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.summary); + if (message.start_state != null && Object.hasOwnProperty.call(message, "start_state")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.start_state); + if (message.current_state != null && Object.hasOwnProperty.call(message, "current_state")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.current_state); + if (message.dry_run_results != null && message.dry_run_results.length) + for (let i = 0; i < message.dry_run_results.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.dry_run_results[i]); return writer; }; /** - * Encodes the specified ValidateVSchemaResponse message, length delimited. Does not implicitly {@link vtctldata.ValidateVSchemaResponse.verify|verify} messages. + * Encodes the specified WorkflowSwitchTrafficResponse message, length delimited. Does not implicitly {@link vtctldata.WorkflowSwitchTrafficResponse.verify|verify} messages. * @function encodeDelimited - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static - * @param {vtctldata.IValidateVSchemaResponse} message ValidateVSchemaResponse message or plain object to encode + * @param {vtctldata.IWorkflowSwitchTrafficResponse} message WorkflowSwitchTrafficResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ValidateVSchemaResponse.encodeDelimited = function encodeDelimited(message, writer) { + WorkflowSwitchTrafficResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ValidateVSchemaResponse message from the specified reader or buffer. + * Decodes a WorkflowSwitchTrafficResponse message from the specified reader or buffer. * @function decode - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse + * @returns {vtctldata.WorkflowSwitchTrafficResponse} WorkflowSwitchTrafficResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVSchemaResponse.decode = function decode(reader, length) { + WorkflowSwitchTrafficResponse.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.ValidateVSchemaResponse(), key, value; + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.vtctldata.WorkflowSwitchTrafficResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - if (!(message.results && message.results.length)) - message.results = []; - message.results.push(reader.string()); + message.summary = reader.string(); break; } case 2: { - if (message.results_by_shard === $util.emptyObject) - message.results_by_shard = {}; - let end2 = reader.uint32() + reader.pos; - key = ""; - value = null; - while (reader.pos < end2) { - let tag2 = reader.uint32(); - switch (tag2 >>> 3) { - case 1: - key = reader.string(); - break; - case 2: - value = $root.vtctldata.ValidateShardResponse.decode(reader, reader.uint32()); - break; - default: - reader.skipType(tag2 & 7); - break; - } - } - message.results_by_shard[key] = value; + message.start_state = reader.string(); + break; + } + case 3: { + message.current_state = reader.string(); + break; + } + case 4: { + if (!(message.dry_run_results && message.dry_run_results.length)) + message.dry_run_results = []; + message.dry_run_results.push(reader.string()); break; } default: @@ -143053,142 +150810,140 @@ export const vtctldata = $root.vtctldata = (() => { }; /** - * Decodes a ValidateVSchemaResponse message from the specified reader or buffer, length delimited. + * Decodes a WorkflowSwitchTrafficResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse + * @returns {vtctldata.WorkflowSwitchTrafficResponse} WorkflowSwitchTrafficResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ValidateVSchemaResponse.decodeDelimited = function decodeDelimited(reader) { + WorkflowSwitchTrafficResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ValidateVSchemaResponse message. + * Verifies a WorkflowSwitchTrafficResponse message. * @function verify - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ValidateVSchemaResponse.verify = function verify(message) { + WorkflowSwitchTrafficResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.results != null && message.hasOwnProperty("results")) { - if (!Array.isArray(message.results)) - return "results: array expected"; - for (let i = 0; i < message.results.length; ++i) - if (!$util.isString(message.results[i])) - return "results: string[] expected"; - } - if (message.results_by_shard != null && message.hasOwnProperty("results_by_shard")) { - if (!$util.isObject(message.results_by_shard)) - return "results_by_shard: object expected"; - let key = Object.keys(message.results_by_shard); - for (let i = 0; i < key.length; ++i) { - let error = $root.vtctldata.ValidateShardResponse.verify(message.results_by_shard[key[i]]); - if (error) - return "results_by_shard." + error; - } + if (message.summary != null && message.hasOwnProperty("summary")) + if (!$util.isString(message.summary)) + return "summary: string expected"; + if (message.start_state != null && message.hasOwnProperty("start_state")) + if (!$util.isString(message.start_state)) + return "start_state: string expected"; + if (message.current_state != null && message.hasOwnProperty("current_state")) + if (!$util.isString(message.current_state)) + return "current_state: string expected"; + if (message.dry_run_results != null && message.hasOwnProperty("dry_run_results")) { + if (!Array.isArray(message.dry_run_results)) + return "dry_run_results: array expected"; + for (let i = 0; i < message.dry_run_results.length; ++i) + if (!$util.isString(message.dry_run_results[i])) + return "dry_run_results: string[] expected"; } return null; }; /** - * Creates a ValidateVSchemaResponse message from a plain object. Also converts values to their respective internal types. + * Creates a WorkflowSwitchTrafficResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static * @param {Object.} object Plain object - * @returns {vtctldata.ValidateVSchemaResponse} ValidateVSchemaResponse + * @returns {vtctldata.WorkflowSwitchTrafficResponse} WorkflowSwitchTrafficResponse */ - ValidateVSchemaResponse.fromObject = function fromObject(object) { - if (object instanceof $root.vtctldata.ValidateVSchemaResponse) + WorkflowSwitchTrafficResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowSwitchTrafficResponse) return object; - let message = new $root.vtctldata.ValidateVSchemaResponse(); - if (object.results) { - if (!Array.isArray(object.results)) - throw TypeError(".vtctldata.ValidateVSchemaResponse.results: array expected"); - message.results = []; - for (let i = 0; i < object.results.length; ++i) - message.results[i] = String(object.results[i]); - } - if (object.results_by_shard) { - if (typeof object.results_by_shard !== "object") - throw TypeError(".vtctldata.ValidateVSchemaResponse.results_by_shard: object expected"); - message.results_by_shard = {}; - for (let keys = Object.keys(object.results_by_shard), i = 0; i < keys.length; ++i) { - if (typeof object.results_by_shard[keys[i]] !== "object") - throw TypeError(".vtctldata.ValidateVSchemaResponse.results_by_shard: object expected"); - message.results_by_shard[keys[i]] = $root.vtctldata.ValidateShardResponse.fromObject(object.results_by_shard[keys[i]]); - } + let message = new $root.vtctldata.WorkflowSwitchTrafficResponse(); + if (object.summary != null) + message.summary = String(object.summary); + if (object.start_state != null) + message.start_state = String(object.start_state); + if (object.current_state != null) + message.current_state = String(object.current_state); + if (object.dry_run_results) { + if (!Array.isArray(object.dry_run_results)) + throw TypeError(".vtctldata.WorkflowSwitchTrafficResponse.dry_run_results: array expected"); + message.dry_run_results = []; + for (let i = 0; i < object.dry_run_results.length; ++i) + message.dry_run_results[i] = String(object.dry_run_results[i]); } return message; }; /** - * Creates a plain object from a ValidateVSchemaResponse message. Also converts values to other types if specified. + * Creates a plain object from a WorkflowSwitchTrafficResponse message. Also converts values to other types if specified. * @function toObject - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static - * @param {vtctldata.ValidateVSchemaResponse} message ValidateVSchemaResponse + * @param {vtctldata.WorkflowSwitchTrafficResponse} message WorkflowSwitchTrafficResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ValidateVSchemaResponse.toObject = function toObject(message, options) { + WorkflowSwitchTrafficResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) - object.results = []; - if (options.objects || options.defaults) - object.results_by_shard = {}; - if (message.results && message.results.length) { - object.results = []; - for (let j = 0; j < message.results.length; ++j) - object.results[j] = message.results[j]; + object.dry_run_results = []; + if (options.defaults) { + object.summary = ""; + object.start_state = ""; + object.current_state = ""; } - let keys2; - if (message.results_by_shard && (keys2 = Object.keys(message.results_by_shard)).length) { - object.results_by_shard = {}; - for (let j = 0; j < keys2.length; ++j) - object.results_by_shard[keys2[j]] = $root.vtctldata.ValidateShardResponse.toObject(message.results_by_shard[keys2[j]], options); + if (message.summary != null && message.hasOwnProperty("summary")) + object.summary = message.summary; + if (message.start_state != null && message.hasOwnProperty("start_state")) + object.start_state = message.start_state; + if (message.current_state != null && message.hasOwnProperty("current_state")) + object.current_state = message.current_state; + if (message.dry_run_results && message.dry_run_results.length) { + object.dry_run_results = []; + for (let j = 0; j < message.dry_run_results.length; ++j) + object.dry_run_results[j] = message.dry_run_results[j]; } return object; }; /** - * Converts this ValidateVSchemaResponse to JSON. + * Converts this WorkflowSwitchTrafficResponse to JSON. * @function toJSON - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @instance * @returns {Object.} JSON object */ - ValidateVSchemaResponse.prototype.toJSON = function toJSON() { + WorkflowSwitchTrafficResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ValidateVSchemaResponse + * Gets the default type url for WorkflowSwitchTrafficResponse * @function getTypeUrl - * @memberof vtctldata.ValidateVSchemaResponse + * @memberof vtctldata.WorkflowSwitchTrafficResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ValidateVSchemaResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + WorkflowSwitchTrafficResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/vtctldata.ValidateVSchemaResponse"; + return typeUrlPrefix + "/vtctldata.WorkflowSwitchTrafficResponse"; }; - return ValidateVSchemaResponse; + return WorkflowSwitchTrafficResponse; })(); vtctldata.WorkflowUpdateRequest = (function() { @@ -143198,7 +150953,7 @@ export const vtctldata = $root.vtctldata = (() => { * @memberof vtctldata * @interface IWorkflowUpdateRequest * @property {string|null} [keyspace] WorkflowUpdateRequest keyspace - * @property {tabletmanagerdata.IUpdateVRWorkflowRequest|null} [tablet_request] WorkflowUpdateRequest tablet_request + * @property {tabletmanagerdata.IUpdateVReplicationWorkflowRequest|null} [tablet_request] WorkflowUpdateRequest tablet_request */ /** @@ -143226,7 +150981,7 @@ export const vtctldata = $root.vtctldata = (() => { /** * WorkflowUpdateRequest tablet_request. - * @member {tabletmanagerdata.IUpdateVRWorkflowRequest|null|undefined} tablet_request + * @member {tabletmanagerdata.IUpdateVReplicationWorkflowRequest|null|undefined} tablet_request * @memberof vtctldata.WorkflowUpdateRequest * @instance */ @@ -143259,7 +151014,7 @@ export const vtctldata = $root.vtctldata = (() => { if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); if (message.tablet_request != null && Object.hasOwnProperty.call(message, "tablet_request")) - $root.tabletmanagerdata.UpdateVRWorkflowRequest.encode(message.tablet_request, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest.encode(message.tablet_request, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); return writer; }; @@ -143299,7 +151054,7 @@ export const vtctldata = $root.vtctldata = (() => { break; } case 2: { - message.tablet_request = $root.tabletmanagerdata.UpdateVRWorkflowRequest.decode(reader, reader.uint32()); + message.tablet_request = $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest.decode(reader, reader.uint32()); break; } default: @@ -143341,7 +151096,7 @@ export const vtctldata = $root.vtctldata = (() => { if (!$util.isString(message.keyspace)) return "keyspace: string expected"; if (message.tablet_request != null && message.hasOwnProperty("tablet_request")) { - let error = $root.tabletmanagerdata.UpdateVRWorkflowRequest.verify(message.tablet_request); + let error = $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify(message.tablet_request); if (error) return "tablet_request." + error; } @@ -143365,7 +151120,7 @@ export const vtctldata = $root.vtctldata = (() => { if (object.tablet_request != null) { if (typeof object.tablet_request !== "object") throw TypeError(".vtctldata.WorkflowUpdateRequest.tablet_request: object expected"); - message.tablet_request = $root.tabletmanagerdata.UpdateVRWorkflowRequest.fromObject(object.tablet_request); + message.tablet_request = $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest.fromObject(object.tablet_request); } return message; }; @@ -143390,7 +151145,7 @@ export const vtctldata = $root.vtctldata = (() => { if (message.keyspace != null && message.hasOwnProperty("keyspace")) object.keyspace = message.keyspace; if (message.tablet_request != null && message.hasOwnProperty("tablet_request")) - object.tablet_request = $root.tabletmanagerdata.UpdateVRWorkflowRequest.toObject(message.tablet_request, options); + object.tablet_request = $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest.toObject(message.tablet_request, options); return object; }; @@ -143674,7 +151429,7 @@ export const vtctldata = $root.vtctldata = (() => { * Properties of a TabletInfo. * @memberof vtctldata.WorkflowUpdateResponse * @interface ITabletInfo - * @property {string|null} [tablet] TabletInfo tablet + * @property {topodata.ITabletAlias|null} [tablet] TabletInfo tablet * @property {boolean|null} [changed] TabletInfo changed */ @@ -143695,11 +151450,11 @@ export const vtctldata = $root.vtctldata = (() => { /** * TabletInfo tablet. - * @member {string} tablet + * @member {topodata.ITabletAlias|null|undefined} tablet * @memberof vtctldata.WorkflowUpdateResponse.TabletInfo * @instance */ - TabletInfo.prototype.tablet = ""; + TabletInfo.prototype.tablet = null; /** * TabletInfo changed. @@ -143734,7 +151489,7 @@ export const vtctldata = $root.vtctldata = (() => { if (!writer) writer = $Writer.create(); if (message.tablet != null && Object.hasOwnProperty.call(message, "tablet")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tablet); + $root.topodata.TabletAlias.encode(message.tablet, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); if (message.changed != null && Object.hasOwnProperty.call(message, "changed")) writer.uint32(/* id 2, wireType 0 =*/16).bool(message.changed); return writer; @@ -143772,7 +151527,7 @@ export const vtctldata = $root.vtctldata = (() => { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet = reader.string(); + message.tablet = $root.topodata.TabletAlias.decode(reader, reader.uint32()); break; } case 2: { @@ -143814,9 +151569,11 @@ export const vtctldata = $root.vtctldata = (() => { TabletInfo.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet != null && message.hasOwnProperty("tablet")) - if (!$util.isString(message.tablet)) - return "tablet: string expected"; + if (message.tablet != null && message.hasOwnProperty("tablet")) { + let error = $root.topodata.TabletAlias.verify(message.tablet); + if (error) + return "tablet." + error; + } if (message.changed != null && message.hasOwnProperty("changed")) if (typeof message.changed !== "boolean") return "changed: boolean expected"; @@ -143835,8 +151592,11 @@ export const vtctldata = $root.vtctldata = (() => { if (object instanceof $root.vtctldata.WorkflowUpdateResponse.TabletInfo) return object; let message = new $root.vtctldata.WorkflowUpdateResponse.TabletInfo(); - if (object.tablet != null) - message.tablet = String(object.tablet); + if (object.tablet != null) { + if (typeof object.tablet !== "object") + throw TypeError(".vtctldata.WorkflowUpdateResponse.TabletInfo.tablet: object expected"); + message.tablet = $root.topodata.TabletAlias.fromObject(object.tablet); + } if (object.changed != null) message.changed = Boolean(object.changed); return message; @@ -143856,11 +151616,11 @@ export const vtctldata = $root.vtctldata = (() => { options = {}; let object = {}; if (options.defaults) { - object.tablet = ""; + object.tablet = null; object.changed = false; } if (message.tablet != null && message.hasOwnProperty("tablet")) - object.tablet = message.tablet; + object.tablet = $root.topodata.TabletAlias.toObject(message.tablet, options); if (message.changed != null && message.hasOwnProperty("changed")) object.changed = message.changed; return object;